[HUDI-3464] Fix wrong exception thrown from HiveSchemaProvider (#4865)
This commit is contained in:
@@ -0,0 +1,32 @@
|
|||||||
|
/*
|
||||||
|
* 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.exception;
|
||||||
|
|
||||||
|
import org.apache.hudi.exception.HoodieException;
|
||||||
|
|
||||||
|
public class HoodieSchemaProviderException extends HoodieException {
|
||||||
|
|
||||||
|
public HoodieSchemaProviderException(String msg, Throwable e) {
|
||||||
|
super(msg, e);
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieSchemaProviderException(String msg) {
|
||||||
|
super(msg);
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -19,12 +19,12 @@
|
|||||||
|
|
||||||
package org.apache.hudi.utilities.schema;
|
package org.apache.hudi.utilities.schema;
|
||||||
|
|
||||||
import org.apache.avro.Schema;
|
|
||||||
import org.apache.hudi.AvroConversionUtils;
|
import org.apache.hudi.AvroConversionUtils;
|
||||||
import org.apache.hudi.DataSourceUtils;
|
import org.apache.hudi.DataSourceUtils;
|
||||||
import org.apache.hudi.common.config.TypedProperties;
|
import org.apache.hudi.common.config.TypedProperties;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.hudi.utilities.exception.HoodieSchemaProviderException;
|
||||||
import org.apache.log4j.Logger;
|
|
||||||
|
import org.apache.avro.Schema;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.apache.spark.sql.SparkSession;
|
import org.apache.spark.sql.SparkSession;
|
||||||
import org.apache.spark.sql.catalyst.TableIdentifier;
|
import org.apache.spark.sql.catalyst.TableIdentifier;
|
||||||
@@ -34,6 +34,9 @@ import org.apache.spark.sql.types.StructType;
|
|||||||
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A schema provider to get data schema through user specified hive table.
|
||||||
|
*/
|
||||||
public class HiveSchemaProvider extends SchemaProvider {
|
public class HiveSchemaProvider extends SchemaProvider {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -46,40 +49,42 @@ public class HiveSchemaProvider extends SchemaProvider {
|
|||||||
private static final String TARGET_SCHEMA_TABLE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.hive.table";
|
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 final Schema sourceSchema;
|
||||||
|
|
||||||
private Schema targetSchema;
|
private Schema targetSchema;
|
||||||
|
|
||||||
public HiveSchemaProvider(TypedProperties props, JavaSparkContext jssc) {
|
public HiveSchemaProvider(TypedProperties props, JavaSparkContext jssc) {
|
||||||
super(props, jssc);
|
super(props, jssc);
|
||||||
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SOURCE_SCHEMA_TABLE_PROP));
|
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SOURCE_SCHEMA_TABLE_PROP));
|
||||||
String sourceSchemaDBName = props.getString(Config.SOURCE_SCHEMA_DATABASE_PROP, "default");
|
String sourceSchemaDatabaseName = props.getString(Config.SOURCE_SCHEMA_DATABASE_PROP, "default");
|
||||||
String sourceSchemaTableName = props.getString(Config.SOURCE_SCHEMA_TABLE_PROP);
|
String sourceSchemaTableName = props.getString(Config.SOURCE_SCHEMA_TABLE_PROP);
|
||||||
SparkSession spark = SparkSession.builder().config(jssc.getConf()).enableHiveSupport().getOrCreate();
|
SparkSession spark = SparkSession.builder().config(jssc.getConf()).enableHiveSupport().getOrCreate();
|
||||||
|
|
||||||
|
// source schema
|
||||||
try {
|
try {
|
||||||
TableIdentifier sourceSchemaTable = new TableIdentifier(sourceSchemaTableName, scala.Option.apply(sourceSchemaDBName));
|
TableIdentifier sourceSchemaTable = new TableIdentifier(sourceSchemaTableName, scala.Option.apply(sourceSchemaDatabaseName));
|
||||||
StructType sourceSchema = spark.sessionState().catalog().getTableMetadata(sourceSchemaTable).schema();
|
StructType sourceSchema = spark.sessionState().catalog().getTableMetadata(sourceSchemaTable).schema();
|
||||||
|
|
||||||
this.sourceSchema = AvroConversionUtils.convertStructTypeToAvroSchema(
|
this.sourceSchema = AvroConversionUtils.convertStructTypeToAvroSchema(
|
||||||
sourceSchema,
|
sourceSchema,
|
||||||
sourceSchemaTableName,
|
sourceSchemaTableName,
|
||||||
"hoodie." + sourceSchemaDBName);
|
"hoodie." + sourceSchemaDatabaseName);
|
||||||
|
} catch (NoSuchTableException | NoSuchDatabaseException e) {
|
||||||
|
throw new HoodieSchemaProviderException(String.format("Can't find Hive table: %s.%s", sourceSchemaDatabaseName, sourceSchemaTableName), e);
|
||||||
|
}
|
||||||
|
|
||||||
if (props.containsKey(Config.TARGET_SCHEMA_TABLE_PROP)) {
|
// target schema
|
||||||
String targetSchemaDBName = props.getString(Config.TARGET_SCHEMA_DATABASE_PROP, "default");
|
if (props.containsKey(Config.TARGET_SCHEMA_TABLE_PROP)) {
|
||||||
String targetSchemaTableName = props.getString(Config.TARGET_SCHEMA_TABLE_PROP);
|
String targetSchemaDatabaseName = props.getString(Config.TARGET_SCHEMA_DATABASE_PROP, "default");
|
||||||
TableIdentifier targetSchemaTable = new TableIdentifier(targetSchemaTableName, scala.Option.apply(targetSchemaDBName));
|
String targetSchemaTableName = props.getString(Config.TARGET_SCHEMA_TABLE_PROP);
|
||||||
|
try {
|
||||||
|
TableIdentifier targetSchemaTable = new TableIdentifier(targetSchemaTableName, scala.Option.apply(targetSchemaDatabaseName));
|
||||||
StructType targetSchema = spark.sessionState().catalog().getTableMetadata(targetSchemaTable).schema();
|
StructType targetSchema = spark.sessionState().catalog().getTableMetadata(targetSchemaTable).schema();
|
||||||
this.targetSchema = AvroConversionUtils.convertStructTypeToAvroSchema(
|
this.targetSchema = AvroConversionUtils.convertStructTypeToAvroSchema(
|
||||||
targetSchema,
|
targetSchema,
|
||||||
targetSchemaTableName,
|
targetSchemaTableName,
|
||||||
"hoodie." + targetSchemaDBName);
|
"hoodie." + targetSchemaDatabaseName);
|
||||||
|
} catch (NoSuchDatabaseException | NoSuchTableException e) {
|
||||||
|
throw new HoodieSchemaProviderException(String.format("Can't find Hive table: %s.%s", targetSchemaDatabaseName, targetSchemaTableName), e);
|
||||||
}
|
}
|
||||||
} catch (NoSuchTableException | NoSuchDatabaseException e) {
|
|
||||||
String message = String.format("Can't find Hive table(s): %s", sourceSchemaTableName + "," + props.getString(Config.TARGET_SCHEMA_TABLE_PROP));
|
|
||||||
throw new IllegalArgumentException(message, e);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
Reference in New Issue
Block a user