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/01/22 01:59:05 UTC

[GitHub] mccheah commented on a change in pull request #63: Use Iceberg writers for Parquet data written from Spark.

mccheah commented on a change in pull request #63: Use Iceberg writers for Parquet data written from Spark.
URL: https://github.com/apache/incubator-iceberg/pull/63#discussion_r249618843
 
 

 ##########
 File path: spark/src/main/java/com/netflix/iceberg/spark/data/SparkParquetWriters.java
 ##########
 @@ -0,0 +1,447 @@
+/*
+ * 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 com.netflix.iceberg.spark.data;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.netflix.iceberg.Schema;
+import com.netflix.iceberg.parquet.ParquetTypeVisitor;
+import com.netflix.iceberg.parquet.ParquetValueReaders.ReusableEntry;
+import com.netflix.iceberg.parquet.ParquetValueWriter;
+import com.netflix.iceberg.parquet.ParquetValueWriters;
+import com.netflix.iceberg.parquet.ParquetValueWriters.PrimitiveWriter;
+import com.netflix.iceberg.parquet.ParquetValueWriters.RepeatedKeyValueWriter;
+import com.netflix.iceberg.parquet.ParquetValueWriters.RepeatedWriter;
+import com.netflix.iceberg.types.TypeUtil;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.util.ArrayData;
+import org.apache.spark.sql.catalyst.util.MapData;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.unsafe.types.UTF8String;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+import static com.netflix.iceberg.parquet.ParquetValueWriters.option;
+import static com.netflix.iceberg.spark.SparkSchemaUtil.convert;
+
+public class SparkParquetWriters {
+  private SparkParquetWriters() {
+  }
+
+  @SuppressWarnings("unchecked")
+  public static <T> ParquetValueWriter<T> buildWriter(Schema schema, MessageType type) {
+    return (ParquetValueWriter<T>) ParquetTypeVisitor.visit(type, new WriteBuilder(schema, type));
+  }
+
+  private static class WriteBuilder extends ParquetTypeVisitor<ParquetValueWriter<?>> {
+    private final Schema schema;
+    private final MessageType type;
+
+    WriteBuilder(Schema schema, MessageType type) {
+      this.schema = schema;
+      this.type = type;
+    }
+
+    @Override
+    public ParquetValueWriter<?> message(MessageType message,
+                                         List<ParquetValueWriter<?>> fieldWriters) {
+      return struct(message.asGroupType(), fieldWriters);
+    }
+
+    @Override
+    public ParquetValueWriter<?> struct(GroupType struct,
+                                        List<ParquetValueWriter<?>> fieldWriters) {
+      List<Type> fields = struct.getFields();
+      List<ParquetValueWriter<?>> writers = Lists.newArrayListWithExpectedSize(fieldWriters.size());
+      List<DataType> sparkTypes = Lists.newArrayList();
+      for (int i = 0; i < fields.size(); i += 1) {
+        Type fieldType = struct.getType(i);
+        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName()));
+        writers.add(option(fieldType, fieldD, fieldWriters.get(i)));
+        sparkTypes.add(convert(schema.findType(fieldType.getId().intValue())));
+      }
+
+      return new InternalRowWriter(writers, sparkTypes);
+    }
+
+    @Override
+    public ParquetValueWriter<?> list(GroupType array, ParquetValueWriter<?> elementWriter) {
+      GroupType repeated = array.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath);
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath);
+
+      org.apache.parquet.schema.Type elementType = repeated.getType(0);
+      int elementD = type.getMaxDefinitionLevel(path(elementType.getName()));
+
+      DataType elementSparkType = convert(schema.findType(elementType.getId().intValue()));
+
+      return new ArrayDataWriter<>(repeatedD, repeatedR,
+          option(elementType, elementD, elementWriter),
+          elementSparkType);
+    }
+
+    @Override
+    public ParquetValueWriter<?> map(GroupType map,
+                                     ParquetValueWriter<?> keyWriter,
+                                     ParquetValueWriter<?> valueWriter) {
+      GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
+      String[] repeatedPath = currentPath();
+
+      int repeatedD = type.getMaxDefinitionLevel(repeatedPath);
+      int repeatedR = type.getMaxRepetitionLevel(repeatedPath);
+
+      org.apache.parquet.schema.Type keyType = repeatedKeyValue.getType(0);
+      int keyD = type.getMaxDefinitionLevel(path(keyType.getName()));
+      DataType keySparkType = convert(schema.findType(keyType.getId().intValue()));
+      org.apache.parquet.schema.Type valueType = repeatedKeyValue.getType(1);
+      int valueD = type.getMaxDefinitionLevel(path(valueType.getName()));
+      DataType valueSparkType = convert(schema.findType(valueType.getId().intValue()));
+
+      return new MapDataWriter<>(repeatedD, repeatedR,
+          option(keyType, keyD, keyWriter), option(valueType, valueD, valueWriter),
+          keySparkType, valueSparkType);
+    }
+
+    @Override
+    public ParquetValueWriter<?> primitive(PrimitiveType primitive) {
+      ColumnDescriptor desc = type.getColumnDescription(currentPath());
+
+      if (primitive.getOriginalType() != null) {
+        switch (primitive.getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+            return utf8Strings(desc);
+          case DATE:
+          case INT_8:
+          case INT_16:
+          case INT_32:
+          case INT_64:
+          case TIME_MICROS:
+          case TIMESTAMP_MICROS:
+            return ParquetValueWriters.unboxed(desc);
+          case DECIMAL:
+            DecimalMetadata decimal = primitive.getDecimalMetadata();
+            switch (primitive.getPrimitiveTypeName()) {
+              case INT32:
+                return decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale());
+              case INT64:
+                return decimalAsLong(desc, decimal.getPrecision(), decimal.getScale());
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale());
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          case BSON:
+            return byteArrays(desc);
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      }
+
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return byteArrays(desc);
+        case BOOLEAN:
+        case INT32:
+        case INT64:
+        case FLOAT:
+        case DOUBLE:
+          return ParquetValueWriters.unboxed(desc);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+
+    private String[] currentPath() {
+      String[] path = new String[fieldNames.size()];
+      if (!fieldNames.isEmpty()) {
+        Iterator<String> iter = fieldNames.descendingIterator();
+        for (int i = 0; iter.hasNext(); i += 1) {
+          path[i] = iter.next();
+        }
+      }
+
+      return path;
+    }
+
+    private String[] path(String name) {
+      String[] path = new String[fieldNames.size() + 1];
+      path[fieldNames.size()] = name;
+
+      if (!fieldNames.isEmpty()) {
+        Iterator<String> iter = fieldNames.descendingIterator();
+        for (int i = 0; iter.hasNext(); i += 1) {
+          path[i] = iter.next();
+        }
+      }
+
+      return path;
+    }
+  }
+
+  private static PrimitiveWriter<UTF8String> utf8Strings(ColumnDescriptor desc) {
+    return new UTF8StringWriter(desc);
+  }
+
+  private static PrimitiveWriter<Decimal> decimalAsInteger(ColumnDescriptor desc,
+                                                           int precision, int scale) {
+    return new IntegerDecimalWriter(desc, precision, scale);
+  }
+
+  private static PrimitiveWriter<Decimal> decimalAsLong(ColumnDescriptor desc,
+                                                        int precision, int scale) {
+    return new LongDecimalWriter(desc, precision, scale);
+  }
+
+  private static PrimitiveWriter<Decimal> decimalAsFixed(ColumnDescriptor desc,
+                                                         int precision, int scale) {
+    return new FixedDecimalWriter(desc, precision, scale);
+  }
+
+  private static PrimitiveWriter<byte[]> byteArrays(ColumnDescriptor desc) {
+    return new ByteArrayWriter(desc);
+  }
+
+  private static class UTF8StringWriter extends PrimitiveWriter<UTF8String> {
+    private UTF8StringWriter(ColumnDescriptor desc) {
+      super(desc);
+    }
+
+    @Override
+    public void write(int repetitionLevel, UTF8String value) {
+      column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(value.getBytes()));
+    }
+  }
+
+  private static class IntegerDecimalWriter extends PrimitiveWriter<Decimal> {
+    private final int precision;
+    private final int scale;
+
+    private IntegerDecimalWriter(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public void write(int repetitionLevel, Decimal decimal) {
+      Preconditions.checkArgument(decimal.scale() == scale,
+          "Cannot write value as decimal(%s,%s), wrong scale: %s", precision, scale, decimal);
+      Preconditions.checkArgument(decimal.precision() <= precision,
+          "Cannot write value as decimal(%s,%s), too large: %s", precision, scale, decimal);
+
+      column.writeInteger(repetitionLevel, (int) decimal.toUnscaledLong());
+    }
+  }
+
+  private static class LongDecimalWriter extends PrimitiveWriter<Decimal> {
+    private final int precision;
+    private final int scale;
+
+    private LongDecimalWriter(ColumnDescriptor desc, int precision, int scale) {
+      super(desc);
+      this.precision = precision;
+      this.scale = scale;
+    }
+
+    @Override
+    public void write(int repetitionLevel, Decimal decimal) {
+      Preconditions.checkArgument(decimal.scale() == scale,
+          "Cannot write value as decimal(%s,%s), wrong scale: %s", precision, scale, decimal);
+      Preconditions.checkArgument(decimal.precision() <= precision,
+          "Cannot write value as decimal(%s,%s), too large: %s", precision, scale, decimal);
+
+      column.writeLong(repetitionLevel, decimal.toUnscaledLong());
+    }
+  }
+
+  private static class FixedDecimalWriter extends PrimitiveWriter<Decimal> {
+    private final int precision;
+    private final int scale;
+    private final int length;
+    private final ThreadLocal<byte[]> bytes;
 
 Review comment:
   At one point would multiple threads access the same writer? Does this truly need to be a `ThreadLocal`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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