You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2019/11/12 21:17:31 UTC

[GitHub] [incubator-iceberg] shardulm94 commented on a change in pull request #227: ORC column map fix

shardulm94 commented on a change in pull request #227: ORC column map fix
URL: https://github.com/apache/incubator-iceberg/pull/227#discussion_r345448631
 
 

 ##########
 File path: orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java
 ##########
 @@ -0,0 +1,488 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.orc;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.TypeDescription;
+
+/**
+ * Utilities for mapping Iceberg to ORC schemas.
+ */
+public final class ORCSchemaUtil {
+
+  private enum BinaryType {
+    UUID, FIXED, BINARY
+  }
+
+  private enum IntegerType {
+    TIME, INTEGER
+  }
+
+  private static class OrcField {
+    private final String name;
+    private final TypeDescription type;
+
+    OrcField(String name, TypeDescription type) {
+      this.name = name;
+      this.type = type;
+    }
+
+    public String name() {
+      return name;
+    }
+
+    public TypeDescription type() {
+      return type;
+    }
+  }
+
+  private static final String ICEBERG_ID_ATTRIBUTE = "iceberg.id";
+  private static final String ICEBERG_REQUIRED_ATTRIBUTE = "iceberg.required";
+
+  private static final String ICEBERG_BINARY_TYPE_ATTRIBUTE = "iceberg.binary-type";
+  private static final String ICEBERG_INTEGER_TYPE_ATTRIBUTE = "iceberg.integer-type";
+  private static final String ICEBERG_FIELD_LENGTH = "iceberg.length";
+
+  private static final Map<Type.TypeID, TypeDescription.Category> TYPE_MAPPING =
+      ImmutableMap.<Type.TypeID, TypeDescription.Category>builder()
+          .put(Type.TypeID.BOOLEAN, TypeDescription.Category.BOOLEAN)
+          .put(Type.TypeID.INTEGER, TypeDescription.Category.INT)
+          .put(Type.TypeID.TIME, TypeDescription.Category.INT)
+          .put(Type.TypeID.LONG, TypeDescription.Category.LONG)
+          .put(Type.TypeID.FLOAT, TypeDescription.Category.FLOAT)
+          .put(Type.TypeID.DOUBLE, TypeDescription.Category.DOUBLE)
+          .put(Type.TypeID.DATE, TypeDescription.Category.DATE)
+          .put(Type.TypeID.TIMESTAMP, TypeDescription.Category.TIMESTAMP)
+          .put(Type.TypeID.STRING, TypeDescription.Category.STRING)
+          .put(Type.TypeID.UUID, TypeDescription.Category.BINARY)
+          .put(Type.TypeID.FIXED, TypeDescription.Category.BINARY)
+          .put(Type.TypeID.BINARY, TypeDescription.Category.BINARY)
+          .put(Type.TypeID.DECIMAL, TypeDescription.Category.DECIMAL)
+          .build();
+
+  private ORCSchemaUtil() {}
+
+  public static TypeDescription convert(Schema schema) {
+    final TypeDescription root = TypeDescription.createStruct();
+    final Types.StructType schemaRoot = schema.asStruct();
+    for (Types.NestedField field : schemaRoot.asStructType().fields()) {
+      TypeDescription orcColumType = convert(field.fieldId(), field.type(), field.isRequired());
+      root.addField(field.name(), orcColumType);
+    }
+    return root;
+  }
+
+  private static TypeDescription convert(Integer fieldId, Type type, boolean isRequired) {
+    final TypeDescription orcType;
+
+    switch (type.typeId()) {
+      case BOOLEAN:
+        orcType = TypeDescription.createBoolean();
+        break;
+      case INTEGER:
+        orcType = TypeDescription.createInt();
+        orcType.setAttribute(ICEBERG_INTEGER_TYPE_ATTRIBUTE, IntegerType.INTEGER.toString());
+        break;
+      case TIME:
+        orcType = TypeDescription.createInt();
+        orcType.setAttribute(ICEBERG_INTEGER_TYPE_ATTRIBUTE, IntegerType.TIME.toString());
+        break;
+      case LONG:
+        orcType = TypeDescription.createLong();
+        break;
+      case FLOAT:
+        orcType = TypeDescription.createFloat();
+        break;
+      case DOUBLE:
+        orcType = TypeDescription.createDouble();
+        break;
+      case DATE:
+        orcType = TypeDescription.createDate();
+        break;
+      case TIMESTAMP:
+        orcType = TypeDescription.createTimestamp();
+        break;
+      case STRING:
+        orcType = TypeDescription.createString();
+        break;
+      case UUID:
+        orcType = TypeDescription.createBinary();
+        orcType.setAttribute(ICEBERG_BINARY_TYPE_ATTRIBUTE, BinaryType.UUID.toString());
+        break;
+      case FIXED:
+        orcType = TypeDescription.createBinary();
+        orcType.setAttribute(ICEBERG_BINARY_TYPE_ATTRIBUTE, BinaryType.FIXED.toString());
+        orcType.setAttribute(ICEBERG_FIELD_LENGTH, Integer.toString(((Types.FixedType) type).length()));
+        break;
+      case BINARY:
+        orcType = TypeDescription.createBinary();
+        orcType.setAttribute(ICEBERG_BINARY_TYPE_ATTRIBUTE, BinaryType.BINARY.toString());
+        break;
+      case DECIMAL: {
+        Types.DecimalType decimal = (Types.DecimalType) type;
+        orcType = TypeDescription.createDecimal()
+            .withScale(decimal.scale())
+            .withPrecision(decimal.precision());
+        break;
+      }
+      case STRUCT: {
+        orcType = TypeDescription.createStruct();
+        for (Types.NestedField field : type.asStructType().fields()) {
+          TypeDescription childType = convert(field.fieldId(), field.type(), field.isRequired());
+          orcType.addField(field.name(), childType);
+        }
+        break;
+      }
+      case LIST: {
+        Types.ListType list = (Types.ListType) type;
+        TypeDescription elementType = convert(list.elementId(), list.elementType(),
+            list.isElementRequired());
+        orcType = TypeDescription.createList(elementType);
+        break;
+      }
+      case MAP: {
+        Types.MapType map = (Types.MapType) type;
+        TypeDescription keyType = convert(map.keyId(), map.keyType(), true);
+        TypeDescription valueType = convert(map.valueId(), map.valueType(), map.isValueRequired());
+        orcType = TypeDescription.createMap(keyType, valueType);
+        break;
+      }
+      default:
+        throw new IllegalArgumentException("Unhandled type " + type.typeId());
+    }
+
+    // Set Iceberg column attributes for mapping
+    orcType.setAttribute(ICEBERG_ID_ATTRIBUTE, String.valueOf(fieldId));
+    orcType.setAttribute(ICEBERG_REQUIRED_ATTRIBUTE, String.valueOf(isRequired));
+    return orcType;
+  }
+
+  /**
+   * Convert an ORC schema to an Iceberg schema. This method handles the convertion from the original
+   * Iceberg column mapping IDs if present in the ORC column attributes, otherwise, ORC column IDs
+   * will be assigned following ORCs pre-order ID assignment.
+   *
+   * @return the Iceberg schema
+   */
+  public static Schema convert(TypeDescription orcSchema) {
+    List<TypeDescription> children = orcSchema.getChildren();
+    List<String> childrenNames = orcSchema.getFieldNames();
+    Preconditions.checkState(children.size() == childrenNames.size(),
+        "Error in ORC file, children fields and names do not match.");
+
+    List<Types.NestedField> icebergFields = Lists.newArrayListWithExpectedSize(children.size());
+    AtomicInteger lastColumnId = new AtomicInteger(orcSchema.getMaximumId());
+    for (int i = 0; i < children.size(); i++) {
+      icebergFields.add(convertOrcToIceberg(children.get(i), childrenNames.get(i),
+          lastColumnId::incrementAndGet));
+    }
+
+    return new Schema(icebergFields);
+  }
+
+  /**
+   * Converts an Iceberg schema to a corresponding ORC schema within the context of an existing
+   * ORC file schema.
+   * This method also handles schema evolution from the original ORC file schema
+   * to the given Iceberg schema. It builds the desired reader schema with the schema
+   * evolution rules and pass that down to the ORC reader,
+   * which would then use its schema evolution to map that to the writer’s schema.
+   *
+   * Example:
+   * <code>
+   * Iceberg writer                                        ORC writer
+   * struct&lt;a (1): int, b (2): string&gt;                     struct&lt;a: int, b: string&gt;
+   * struct&lt;a (1): struct&lt;b (2): string, c (3): date&gt;&gt;     struct&lt;a: struct&lt;b:string, c:date&gt;&gt;
+   * </code>
+   *
+   * Iceberg reader                                        ORC reader
+   * <code>
+   * struct&lt;a (2): string, c (3): date&gt;                    struct&lt;b: string, c: date&gt;
+   * struct&lt;aa (1): struct&lt;cc (3): date, bb (2): string&gt;&gt;  struct&lt;a: struct&lt;c:date, b:string&gt;&gt;
+   * </code>
+   *
+   * @param schema an Iceberg schema
+   * @param originalOrcSchema an existing ORC file schema
+   * @return the resulting ORC schema
+   */
+  public static TypeDescription buildOrcProjection(Schema schema,
+                                                   TypeDescription originalOrcSchema) {
+    AtomicInteger lastColumnId = new AtomicInteger(getMaxIcebergId(originalOrcSchema));
+    final Map<Integer, OrcField> icebergToOrc = icebergToOrcMapping("root",
+        originalOrcSchema, lastColumnId::incrementAndGet);
+    return buildOrcProjection(Integer.MIN_VALUE, schema.asStruct(), true, icebergToOrc);
+  }
+
+  private static TypeDescription buildOrcProjection(Integer fieldId, Type type, boolean isRequired,
+                                                    Map<Integer, OrcField> mapping) {
+    final TypeDescription orcType;
+
+    switch (type.typeId()) {
+      case STRUCT:
+        orcType = TypeDescription.createStruct();
+        for (Types.NestedField nestedField : type.asStructType().fields()) {
+          String name = Optional.ofNullable(mapping.get(nestedField.fieldId()))
+              .map(OrcField::name)
+              .orElse(nestedField.name());
+          TypeDescription childType = buildOrcProjection(nestedField.fieldId(), nestedField.type(),
+              nestedField.isRequired(), mapping);
+          orcType.addField(name, childType);
+        }
+        break;
+      case LIST:
+        Types.ListType list = (Types.ListType) type;
+        TypeDescription elementType = buildOrcProjection(list.elementId(), list.elementType(),
+            list.isElementRequired(), mapping);
+        orcType = TypeDescription.createList(elementType);
+        break;
+      case MAP:
+        Types.MapType map = (Types.MapType) type;
+        TypeDescription keyType = buildOrcProjection(map.keyId(), map.keyType(), true, mapping);
+        TypeDescription valueType = buildOrcProjection(map.valueId(), map.valueType(), map.isValueRequired(),
+            mapping);
+        orcType = TypeDescription.createMap(keyType, valueType);
+        break;
+      default:
+        if (mapping.containsKey(fieldId)) {
+          TypeDescription originalType = mapping.get(fieldId).type();
+          Optional<TypeDescription> promotedType = getPromotedType(type, originalType);
+
+          if (promotedType.isPresent()) {
+            orcType = promotedType.get();
+          } else {
+            Preconditions.checkArgument(isSameType(originalType, type),
+                "Can not promote %s type to %s",
+                originalType.getCategory(), type.typeId().name());
+            orcType = originalType.clone();
+          }
+        } else {
+          if (isRequired) {
+            throw new IllegalArgumentException(
+                String.format("Field %d of type %s is required and was not found.", fieldId, type.toString()));
+          }
+
+          orcType = convert(fieldId, type, isRequired);
+        }
+    }
+
+    return orcType;
+  }
+
+  private static Map<Integer, OrcField> icebergToOrcMapping(String name, TypeDescription orcType,
+                                                            TypeUtil.NextID nextID) {
+    Map<Integer, OrcField> icebergToOrc = Maps.newHashMap();
+    switch (orcType.getCategory()) {
+      case STRUCT:
+        List<String> childrenNames = orcType.getFieldNames();
+        List<TypeDescription> children = orcType.getChildren();
+        for (int i = 0; i < children.size(); i++) {
+          icebergToOrc.putAll(icebergToOrcMapping(childrenNames.get(i), children.get(i), nextID));
+        }
+        break;
+      case LIST:
+        icebergToOrc.putAll(icebergToOrcMapping("element", orcType.getChildren().get(0), nextID));
+        break;
+      case MAP:
+        icebergToOrc.putAll(icebergToOrcMapping("key", orcType.getChildren().get(0), nextID));
+        icebergToOrc.putAll(icebergToOrcMapping("value", orcType.getChildren().get(1), nextID));
+        break;
+    }
+
+    if (orcType.getId() > 0) {
+      // Only add to non-root types.
+      int icebergId = icebergID(orcType).orElseGet(nextID::get);
 
 Review comment:
   If the field id is missing, we seem to map based on the pre-order ID assignment. However these ids are completely different from Iceberg ids as they are not necessarily pre-order. This causes issues while reading the dataset as fields are not mapped correctly. IIRC in Avro at least, if the field id is missing for a particular field, we do not project the field. Should the same be done here? 
   (In Avro, if file has no ids at all, then we do name-based matching or we also have to option to use name mapping, but that can be done separately).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org