You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2023/01/13 19:55:44 UTC

[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #7512: [HUDI-3217] support to read avro from non-legacy map/list in parquet log

alexeykudinkin commented on code in PR #7512:
URL: https://github.com/apache/hudi/pull/7512#discussion_r1067608145


##########
hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetReaderBuilder.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.parquet.avro;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.io.InputFile;
+
+/**
+ * Not support withDataModel.

Review Comment:
   Can you elaborate on this one?



##########
hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroParquetReaderBuilder.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.parquet.avro;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.io.InputFile;
+
+/**
+ * Not support withDataModel.
+ */
+public class HoodieAvroParquetReaderBuilder<T> extends ParquetReader.Builder<T> {
+
+  public HoodieAvroParquetReaderBuilder(Path path) {
+    super(path);
+  }
+
+  public HoodieAvroParquetReaderBuilder(InputFile file) {
+    super(file);
+  }
+
+  @Override
+  protected ReadSupport<T> getReadSupport() {
+    conf.setBoolean(AvroReadSupport.AVRO_COMPATIBILITY, false);

Review Comment:
   Let's add a comment why this is necessary



##########
hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.parquet.avro;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.Type;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class HoodieAvroReadSupport<T> extends AvroReadSupport<T> {
+
+  public HoodieAvroReadSupport() {
+  }
+
+  @Override
+  public ReadContext init(Configuration configuration, Map<String, String> keyValueMetaData, MessageType fileSchema) {
+    boolean legacyMode = checkLegacyMode(fileSchema.getFields());
+    if (!legacyMode) {
+      configuration.set(AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE, "false");
+    }
+    ReadContext readContext = super.init(configuration, keyValueMetaData, fileSchema);
+    MessageType requestedSchema = readContext.getRequestedSchema();
+    if (!legacyMode) {
+      requestedSchema = new MessageType(requestedSchema.getName(), convertLegacyMap(requestedSchema.getFields()));
+    }
+    return new ReadContext(requestedSchema, readContext.getReadSupportMetadata());
+  }
+
+  /**
+   * Check whether write map/list with legacy mode.
+   * list:
+   * optional group obj_ids (LIST) {
+   *   repeated binary array (UTF8);
+   * }
+   * map:
+   * optional group obj_ids (MAP) {
+   *   repeated group map (MAP_KEY_VALUE) {
+   *      required binary key (UTF8);
+   *      required binary value (UTF8);
+   *   }
+   * }
+   */
+  private boolean checkLegacyMode(List<Type> parquetFields) {
+    for (Type type : parquetFields) {
+      if (!type.isPrimitive()) {
+        GroupType groupType = type.asGroupType();
+        OriginalType originalType = groupType.getOriginalType();
+        if (originalType == OriginalType.MAP
+            && groupType.getFields().get(0).getOriginalType() != OriginalType.MAP_KEY_VALUE) {
+          return false;
+        }
+        if (originalType == OriginalType.LIST
+            && !groupType.getType(0).getName().equals("array")) {
+          return false;
+        }
+        if (!checkLegacyMode(groupType.getFields())) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  private List<Type> convertLegacyMap(List<Type> oldTypes) {
+    List<Type> newTypes = new ArrayList<>(oldTypes.size());
+    for (Type type : oldTypes) {
+      if (!type.isPrimitive()) {
+        GroupType parent = type.asGroupType();
+        List<Type> types = convertLegacyMap(parent.getFields());
+        if (type.getOriginalType() == OriginalType.MAP_KEY_VALUE) {

Review Comment:
   Why do we need to do this conversion?



##########
hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.parquet.avro;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.Type;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class HoodieAvroReadSupport<T> extends AvroReadSupport<T> {
+
+  public HoodieAvroReadSupport() {
+  }
+
+  @Override
+  public ReadContext init(Configuration configuration, Map<String, String> keyValueMetaData, MessageType fileSchema) {
+    boolean legacyMode = checkLegacyMode(fileSchema.getFields());
+    if (!legacyMode) {
+      configuration.set(AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE, "false");
+    }
+    ReadContext readContext = super.init(configuration, keyValueMetaData, fileSchema);
+    MessageType requestedSchema = readContext.getRequestedSchema();
+    if (!legacyMode) {
+      requestedSchema = new MessageType(requestedSchema.getName(), convertLegacyMap(requestedSchema.getFields()));
+    }
+    return new ReadContext(requestedSchema, readContext.getReadSupportMetadata());
+  }
+
+  /**
+   * Check whether write map/list with legacy mode.

Review Comment:
   Let's expand this comment to explain the diff b/w legacy and non-legacy modes



##########
hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.parquet.avro;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.Type;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class HoodieAvroReadSupport<T> extends AvroReadSupport<T> {
+
+  public HoodieAvroReadSupport() {
+  }
+
+  @Override
+  public ReadContext init(Configuration configuration, Map<String, String> keyValueMetaData, MessageType fileSchema) {
+    boolean legacyMode = checkLegacyMode(fileSchema.getFields());
+    if (!legacyMode) {
+      configuration.set(AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE, "false");

Review Comment:
   Take a look at `HoodieAvroParquetReader.tryOverrideDefaultConfigs` for context



##########
hudi-common/src/main/java/org/apache/parquet/avro/HoodieAvroReadSupport.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.parquet.avro;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.schema.GroupType;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.Type;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class HoodieAvroReadSupport<T> extends AvroReadSupport<T> {
+
+  public HoodieAvroReadSupport() {
+  }
+
+  @Override
+  public ReadContext init(Configuration configuration, Map<String, String> keyValueMetaData, MessageType fileSchema) {
+    boolean legacyMode = checkLegacyMode(fileSchema.getFields());
+    if (!legacyMode) {
+      configuration.set(AvroWriteSupport.WRITE_OLD_LIST_STRUCTURE, "false");

Review Comment:
   We need to make sure we're not overwriting configs if they are already set (by the user for ex) 



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceWithParquetLog.scala:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.hudi.functional
+
+
+import org.apache.hudi.HoodieConversionUtils.toJavaOption
+
+import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieStorageConfig}
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType
+import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
+import org.apache.hudi.common.util
+import org.apache.hudi.config.{HoodieCompactionConfig, HoodieIndexConfig, HoodieWriteConfig}
+import org.apache.hudi.index.HoodieIndex.IndexType
+import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieClientTestBase}
+import org.apache.hudi.util.JFunction
+import org.apache.hudi.{DataSourceReadOptions, DataSourceUtils, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkRecordMerger, SparkDatasetMixin}
+import org.apache.log4j.LogManager
+import org.apache.spark.sql._
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.hudi.HoodieSparkSessionExtension
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.{CsvSource, EnumSource}
+
+import java.util.function.Consumer
+import scala.collection.JavaConverters._
+
+class TestMORDataSourceWithParquetLog extends HoodieClientTestBase {
+
+  var spark: SparkSession = null
+  private val log = LogManager.getLogger(classOf[TestMORDataSource])
+  val commonOpts = Map(
+    "hoodie.insert.shuffle.parallelism" -> "4",
+    "hoodie.upsert.shuffle.parallelism" -> "4",
+    DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
+    DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
+    DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
+    HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
+  )
+
+  val sparkOpts = Map(
+    HoodieWriteConfig.MERGER_IMPLS.key -> classOf[HoodieSparkRecordMerger].getName,
+    HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> "parquet"
+  )
+
+  val verificationCol: String = "driver"
+  val updatedVerificationVal: String = "driver_update"
+
+  @BeforeEach override def setUp() {
+    setTableName("hoodie_test")
+    initPath()
+    initSparkContexts()
+    spark = sqlContext.sparkSession
+    initTestDataGenerator()
+    initFileSystem()
+  }
+
+  @AfterEach override def tearDown() = {
+    cleanupSparkContexts()
+    cleanupTestDataGenerator()
+    cleanupFileSystem()
+  }
+
+  override def getSparkSessionExtensionsInjector: util.Option[Consumer[SparkSessionExtensions]] =
+    toJavaOption(
+      Some(
+        JFunction.toJavaConsumer((receiver: SparkSessionExtensions) => new HoodieSparkSessionExtension().apply(receiver)))
+    )
+
+  @ParameterizedTest
+  @CsvSource(Array(
+    "AVRO, AVRO, END_MAP",
+    "AVRO, SPARK, END_MAP",
+    "SPARK, AVRO, END_MAP",
+    "AVRO, AVRO, END_ARRAY",
+    "AVRO, SPARK, END_ARRAY",
+    "SPARK, AVRO, END_ARRAY"))
+  def testRecordTypeCompatibilityWithParquetLog(readType: HoodieRecordType,
+                                                writeType: HoodieRecordType,
+                                                transformMode: String): Unit = {
+    var (_, readOpts) = getOpts(readType)
+    var (writeOpts, _) = getOpts(writeType)
+    readOpts = readOpts ++ Map(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> "parquet")
+    writeOpts = writeOpts ++ Map(HoodieStorageConfig.LOGFILE_DATA_BLOCK_FORMAT.key -> "parquet")
+    val records = dataGen.generateInserts("001", 10)
+
+    // End with array
+    val inputDF1 = transfrom(spark.read.json(

Review Comment:
   Let's break this expression into more readable parts (sourceDF, transformed, etc)



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceWithParquetLog.scala:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.hudi.functional
+
+
+import org.apache.hudi.HoodieConversionUtils.toJavaOption
+
+import org.apache.hudi.common.config.{HoodieMetadataConfig, HoodieStorageConfig}
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType
+import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
+import org.apache.hudi.common.util
+import org.apache.hudi.config.{HoodieCompactionConfig, HoodieIndexConfig, HoodieWriteConfig}
+import org.apache.hudi.index.HoodieIndex.IndexType
+import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieClientTestBase}
+import org.apache.hudi.util.JFunction
+import org.apache.hudi.{DataSourceReadOptions, DataSourceUtils, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkRecordMerger, SparkDatasetMixin}
+import org.apache.log4j.LogManager
+import org.apache.spark.sql._
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.hudi.HoodieSparkSessionExtension
+import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
+import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
+import org.junit.jupiter.params.ParameterizedTest
+import org.junit.jupiter.params.provider.{CsvSource, EnumSource}
+
+import java.util.function.Consumer
+import scala.collection.JavaConverters._
+
+class TestMORDataSourceWithParquetLog extends HoodieClientTestBase {

Review Comment:
   Do we need new class? Can we merge this into `TestMORDataSource` class?



-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org