Skip to content
Merged
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
/*
* 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);
LOG.warn(this.targetSchema.toString(true));

}
} 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));
throw new IllegalArgumentException(message, e);
}
}

@Override
public Schema getSourceSchema() {
return sourceSchema;
}

@Override
public Schema getTargetSchema() {
if (targetSchema != null) {
return targetSchema;
} else {
return super.getTargetSchema();
}
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,125 @@
/*
* 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.functional;

import org.apache.avro.Schema;
import org.apache.commons.lang3.tuple.ImmutablePair;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.testutils.FunctionalTestHarness;
import org.apache.hudi.utilities.UtilHelpers;
import org.apache.hudi.utilities.schema.HiveSchemaProvider;
import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;

import java.io.IOException;

import static org.junit.jupiter.api.Assertions.assertEquals;

@Tag("functional")
public class TestHiveSchemaProvider extends FunctionalTestHarness {
private static final Logger LOG = LogManager.getLogger(TestJdbcbasedSchemaProvider.class);
private static final TypedProperties PROPS = new TypedProperties();
private static final String SOURCE_SCHEMA_TABLE_NAME = "schema_registry.source_schema_tab";
private static final String TARGET_SCHEMA_TABLE_NAME = "schema_registry.target_schema_tab";

@BeforeAll
public static void init() {
Pair<String, String> dbAndTableName = getDBandTableName(SOURCE_SCHEMA_TABLE_NAME);
PROPS.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.hive.database", dbAndTableName.getLeft());
PROPS.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.hive.table", dbAndTableName.getRight());
}

@Test
public void testSourceSchema() throws Exception {
try {
createSchemaTable(SOURCE_SCHEMA_TABLE_NAME);
Schema sourceSchema = UtilHelpers.createSchemaProvider(HiveSchemaProvider.class.getName(), PROPS, jsc()).getSourceSchema();
assertEquals(
sourceSchema.toString().toUpperCase(),
new Schema.Parser().parse(
UtilitiesTestBase.Helpers.readFile("delta-streamer-config/hive_schema_provider_source.avsc")
).toString().toUpperCase());
} catch (HoodieException e) {
LOG.error("Failed to get source schema. ", e);
}
}


@Test
public void testTargetSchema() throws Exception {
try {
Pair<String, String> dbAndTableName = getDBandTableName(TARGET_SCHEMA_TABLE_NAME);
PROPS.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.hive.database", dbAndTableName.getLeft());
PROPS.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.hive.table", dbAndTableName.getRight());
createSchemaTable(SOURCE_SCHEMA_TABLE_NAME);
createSchemaTable(TARGET_SCHEMA_TABLE_NAME);
Schema targetSchema = UtilHelpers.createSchemaProvider(HiveSchemaProvider.class.getName(), PROPS, jsc()).getTargetSchema();
assertEquals(
targetSchema.toString().toUpperCase(),
new Schema.Parser().parse(
UtilitiesTestBase.Helpers.readFile("delta-streamer-config/hive_schema_provider_target.avsc")
).toString().toUpperCase());
} catch (HoodieException e) {
LOG.error("Failed to get source/target schema. ", e);
}
}

@Test
public void testNotExistTable() {
String wrongName = "wrong_schema_tab";
PROPS.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.hive.table", wrongName);
Assertions.assertThrows(NoSuchTableException.class, () -> {
try {
UtilHelpers.createSchemaProvider(HiveSchemaProvider.class.getName(), PROPS, jsc()).getSourceSchema();
} catch (Throwable exception) {
while (exception.getCause() != null) {
exception = exception.getCause();
}
throw exception;
}
});
}

private static Pair<String, String> getDBandTableName(String fullName) {
String[] dbAndTableName = fullName.split("\\.");
if (dbAndTableName.length > 1) {
return new ImmutablePair<>(dbAndTableName[0], dbAndTableName[1]);
} else {
return new ImmutablePair<>("default", dbAndTableName[0]);
}
}

private void createSchemaTable(String fullName) throws IOException {
String createTableSQL = UtilitiesTestBase.Helpers.readFile(String.format("delta-streamer-config/%s.sql", fullName));
SparkSession spark = spark();
Pair<String, String> dbAndTableName = getDBandTableName(fullName);
spark.sql(String.format("CREATE DATABASE IF NOT EXISTS %s", dbAndTableName.getLeft()));
spark.sql(createTableSQL);
spark.sql(String.format("SHOW CREATE TABLE %s.%s", dbAndTableName.getLeft(), dbAndTableName.getRight())).show(false);
}
}
Original file line number Diff line number Diff line change
@@ -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.
*/
{
"type": "record",
"name": "source_schema_tab",
"namespace": "hoodie.schema_registry",
"fields": [
{
"name": "id",
"type": [
"long",
"null"
]
},
{
"name": "name",
"type": [
"string",
"null"
]
},
{
"name": "num1",
"type": [
"int",
"null"
]
},
{
"name": "num2",
"type": [
"long",
"null"
]
},
{
"name": "num3",
"type": [
{
"type": "fixed",
"name": "fixed",
"namespace": "hoodie.schema_registry.source_schema_tab.num3",
"size": 9,
"logicalType": "decimal",
"precision": 20,
"scale": 0
},
"null"
]
},
{
"name": "num4",
"type": [
"int",
"null"
]
},
{
"name": "num5",
"type": [
"float",
"null"
]
},
{
"name": "num6",
"type": [
"double",
"null"
]
},
{
"name": "bool",
"type": [
"boolean",
"null"
]
},
{
"name": "bin",
"type": [
"bytes",
"null"
]
}
]
}
Loading