TypeDescription convertTypeFromProtobuf()

in java/core/src/java/org/apache/orc/OrcUtils.java [269:373]


        TypeDescription convertTypeFromProtobuf(List<OrcProto.Type> types,
                                                int rootColumn)
          throws FileFormatException {
    OrcProto.Type type = types.get(rootColumn);
    TypeDescription result;
    switch (type.getKind()) {
      case BOOLEAN:
        result = TypeDescription.createBoolean();
        break;
      case BYTE:
        result = TypeDescription.createByte();
        break;
      case SHORT:
        result = TypeDescription.createShort();
        break;
      case INT:
        result = TypeDescription.createInt();
        break;
      case LONG:
        result = TypeDescription.createLong();
        break;
      case FLOAT:
        result = TypeDescription.createFloat();
        break;
      case DOUBLE:
        result = TypeDescription.createDouble();
        break;
      case STRING:
        result = TypeDescription.createString();
        break;
      case CHAR:
      case VARCHAR:
        result = type.getKind() == OrcProto.Type.Kind.CHAR ?
            TypeDescription.createChar() : TypeDescription.createVarchar();
        if (type.hasMaximumLength()) {
          result.withMaxLength(type.getMaximumLength());
        }
        break;
      case BINARY:
        result = TypeDescription.createBinary();
        break;
      case TIMESTAMP:
        result = TypeDescription.createTimestamp();
        break;
      case TIMESTAMP_INSTANT:
        result = TypeDescription.createTimestampInstant();
        break;
      case DATE:
        result = TypeDescription.createDate();
        break;
      case DECIMAL:
        result = TypeDescription.createDecimal();
        if (type.hasScale()) {
          result.withScale(type.getScale());
        }
        if (type.hasPrecision()) {
          result.withPrecision(type.getPrecision());
        }
        break;
      case LIST:
        if (type.getSubtypesCount() != 1) {
          throw new FileFormatException("LIST type should contain exactly " +
                  "one subtype but has " + type.getSubtypesCount());
        }
        result = TypeDescription.createList(
            convertTypeFromProtobuf(types, type.getSubtypes(0)));
        break;
      case MAP:
        if (type.getSubtypesCount() != 2) {
          throw new FileFormatException("MAP type should contain exactly " +
                  "two subtypes but has " + type.getSubtypesCount());
        }
        result = TypeDescription.createMap(
            convertTypeFromProtobuf(types, type.getSubtypes(0)),
            convertTypeFromProtobuf(types, type.getSubtypes(1)));
        break;
      case STRUCT:
        result = TypeDescription.createStruct();
        for(int f=0; f < type.getSubtypesCount(); ++f) {
          String name = type.getFieldNames(f);
          name = name.startsWith("`") ? name : "`" + name + "`";
          String fieldName = ParserUtils.parseName(new ParserUtils.StringPosition(name));
          result.addField(fieldName, convertTypeFromProtobuf(types, type.getSubtypes(f)));
        }
        break;
      case UNION:
        if (type.getSubtypesCount() == 0) {
          throw new FileFormatException("UNION type should contain at least" +
                " one subtype but has none");
        }
        result = TypeDescription.createUnion();
        for(int f=0; f < type.getSubtypesCount(); ++f) {
          result.addUnionChild(
              convertTypeFromProtobuf(types, type.getSubtypes(f)));
        }
        break;
      default:
        throw new IllegalArgumentException("Unknown ORC type " + type.getKind());
    }
    for(int i = 0; i < type.getAttributesCount(); ++i) {
      OrcProto.StringPair pair = type.getAttributes(i);
      result.setAttribute(pair.getKey(), pair.getValue());
    }
    return result;
  }