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 2020/07/31 23:46:22 UTC

[GitHub] [hudi] umehrot2 commented on a change in pull request #1876: [HUDI-242] Support for RFC-12/Bootstrapping of external datasets

umehrot2 commented on a change in pull request #1876:
URL: https://github.com/apache/hudi/pull/1876#discussion_r463889083



##########
File path: hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapSourceSchemaProvider.java
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.client.bootstrap;
+
+import org.apache.hudi.avro.model.HoodieFileStatus;
+import org.apache.hudi.common.bootstrap.FileStatusUtils;
+import org.apache.hudi.common.util.ParquetUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.Path;
+
+import java.util.List;
+
+/**
+ * Bootstrap Schema Provider. Schema provided in config is used. If not available, use schema from Parquet
+ */
+public class BootstrapSourceSchemaProvider {
+
+  protected final HoodieWriteConfig writeConfig;
+
+  public BootstrapSourceSchemaProvider(HoodieWriteConfig writeConfig) {
+    this.writeConfig = writeConfig;
+  }
+
+  /**
+   * Main API to select avro schema for bootstrapping.
+   * @param jsc Java Spark Context
+   * @param partitions  List of partitions with files within them
+   * @return Avro Schema
+   */
+  public final Schema getBootstrapSchema(JavaSparkContext jsc, List<Pair<String, List<HoodieFileStatus>>> partitions) {
+    if (writeConfig.getSchema() != null) {
+      // Use schema specified by user if set
+      return Schema.parse(writeConfig.getSchema());
+    }
+    return getBootstrapSourceSchema(jsc, partitions);
+  }
+
+  /**
+   * Select a random file to be used to generate avro schema.
+   * Override this method to get custom schema selection.
+   * @param jsc Java Spark Context
+   * @param partitions  List of partitions with files within them
+   * @return Avro Schema
+   */
+  protected Schema getBootstrapSourceSchema(JavaSparkContext jsc,
+      List<Pair<String, List<HoodieFileStatus>>> partitions) {
+    return partitions.stream().flatMap(p -> p.getValue().stream())
+        .map(fs -> {
+          try {
+            Path filePath = FileStatusUtils.toPath(fs.getPath());
+            return ParquetUtils.readAvroSchema(jsc.hadoopConfiguration(), filePath);

Review comment:
       @vinothchandar I think we should move this to `hudi-spark` module. You had this comment on my pull request: https://github.com/apache/hudi/pull/1702/files#r444571139 . Please check out the changes I have done in this class.
   
   Basically we were running into some schema compatibility issues, in particular with the `RecordDataBootstrapProvider`. The issue was happening because here we are reading `avro schema` using `parquet utils` whereas when we will later perform an upsert etc. then we will get `avro schema` via the regular path where we use `spark-avro` to convert `spark schema` to `avro schema`. The `avro schema` obtained from these two different approaches has compatibility issues.
   
   Thus to maintain compatibility what I had to do is:
   - read parquet schema
   - using spark's parquet-spark schema convertor convert it to `spark schema`
   - using spark-avro convert `spark schema` to `avro schema`
   
   That is why I had to introduce `spark-avro` in `hudi-client`. If you agree with the above suggestion, and do not want `spark-avro` to be added to `hudi-client` then I would suggest moving this class to `hudi-spark`.




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