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 2021/11/03 06:40:26 UTC

[GitHub] [hudi] xushiyan commented on a change in pull request #3671: [HUDI-2418] add HiveSchemaProvider

xushiyan commented on a change in pull request #3671:
URL: https://github.com/apache/hudi/pull/3671#discussion_r741649195



##########
File path: hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java
##########
@@ -164,7 +164,7 @@ public synchronized void runBeforeEach() {
       SparkConf sparkConf = conf();
       SparkRDDWriteClient.registerClasses(sparkConf);
       HoodieReadClient.addHoodieSupport(sparkConf);
-      spark = SparkSession.builder().config(sparkConf).getOrCreate();
+      spark = SparkSession.builder().config(sparkConf).enableHiveSupport().getOrCreate();

Review comment:
       instead of changing this for all tests, can you override `org.apache.hudi.testutils.providers.SparkProvider#conf()` in your specific test class to pass in the configs you need for your testcase?

##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/HiveSchemaProvider.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.utilities.schema;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.DataSourceUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class HiveSchemaProvider extends SchemaProvider {
+
+  /**
+   * Configs supported.
+   */
+  public static class Config {
+    private static final String SOURCE_SCHEMA_DATABASE_PROP = "hoodie.deltastreamer.schemaprovider.source.schema.hive.database";
+    private static final String SOURCE_SCHEMA_TABLE_PROP = "hoodie.deltastreamer.schemaprovider.source.schema.hive.table";
+    private static final String TARGET_SCHEMA_DATABASE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.hive.database";
+    private static final String TARGET_SCHEMA_TABLE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.hive.table";
+  }
+
+  private static final Logger LOG = LogManager.getLogger(HiveSchemaProvider.class);
+
+  private final Schema sourceSchema;
+
+  private Schema targetSchema;
+
+  public HiveSchemaProvider(TypedProperties props, JavaSparkContext jssc) {
+    super(props, jssc);
+    DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SOURCE_SCHEMA_TABLE_PROP));
+    String sourceSchemaDBName = props.getString(Config.SOURCE_SCHEMA_DATABASE_PROP, "default");
+    String sourceSchemaTableName = props.getString(Config.SOURCE_SCHEMA_TABLE_PROP);
+    SparkSession spark = SparkSession.builder().config(jssc.getConf()).enableHiveSupport().getOrCreate();
+    try {
+      TableIdentifier sourceSchemaTable = new TableIdentifier(sourceSchemaTableName, scala.Option.apply(sourceSchemaDBName));
+      StructType sourceSchema = spark.sessionState().catalog().getTableMetadata(sourceSchemaTable).schema();
+
+      this.sourceSchema = AvroConversionUtils.convertStructTypeToAvroSchema(
+              sourceSchema,
+              sourceSchemaTableName,
+              "hoodie." + sourceSchemaDBName);
+
+      if (props.containsKey(Config.TARGET_SCHEMA_TABLE_PROP)) {
+        String targetSchemaDBName = props.getString(Config.TARGET_SCHEMA_DATABASE_PROP, "default");
+        String targetSchemaTableName = props.getString(Config.TARGET_SCHEMA_TABLE_PROP);
+        TableIdentifier targetSchemaTable = new TableIdentifier(targetSchemaTableName, scala.Option.apply(targetSchemaDBName));
+        StructType targetSchema = spark.sessionState().catalog().getTableMetadata(targetSchemaTable).schema();
+        this.targetSchema = AvroConversionUtils.convertStructTypeToAvroSchema(
+                targetSchema,
+                targetSchemaTableName,
+                "hoodie." + targetSchemaDBName);
+      }
+    } catch (NoSuchTableException | NoSuchDatabaseException e) {
+      List<String> tables = new ArrayList<String>() {
+        {
+          add(sourceSchemaTableName);
+        }
+      };
+      if (props.containsKey(Config.TARGET_SCHEMA_TABLE_PROP)) {
+        tables.add(props.getString(Config.SOURCE_SCHEMA_TABLE_PROP));
+      }
+      String message = String.format("Can't find Hive table(s): %s", String.join(",", tables));

Review comment:
       these lines look weird and verbose.. you just need to log 2 variables i guess? why not just format them all at once in one line? also i think you're trying to `props.getString(Config.TARGET_SCHEMA_TABLE_PROP)` instead.
   
   catch block should be lightweight on logic. you should be able to `props.get()` with a default `""` to help formating to save the if checking. Can you simplify these lines pls

##########
File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/HiveSchemaProvider.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.utilities.schema;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.DataSourceUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class HiveSchemaProvider extends SchemaProvider {
+
+  /**
+   * Configs supported.
+   */
+  public static class Config {
+    private static final String SOURCE_SCHEMA_DATABASE_PROP = "hoodie.deltastreamer.schemaprovider.source.schema.hive.database";
+    private static final String SOURCE_SCHEMA_TABLE_PROP = "hoodie.deltastreamer.schemaprovider.source.schema.hive.table";
+    private static final String TARGET_SCHEMA_DATABASE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.hive.database";
+    private static final String TARGET_SCHEMA_TABLE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.hive.table";
+  }
+
+  private static final Logger LOG = LogManager.getLogger(HiveSchemaProvider.class);
+
+  private final Schema sourceSchema;
+
+  private Schema targetSchema;
+
+  public HiveSchemaProvider(TypedProperties props, JavaSparkContext jssc) {
+    super(props, jssc);
+    DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SOURCE_SCHEMA_TABLE_PROP));
+    String sourceSchemaDBName = props.getString(Config.SOURCE_SCHEMA_DATABASE_PROP, "default");
+    String sourceSchemaTableName = props.getString(Config.SOURCE_SCHEMA_TABLE_PROP);
+    SparkSession spark = SparkSession.builder().config(jssc.getConf()).enableHiveSupport().getOrCreate();
+    try {
+      TableIdentifier sourceSchemaTable = new TableIdentifier(sourceSchemaTableName, scala.Option.apply(sourceSchemaDBName));
+      StructType sourceSchema = spark.sessionState().catalog().getTableMetadata(sourceSchemaTable).schema();
+
+      this.sourceSchema = AvroConversionUtils.convertStructTypeToAvroSchema(
+              sourceSchema,
+              sourceSchemaTableName,
+              "hoodie." + sourceSchemaDBName);

Review comment:
       have you tested this with different spark versions? 2.4, 3.0.1, 3.1, etc. we want to thorough about this.




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