[HUDI-654] Rename hudi-hive to hudi-hive-sync
This commit is contained in:
@@ -0,0 +1,366 @@
|
||||
/*
|
||||
* 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.hive;
|
||||
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.SchemaTestUtil;
|
||||
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent;
|
||||
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent.PartitionEventType;
|
||||
import org.apache.hudi.hive.util.SchemaUtil;
|
||||
|
||||
import org.apache.hadoop.hive.metastore.api.Partition;
|
||||
import org.apache.parquet.schema.MessageType;
|
||||
import org.apache.parquet.schema.OriginalType;
|
||||
import org.apache.parquet.schema.PrimitiveType;
|
||||
import org.apache.parquet.schema.Types;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class TestHiveSyncTool {
|
||||
|
||||
// Test sync tool using both jdbc and metastore client
|
||||
private boolean useJdbc;
|
||||
|
||||
public TestHiveSyncTool(Boolean useJdbc) {
|
||||
this.useJdbc = useJdbc;
|
||||
}
|
||||
|
||||
@Parameterized.Parameters(name = "UseJdbc")
|
||||
public static Collection<Boolean[]> data() {
|
||||
return Arrays.asList(new Boolean[][] {{false}, {true}});
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws IOException, InterruptedException {
|
||||
TestUtil.setUp();
|
||||
}
|
||||
|
||||
@After
|
||||
public void teardown() throws IOException {
|
||||
TestUtil.clear();
|
||||
}
|
||||
|
||||
/**
|
||||
* Testing converting array types to Hive field declaration strings. According to the Parquet-113 spec:
|
||||
* https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists
|
||||
*/
|
||||
@Test
|
||||
public void testSchemaConvertArray() throws IOException {
|
||||
// Testing the 3-level annotation structure
|
||||
MessageType schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup()
|
||||
.optional(PrimitiveType.PrimitiveTypeName.INT32).named("element").named("list").named("int_list")
|
||||
.named("ArrayOfInts");
|
||||
|
||||
String schemaString = SchemaUtil.generateSchemaString(schema);
|
||||
assertEquals("`int_list` ARRAY< int>", schemaString);
|
||||
|
||||
// A array of arrays
|
||||
schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup().requiredGroup()
|
||||
.as(OriginalType.LIST).repeatedGroup().required(PrimitiveType.PrimitiveTypeName.INT32).named("element")
|
||||
.named("list").named("element").named("list").named("int_list_list").named("ArrayOfArrayOfInts");
|
||||
|
||||
schemaString = SchemaUtil.generateSchemaString(schema);
|
||||
assertEquals("`int_list_list` ARRAY< ARRAY< int>>", schemaString);
|
||||
|
||||
// A list of integers
|
||||
schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeated(PrimitiveType.PrimitiveTypeName.INT32)
|
||||
.named("element").named("int_list").named("ArrayOfInts");
|
||||
|
||||
schemaString = SchemaUtil.generateSchemaString(schema);
|
||||
assertEquals("`int_list` ARRAY< int>", schemaString);
|
||||
|
||||
// A list of structs with two fields
|
||||
schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup()
|
||||
.required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").required(PrimitiveType.PrimitiveTypeName.INT32)
|
||||
.named("num").named("element").named("tuple_list").named("ArrayOfTuples");
|
||||
|
||||
schemaString = SchemaUtil.generateSchemaString(schema);
|
||||
assertEquals("`tuple_list` ARRAY< STRUCT< `str` : binary, `num` : int>>", schemaString);
|
||||
|
||||
// A list of structs with a single field
|
||||
// For this case, since the inner group name is "array", we treat the
|
||||
// element type as a one-element struct.
|
||||
schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup()
|
||||
.required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("array").named("one_tuple_list")
|
||||
.named("ArrayOfOneTuples");
|
||||
|
||||
schemaString = SchemaUtil.generateSchemaString(schema);
|
||||
assertEquals("`one_tuple_list` ARRAY< STRUCT< `str` : binary>>", schemaString);
|
||||
|
||||
// A list of structs with a single field
|
||||
// For this case, since the inner group name ends with "_tuple", we also treat the
|
||||
// element type as a one-element struct.
|
||||
schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup()
|
||||
.required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("one_tuple_list_tuple")
|
||||
.named("one_tuple_list").named("ArrayOfOneTuples2");
|
||||
|
||||
schemaString = SchemaUtil.generateSchemaString(schema);
|
||||
assertEquals("`one_tuple_list` ARRAY< STRUCT< `str` : binary>>", schemaString);
|
||||
|
||||
// A list of structs with a single field
|
||||
// Unlike the above two cases, for this the element type is the type of the
|
||||
// only field in the struct.
|
||||
schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup()
|
||||
.required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("one_tuple_list").named("one_tuple_list")
|
||||
.named("ArrayOfOneTuples3");
|
||||
|
||||
schemaString = SchemaUtil.generateSchemaString(schema);
|
||||
assertEquals("`one_tuple_list` ARRAY< binary>", schemaString);
|
||||
|
||||
// A list of maps
|
||||
schema = Types.buildMessage().optionalGroup().as(OriginalType.LIST).repeatedGroup().as(OriginalType.MAP)
|
||||
.repeatedGroup().as(OriginalType.MAP_KEY_VALUE).required(PrimitiveType.PrimitiveTypeName.BINARY)
|
||||
.as(OriginalType.UTF8).named("string_key").required(PrimitiveType.PrimitiveTypeName.INT32).named("int_value")
|
||||
.named("key_value").named("array").named("map_list").named("ArrayOfMaps");
|
||||
|
||||
schemaString = SchemaUtil.generateSchemaString(schema);
|
||||
assertEquals("`map_list` ARRAY< MAP< string, int>>", schemaString);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBasicSync() throws Exception {
|
||||
TestUtil.hiveSyncConfig.useJdbc = this.useJdbc;
|
||||
String commitTime = "100";
|
||||
TestUtil.createCOWTable(commitTime, 5);
|
||||
HoodieHiveClient hiveClient =
|
||||
new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + " should not exist initially",
|
||||
hiveClient.doesTableExist(TestUtil.hiveSyncConfig.tableName));
|
||||
// Lets do the sync
|
||||
HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
tool.syncHoodieTable();
|
||||
assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + " should exist after sync completes",
|
||||
hiveClient.doesTableExist(TestUtil.hiveSyncConfig.tableName));
|
||||
assertEquals("Hive Schema should match the table schema + partition field",
|
||||
hiveClient.getTableSchema(TestUtil.hiveSyncConfig.tableName).size(),
|
||||
hiveClient.getDataSchema().getColumns().size() + 1);
|
||||
assertEquals("Table partitions should match the number of partitions we wrote", 5,
|
||||
hiveClient.scanTablePartitions(TestUtil.hiveSyncConfig.tableName).size());
|
||||
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", commitTime,
|
||||
hiveClient.getLastCommitTimeSynced(TestUtil.hiveSyncConfig.tableName).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSyncIncremental() throws Exception {
|
||||
TestUtil.hiveSyncConfig.useJdbc = this.useJdbc;
|
||||
String commitTime1 = "100";
|
||||
TestUtil.createCOWTable(commitTime1, 5);
|
||||
HoodieHiveClient hiveClient =
|
||||
new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
// Lets do the sync
|
||||
HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
tool.syncHoodieTable();
|
||||
assertEquals("Table partitions should match the number of partitions we wrote", 5,
|
||||
hiveClient.scanTablePartitions(TestUtil.hiveSyncConfig.tableName).size());
|
||||
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", commitTime1,
|
||||
hiveClient.getLastCommitTimeSynced(TestUtil.hiveSyncConfig.tableName).get());
|
||||
|
||||
// Now lets create more parititions and these are the only ones which needs to be synced
|
||||
DateTime dateTime = DateTime.now().plusDays(6);
|
||||
String commitTime2 = "101";
|
||||
TestUtil.addCOWPartitions(1, true, dateTime, commitTime2);
|
||||
|
||||
// Lets do the sync
|
||||
hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
List<String> writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.of(commitTime1));
|
||||
assertEquals("We should have one partition written after 100 commit", 1, writtenPartitionsSince.size());
|
||||
List<Partition> hivePartitions = hiveClient.scanTablePartitions(TestUtil.hiveSyncConfig.tableName);
|
||||
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince);
|
||||
assertEquals("There should be only one paritition event", 1, partitionEvents.size());
|
||||
assertEquals("The one partition event must of type ADD", PartitionEventType.ADD,
|
||||
partitionEvents.iterator().next().eventType);
|
||||
|
||||
tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
tool.syncHoodieTable();
|
||||
// Sync should add the one partition
|
||||
assertEquals("The one partition we wrote should be added to hive", 6,
|
||||
hiveClient.scanTablePartitions(TestUtil.hiveSyncConfig.tableName).size());
|
||||
assertEquals("The last commit that was sycned should be 101", commitTime2,
|
||||
hiveClient.getLastCommitTimeSynced(TestUtil.hiveSyncConfig.tableName).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSyncIncrementalWithSchemaEvolution() throws Exception {
|
||||
TestUtil.hiveSyncConfig.useJdbc = this.useJdbc;
|
||||
String commitTime1 = "100";
|
||||
TestUtil.createCOWTable(commitTime1, 5);
|
||||
HoodieHiveClient hiveClient =
|
||||
new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
// Lets do the sync
|
||||
HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
tool.syncHoodieTable();
|
||||
|
||||
int fields = hiveClient.getTableSchema(TestUtil.hiveSyncConfig.tableName).size();
|
||||
|
||||
// Now lets create more parititions and these are the only ones which needs to be synced
|
||||
DateTime dateTime = DateTime.now().plusDays(6);
|
||||
String commitTime2 = "101";
|
||||
TestUtil.addCOWPartitions(1, false, dateTime, commitTime2);
|
||||
|
||||
// Lets do the sync
|
||||
tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
tool.syncHoodieTable();
|
||||
|
||||
assertEquals("Hive Schema has evolved and should not be 3 more field", fields + 3,
|
||||
hiveClient.getTableSchema(TestUtil.hiveSyncConfig.tableName).size());
|
||||
assertEquals("Hive Schema has evolved - Field favorite_number has evolved from int to long", "BIGINT",
|
||||
hiveClient.getTableSchema(TestUtil.hiveSyncConfig.tableName).get("favorite_number"));
|
||||
assertTrue("Hive Schema has evolved - Field favorite_movie was added",
|
||||
hiveClient.getTableSchema(TestUtil.hiveSyncConfig.tableName).containsKey("favorite_movie"));
|
||||
|
||||
// Sync should add the one partition
|
||||
assertEquals("The one partition we wrote should be added to hive", 6,
|
||||
hiveClient.scanTablePartitions(TestUtil.hiveSyncConfig.tableName).size());
|
||||
assertEquals("The last commit that was sycned should be 101", commitTime2,
|
||||
hiveClient.getLastCommitTimeSynced(TestUtil.hiveSyncConfig.tableName).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSyncMergeOnRead() throws Exception {
|
||||
TestUtil.hiveSyncConfig.useJdbc = this.useJdbc;
|
||||
String commitTime = "100";
|
||||
String deltaCommitTime = "101";
|
||||
TestUtil.createMORTable(commitTime, deltaCommitTime, 5);
|
||||
|
||||
String roTableName = TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE;
|
||||
HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + " should not exist initially", hiveClient.doesTableExist(roTableName));
|
||||
// Lets do the sync
|
||||
HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
tool.syncHoodieTable();
|
||||
|
||||
assertTrue("Table " + roTableName + " should exist after sync completes",
|
||||
hiveClient.doesTableExist(roTableName));
|
||||
assertEquals("Hive Schema should match the table schema + partition field", hiveClient.getTableSchema(roTableName).size(),
|
||||
SchemaTestUtil.getSimpleSchema().getFields().size() + 1);
|
||||
assertEquals("Table partitions should match the number of partitions we wrote", 5,
|
||||
hiveClient.scanTablePartitions(roTableName).size());
|
||||
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", deltaCommitTime,
|
||||
hiveClient.getLastCommitTimeSynced(roTableName).get());
|
||||
|
||||
// Now lets create more partitions and these are the only ones which needs to be synced
|
||||
DateTime dateTime = DateTime.now().plusDays(6);
|
||||
String commitTime2 = "102";
|
||||
String deltaCommitTime2 = "103";
|
||||
|
||||
TestUtil.addCOWPartitions(1, true, dateTime, commitTime2);
|
||||
TestUtil.addMORPartitions(1, true, false, dateTime, commitTime2, deltaCommitTime2);
|
||||
// Lets do the sync
|
||||
tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
tool.syncHoodieTable();
|
||||
hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
|
||||
assertEquals("Hive Schema should match the evolved table schema + partition field",
|
||||
hiveClient.getTableSchema(roTableName).size(), SchemaTestUtil.getEvolvedSchema().getFields().size() + 1);
|
||||
// Sync should add the one partition
|
||||
assertEquals("The 2 partitions we wrote should be added to hive", 6, hiveClient.scanTablePartitions(roTableName).size());
|
||||
assertEquals("The last commit that was synced should be 103", deltaCommitTime2,
|
||||
hiveClient.getLastCommitTimeSynced(roTableName).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSyncMergeOnReadRT() throws Exception {
|
||||
TestUtil.hiveSyncConfig.useJdbc = this.useJdbc;
|
||||
String commitTime = "100";
|
||||
String deltaCommitTime = "101";
|
||||
String snapshotTableName = TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE;
|
||||
TestUtil.createMORTable(commitTime, deltaCommitTime, 5);
|
||||
HoodieHiveClient hiveClientRT =
|
||||
new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
|
||||
assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE
|
||||
+ " should not exist initially", hiveClientRT.doesTableExist(snapshotTableName));
|
||||
|
||||
// Lets do the sync
|
||||
HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
tool.syncHoodieTable();
|
||||
|
||||
assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE
|
||||
+ " should exist after sync completes", hiveClientRT.doesTableExist(snapshotTableName));
|
||||
|
||||
assertEquals("Hive Schema should match the table schema + partition field", hiveClientRT.getTableSchema(snapshotTableName).size(),
|
||||
SchemaTestUtil.getSimpleSchema().getFields().size() + 1);
|
||||
assertEquals("Table partitions should match the number of partitions we wrote", 5,
|
||||
hiveClientRT.scanTablePartitions(snapshotTableName).size());
|
||||
assertEquals("The last commit that was synced should be updated in the TBLPROPERTIES", deltaCommitTime,
|
||||
hiveClientRT.getLastCommitTimeSynced(snapshotTableName).get());
|
||||
|
||||
// Now lets create more parititions and these are the only ones which needs to be synced
|
||||
DateTime dateTime = DateTime.now().plusDays(6);
|
||||
String commitTime2 = "102";
|
||||
String deltaCommitTime2 = "103";
|
||||
|
||||
TestUtil.addCOWPartitions(1, true, dateTime, commitTime2);
|
||||
TestUtil.addMORPartitions(1, true, false, dateTime, commitTime2, deltaCommitTime2);
|
||||
// Lets do the sync
|
||||
tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
tool.syncHoodieTable();
|
||||
hiveClientRT = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
|
||||
assertEquals("Hive Schema should match the evolved table schema + partition field",
|
||||
hiveClientRT.getTableSchema(snapshotTableName).size(), SchemaTestUtil.getEvolvedSchema().getFields().size() + 1);
|
||||
// Sync should add the one partition
|
||||
assertEquals("The 2 partitions we wrote should be added to hive", 6, hiveClientRT.scanTablePartitions(snapshotTableName).size());
|
||||
assertEquals("The last commit that was sycned should be 103", deltaCommitTime2,
|
||||
hiveClientRT.getLastCommitTimeSynced(snapshotTableName).get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiPartitionKeySync() throws Exception {
|
||||
TestUtil.hiveSyncConfig.useJdbc = this.useJdbc;
|
||||
String commitTime = "100";
|
||||
TestUtil.createCOWTable(commitTime, 5);
|
||||
|
||||
HiveSyncConfig hiveSyncConfig = HiveSyncConfig.copy(TestUtil.hiveSyncConfig);
|
||||
hiveSyncConfig.partitionValueExtractorClass = MultiPartKeysValueExtractor.class.getCanonicalName();
|
||||
hiveSyncConfig.tableName = "multi_part_key";
|
||||
hiveSyncConfig.partitionFields = Arrays.asList("year", "month", "day");
|
||||
TestUtil.getCreatedTablesSet().add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName);
|
||||
|
||||
HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
assertFalse("Table " + hiveSyncConfig.tableName + " should not exist initially",
|
||||
hiveClient.doesTableExist(hiveSyncConfig.tableName));
|
||||
// Lets do the sync
|
||||
HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
tool.syncHoodieTable();
|
||||
assertTrue("Table " + hiveSyncConfig.tableName + " should exist after sync completes",
|
||||
hiveClient.doesTableExist(hiveSyncConfig.tableName));
|
||||
assertEquals("Hive Schema should match the table schema + partition fields",
|
||||
hiveClient.getTableSchema(hiveSyncConfig.tableName).size(),
|
||||
hiveClient.getDataSchema().getColumns().size() + 3);
|
||||
assertEquals("Table partitions should match the number of partitions we wrote", 5,
|
||||
hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size());
|
||||
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES", commitTime,
|
||||
hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get());
|
||||
}
|
||||
}
|
||||
348
hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestUtil.java
Normal file
348
hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestUtil.java
Normal file
@@ -0,0 +1,348 @@
|
||||
/*
|
||||
* 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.hive;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.common.bloom.filter.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.filter.BloomFilterFactory;
|
||||
import org.apache.hudi.common.bloom.filter.BloomFilterTypeCode;
|
||||
import org.apache.hudi.common.minicluster.HdfsTestService;
|
||||
import org.apache.hudi.common.minicluster.ZookeeperTestService;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieDeltaWriteStat;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
|
||||
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.SchemaTestUtil;
|
||||
import org.apache.hudi.hive.util.HiveTestService;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hive.service.server.HiveServer2;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
import org.apache.zookeeper.server.ZooKeeperServer;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.format.DateTimeFormat;
|
||||
import org.joda.time.format.DateTimeFormatter;
|
||||
import org.junit.runners.model.InitializationError;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
@SuppressWarnings("SameParameterValue")
|
||||
public class TestUtil {
|
||||
|
||||
private static MiniDFSCluster dfsCluster;
|
||||
private static ZooKeeperServer zkServer;
|
||||
private static HiveServer2 hiveServer;
|
||||
private static Configuration configuration;
|
||||
static HiveSyncConfig hiveSyncConfig;
|
||||
private static DateTimeFormatter dtfOut;
|
||||
static FileSystem fileSystem;
|
||||
private static Set<String> createdTablesSet = new HashSet<>();
|
||||
|
||||
public static void setUp() throws IOException, InterruptedException {
|
||||
if (dfsCluster == null) {
|
||||
HdfsTestService service = new HdfsTestService();
|
||||
dfsCluster = service.start(true);
|
||||
configuration = service.getHadoopConf();
|
||||
}
|
||||
if (zkServer == null) {
|
||||
ZookeeperTestService zkService = new ZookeeperTestService(configuration);
|
||||
zkServer = zkService.start();
|
||||
}
|
||||
if (hiveServer == null) {
|
||||
HiveTestService hiveService = new HiveTestService(configuration);
|
||||
hiveServer = hiveService.start();
|
||||
}
|
||||
fileSystem = FileSystem.get(configuration);
|
||||
|
||||
hiveSyncConfig = new HiveSyncConfig();
|
||||
hiveSyncConfig.jdbcUrl = "jdbc:hive2://127.0.0.1:9999/";
|
||||
hiveSyncConfig.databaseName = "hdrone_test";
|
||||
hiveSyncConfig.hiveUser = "";
|
||||
hiveSyncConfig.hivePass = "";
|
||||
hiveSyncConfig.databaseName = "testdb";
|
||||
hiveSyncConfig.tableName = "test1";
|
||||
hiveSyncConfig.basePath = "/tmp/hdfs/TestHiveSyncTool/";
|
||||
hiveSyncConfig.assumeDatePartitioning = true;
|
||||
hiveSyncConfig.usePreApacheInputFormat = false;
|
||||
hiveSyncConfig.partitionFields = Collections.singletonList("datestr");
|
||||
|
||||
dtfOut = DateTimeFormat.forPattern("yyyy/MM/dd");
|
||||
|
||||
clear();
|
||||
}
|
||||
|
||||
static void clear() throws IOException {
|
||||
fileSystem.delete(new Path(hiveSyncConfig.basePath), true);
|
||||
HoodieTableMetaClient.initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE,
|
||||
hiveSyncConfig.tableName, HoodieAvroPayload.class.getName());
|
||||
|
||||
HoodieHiveClient client = new HoodieHiveClient(hiveSyncConfig, hiveServer.getHiveConf(), fileSystem);
|
||||
for (String tableName : createdTablesSet) {
|
||||
client.updateHiveSQL("drop table if exists " + tableName);
|
||||
}
|
||||
createdTablesSet.clear();
|
||||
client.updateHiveSQL("drop database if exists " + hiveSyncConfig.databaseName);
|
||||
client.updateHiveSQL("create database " + hiveSyncConfig.databaseName);
|
||||
}
|
||||
|
||||
static HiveConf getHiveConf() {
|
||||
return hiveServer.getHiveConf();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
public static void shutdown() {
|
||||
if (hiveServer != null) {
|
||||
hiveServer.stop();
|
||||
}
|
||||
if (dfsCluster != null) {
|
||||
dfsCluster.shutdown();
|
||||
}
|
||||
if (zkServer != null) {
|
||||
zkServer.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
static void createCOWTable(String commitTime, int numberOfPartitions)
|
||||
throws IOException, InitializationError, URISyntaxException {
|
||||
Path path = new Path(hiveSyncConfig.basePath);
|
||||
FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
|
||||
HoodieTableMetaClient.initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE,
|
||||
hiveSyncConfig.tableName, HoodieAvroPayload.class.getName());
|
||||
boolean result = fileSystem.mkdirs(path);
|
||||
checkResult(result);
|
||||
DateTime dateTime = DateTime.now();
|
||||
HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, true, dateTime, commitTime);
|
||||
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName);
|
||||
createCommitFile(commitMetadata, commitTime);
|
||||
}
|
||||
|
||||
static void createMORTable(String commitTime, String deltaCommitTime, int numberOfPartitions)
|
||||
throws IOException, InitializationError, URISyntaxException, InterruptedException {
|
||||
Path path = new Path(hiveSyncConfig.basePath);
|
||||
FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
|
||||
HoodieTableMetaClient.initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.MERGE_ON_READ,
|
||||
hiveSyncConfig.tableName, HoodieAvroPayload.class.getName());
|
||||
|
||||
boolean result = fileSystem.mkdirs(path);
|
||||
checkResult(result);
|
||||
DateTime dateTime = DateTime.now();
|
||||
HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, true, dateTime, commitTime);
|
||||
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName);
|
||||
createdTablesSet
|
||||
.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE);
|
||||
HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata();
|
||||
commitMetadata.getPartitionToWriteStats()
|
||||
.forEach((key, value) -> value.forEach(l -> compactionMetadata.addWriteStat(key, l)));
|
||||
createCompactionCommitFile(compactionMetadata, commitTime);
|
||||
// Write a delta commit
|
||||
HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(), true);
|
||||
createDeltaCommitFile(deltaMetadata, deltaCommitTime);
|
||||
}
|
||||
|
||||
static void addCOWPartitions(int numberOfPartitions, boolean isParquetSchemaSimple, DateTime startFrom,
|
||||
String commitTime) throws IOException, URISyntaxException {
|
||||
HoodieCommitMetadata commitMetadata =
|
||||
createPartitions(numberOfPartitions, isParquetSchemaSimple, startFrom, commitTime);
|
||||
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName);
|
||||
createCommitFile(commitMetadata, commitTime);
|
||||
}
|
||||
|
||||
static void addMORPartitions(int numberOfPartitions, boolean isParquetSchemaSimple, boolean isLogSchemaSimple,
|
||||
DateTime startFrom, String commitTime, String deltaCommitTime)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
HoodieCommitMetadata commitMetadata =
|
||||
createPartitions(numberOfPartitions, isParquetSchemaSimple, startFrom, commitTime);
|
||||
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE);
|
||||
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE);
|
||||
HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata();
|
||||
commitMetadata.getPartitionToWriteStats()
|
||||
.forEach((key, value) -> value.forEach(l -> compactionMetadata.addWriteStat(key, l)));
|
||||
createCompactionCommitFile(compactionMetadata, commitTime);
|
||||
HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(), isLogSchemaSimple);
|
||||
createDeltaCommitFile(deltaMetadata, deltaCommitTime);
|
||||
}
|
||||
|
||||
private static HoodieCommitMetadata createLogFiles(Map<String, List<HoodieWriteStat>> partitionWriteStats,
|
||||
boolean isLogSchemaSimple) throws InterruptedException, IOException, URISyntaxException {
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
for (Entry<String, List<HoodieWriteStat>> wEntry : partitionWriteStats.entrySet()) {
|
||||
String partitionPath = wEntry.getKey();
|
||||
for (HoodieWriteStat wStat : wEntry.getValue()) {
|
||||
Path path = new Path(wStat.getPath());
|
||||
HoodieBaseFile dataFile = new HoodieBaseFile(fileSystem.getFileStatus(path));
|
||||
HoodieLogFile logFile = generateLogData(path, isLogSchemaSimple);
|
||||
HoodieDeltaWriteStat writeStat = new HoodieDeltaWriteStat();
|
||||
writeStat.setFileId(dataFile.getFileId());
|
||||
writeStat.setPath(logFile.getPath().toString());
|
||||
commitMetadata.addWriteStat(partitionPath, writeStat);
|
||||
}
|
||||
}
|
||||
return commitMetadata;
|
||||
}
|
||||
|
||||
private static HoodieCommitMetadata createPartitions(int numberOfPartitions, boolean isParquetSchemaSimple,
|
||||
DateTime startFrom, String commitTime) throws IOException, URISyntaxException {
|
||||
startFrom = startFrom.withTimeAtStartOfDay();
|
||||
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
for (int i = 0; i < numberOfPartitions; i++) {
|
||||
String partitionPath = dtfOut.print(startFrom);
|
||||
Path partPath = new Path(hiveSyncConfig.basePath + "/" + partitionPath);
|
||||
fileSystem.makeQualified(partPath);
|
||||
fileSystem.mkdirs(partPath);
|
||||
List<HoodieWriteStat> writeStats = createTestData(partPath, isParquetSchemaSimple, commitTime);
|
||||
startFrom = startFrom.minusDays(1);
|
||||
writeStats.forEach(s -> commitMetadata.addWriteStat(partitionPath, s));
|
||||
}
|
||||
return commitMetadata;
|
||||
}
|
||||
|
||||
private static List<HoodieWriteStat> createTestData(Path partPath, boolean isParquetSchemaSimple, String commitTime)
|
||||
throws IOException, URISyntaxException {
|
||||
List<HoodieWriteStat> writeStats = new ArrayList<>();
|
||||
for (int i = 0; i < 5; i++) {
|
||||
// Create 5 files
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeDataFileName(commitTime, "1-0-1", fileId));
|
||||
generateParquetData(filePath, isParquetSchemaSimple);
|
||||
HoodieWriteStat writeStat = new HoodieWriteStat();
|
||||
writeStat.setFileId(fileId);
|
||||
writeStat.setPath(filePath.toString());
|
||||
writeStats.add(writeStat);
|
||||
}
|
||||
return writeStats;
|
||||
}
|
||||
|
||||
@SuppressWarnings({"unchecked", "deprecation"})
|
||||
private static void generateParquetData(Path filePath, boolean isParquetSchemaSimple)
|
||||
throws IOException, URISyntaxException {
|
||||
Schema schema = (isParquetSchemaSimple ? SchemaTestUtil.getSimpleSchema() : SchemaTestUtil.getEvolvedSchema());
|
||||
org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema);
|
||||
BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.0001, -1,
|
||||
BloomFilterTypeCode.SIMPLE.name());
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, filter);
|
||||
ParquetWriter writer = new ParquetWriter(filePath, writeSupport, CompressionCodecName.GZIP, 120 * 1024 * 1024,
|
||||
ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED,
|
||||
ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION, fileSystem.getConf());
|
||||
|
||||
List<IndexedRecord> testRecords = (isParquetSchemaSimple ? SchemaTestUtil.generateTestRecords(0, 100)
|
||||
: SchemaTestUtil.generateEvolvedTestRecords(100, 100));
|
||||
testRecords.forEach(s -> {
|
||||
try {
|
||||
writer.write(s);
|
||||
} catch (IOException e) {
|
||||
fail("IOException while writing test records as parquet" + e.toString());
|
||||
}
|
||||
});
|
||||
writer.close();
|
||||
}
|
||||
|
||||
private static HoodieLogFile generateLogData(Path parquetFilePath, boolean isLogSchemaSimple)
|
||||
throws IOException, InterruptedException, URISyntaxException {
|
||||
Schema schema = (isLogSchemaSimple ? SchemaTestUtil.getSimpleSchema() : SchemaTestUtil.getEvolvedSchema());
|
||||
HoodieBaseFile dataFile = new HoodieBaseFile(fileSystem.getFileStatus(parquetFilePath));
|
||||
// Write a log file for this parquet file
|
||||
Writer logWriter = HoodieLogFormat.newWriterBuilder().onParentPath(parquetFilePath.getParent())
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(dataFile.getFileId())
|
||||
.overBaseCommit(dataFile.getCommitTime()).withFs(fileSystem).build();
|
||||
List<IndexedRecord> records = (isLogSchemaSimple ? SchemaTestUtil.generateTestRecords(0, 100)
|
||||
: SchemaTestUtil.generateEvolvedTestRecords(100, 100));
|
||||
Map<HeaderMetadataType, String> header = new HashMap<>(2);
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, dataFile.getCommitTime());
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
|
||||
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
|
||||
logWriter.appendBlock(dataBlock);
|
||||
logWriter.close();
|
||||
return logWriter.getLogFile();
|
||||
}
|
||||
|
||||
private static void checkResult(boolean result) throws InitializationError {
|
||||
if (!result) {
|
||||
throw new InitializationError("Could not initialize");
|
||||
}
|
||||
}
|
||||
|
||||
private static void createCommitFile(HoodieCommitMetadata commitMetadata, String commitTime) throws IOException {
|
||||
byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8);
|
||||
Path fullPath = new Path(hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeCommitFileName(commitTime));
|
||||
FSDataOutputStream fsout = fileSystem.create(fullPath, true);
|
||||
fsout.write(bytes);
|
||||
fsout.close();
|
||||
}
|
||||
|
||||
private static void createCompactionCommitFile(HoodieCommitMetadata commitMetadata, String commitTime)
|
||||
throws IOException {
|
||||
byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8);
|
||||
Path fullPath = new Path(hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeCommitFileName(commitTime));
|
||||
FSDataOutputStream fsout = fileSystem.create(fullPath, true);
|
||||
fsout.write(bytes);
|
||||
fsout.close();
|
||||
}
|
||||
|
||||
private static void createDeltaCommitFile(HoodieCommitMetadata deltaCommitMetadata, String deltaCommitTime)
|
||||
throws IOException {
|
||||
byte[] bytes = deltaCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8);
|
||||
Path fullPath = new Path(hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeDeltaFileName(deltaCommitTime));
|
||||
FSDataOutputStream fsout = fileSystem.create(fullPath, true);
|
||||
fsout.write(bytes);
|
||||
fsout.close();
|
||||
}
|
||||
|
||||
public static Set<String> getCreatedTablesSet() {
|
||||
return createdTablesSet;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,287 @@
|
||||
/*
|
||||
* 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.hive.util;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hadoop.hive.metastore.HiveMetaStore;
|
||||
import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
|
||||
import org.apache.hadoop.hive.metastore.IHMSHandler;
|
||||
import org.apache.hadoop.hive.metastore.TSetIpAddressProcessor;
|
||||
import org.apache.hadoop.hive.metastore.TUGIBasedProcessor;
|
||||
import org.apache.hadoop.hive.metastore.api.MetaException;
|
||||
import org.apache.hadoop.hive.thrift.TUGIContainingTransport;
|
||||
import org.apache.hive.service.server.HiveServer2;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.thrift.TProcessor;
|
||||
import org.apache.thrift.protocol.TBinaryProtocol;
|
||||
import org.apache.thrift.server.TServer;
|
||||
import org.apache.thrift.server.TThreadPoolServer;
|
||||
import org.apache.thrift.transport.TFramedTransport;
|
||||
import org.apache.thrift.transport.TServerSocket;
|
||||
import org.apache.thrift.transport.TServerTransport;
|
||||
import org.apache.thrift.transport.TSocket;
|
||||
import org.apache.thrift.transport.TTransport;
|
||||
import org.apache.thrift.transport.TTransportException;
|
||||
import org.apache.thrift.transport.TTransportFactory;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.SocketException;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
public class HiveTestService {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HiveTestService.class);
|
||||
|
||||
private static final int CONNECTION_TIMEOUT = 30000;
|
||||
|
||||
/**
|
||||
* Configuration settings.
|
||||
*/
|
||||
private Configuration hadoopConf;
|
||||
private String workDir;
|
||||
private String bindIP = "127.0.0.1";
|
||||
private int metastorePort = 9083;
|
||||
private int serverPort = 9999;
|
||||
private boolean clean = true;
|
||||
|
||||
private Map<String, String> sysProps = Maps.newHashMap();
|
||||
private ExecutorService executorService;
|
||||
private TServer tServer;
|
||||
private HiveServer2 hiveServer;
|
||||
|
||||
public HiveTestService(Configuration configuration) {
|
||||
this.workDir = Files.createTempDir().getAbsolutePath();
|
||||
}
|
||||
|
||||
public Configuration getHadoopConf() {
|
||||
return hadoopConf;
|
||||
}
|
||||
|
||||
public HiveServer2 start() throws IOException {
|
||||
Preconditions.checkState(workDir != null, "The work dir must be set before starting cluster.");
|
||||
|
||||
if (hadoopConf == null) {
|
||||
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
|
||||
}
|
||||
|
||||
String localHiveLocation = getHiveLocation(workDir);
|
||||
if (clean) {
|
||||
LOG.info("Cleaning Hive cluster data at: " + localHiveLocation + " and starting fresh.");
|
||||
File file = new File(localHiveLocation);
|
||||
FileIOUtils.deleteDirectory(file);
|
||||
}
|
||||
|
||||
HiveConf serverConf = configureHive(hadoopConf, localHiveLocation);
|
||||
|
||||
executorService = Executors.newSingleThreadExecutor();
|
||||
tServer = startMetaStore(bindIP, metastorePort, serverConf);
|
||||
|
||||
hiveServer = startHiveServer(serverConf);
|
||||
|
||||
String serverHostname;
|
||||
if (bindIP.equals("0.0.0.0")) {
|
||||
serverHostname = "localhost";
|
||||
} else {
|
||||
serverHostname = bindIP;
|
||||
}
|
||||
if (!waitForServerUp(serverConf, serverHostname, metastorePort, CONNECTION_TIMEOUT)) {
|
||||
throw new IOException("Waiting for startup of standalone server");
|
||||
}
|
||||
|
||||
LOG.info("Hive Minicluster service started.");
|
||||
return hiveServer;
|
||||
}
|
||||
|
||||
private HiveConf configureHive(Configuration conf, String localHiveLocation) throws IOException {
|
||||
conf.set("hive.metastore.local", "false");
|
||||
conf.set(HiveConf.ConfVars.METASTOREURIS.varname, "thrift://" + bindIP + ":" + metastorePort);
|
||||
conf.set(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.varname, bindIP);
|
||||
conf.setInt(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, serverPort);
|
||||
// The following line to turn of SASL has no effect since HiveAuthFactory calls
|
||||
// 'new HiveConf()'. This is fixed by https://issues.apache.org/jira/browse/HIVE-6657,
|
||||
// in Hive 0.14.
|
||||
// As a workaround, the property is set in hive-site.xml in this module.
|
||||
// conf.set(HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION.varname, "NOSASL");
|
||||
File localHiveDir = new File(localHiveLocation);
|
||||
localHiveDir.mkdirs();
|
||||
File metastoreDbDir = new File(localHiveDir, "metastore_db");
|
||||
conf.set(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname,
|
||||
"jdbc:derby:" + metastoreDbDir.getPath() + ";create=true");
|
||||
File derbyLogFile = new File(localHiveDir, "derby.log");
|
||||
derbyLogFile.createNewFile();
|
||||
setSystemProperty("derby.stream.error.file", derbyLogFile.getPath());
|
||||
conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, Files.createTempDir().getAbsolutePath());
|
||||
conf.set("datanucleus.schema.autoCreateTables", "true");
|
||||
conf.set("hive.metastore.schema.verification", "false");
|
||||
setSystemProperty("derby.stream.error.file", derbyLogFile.getPath());
|
||||
|
||||
return new HiveConf(conf, this.getClass());
|
||||
}
|
||||
|
||||
private boolean waitForServerUp(HiveConf serverConf, String hostname, int port, int timeout) {
|
||||
long start = System.currentTimeMillis();
|
||||
while (true) {
|
||||
try {
|
||||
new HiveMetaStoreClient(serverConf);
|
||||
return true;
|
||||
} catch (MetaException e) {
|
||||
// ignore as this is expected
|
||||
LOG.info("server " + hostname + ":" + port + " not up " + e);
|
||||
}
|
||||
|
||||
if (System.currentTimeMillis() > start + timeout) {
|
||||
break;
|
||||
}
|
||||
try {
|
||||
Thread.sleep(250);
|
||||
} catch (InterruptedException e) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private void setSystemProperty(String name, String value) {
|
||||
if (!sysProps.containsKey(name)) {
|
||||
String currentValue = System.getProperty(name);
|
||||
sysProps.put(name, currentValue);
|
||||
}
|
||||
if (value != null) {
|
||||
System.setProperty(name, value);
|
||||
} else {
|
||||
System.getProperties().remove(name);
|
||||
}
|
||||
}
|
||||
|
||||
private static String getHiveLocation(String baseLocation) {
|
||||
return baseLocation + Path.SEPARATOR + "hive";
|
||||
}
|
||||
|
||||
private HiveServer2 startHiveServer(HiveConf serverConf) {
|
||||
HiveServer2 hiveServer = new HiveServer2();
|
||||
hiveServer.init(serverConf);
|
||||
hiveServer.start();
|
||||
return hiveServer;
|
||||
}
|
||||
|
||||
// XXX: From org.apache.hadoop.hive.metastore.HiveMetaStore,
|
||||
// with changes to support binding to a specified IP address (not only 0.0.0.0)
|
||||
private static final class ChainedTTransportFactory extends TTransportFactory {
|
||||
|
||||
private final TTransportFactory parentTransFactory;
|
||||
private final TTransportFactory childTransFactory;
|
||||
|
||||
private ChainedTTransportFactory(TTransportFactory parentTransFactory, TTransportFactory childTransFactory) {
|
||||
this.parentTransFactory = parentTransFactory;
|
||||
this.childTransFactory = childTransFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public TTransport getTransport(TTransport trans) {
|
||||
return childTransFactory.getTransport(parentTransFactory.getTransport(trans));
|
||||
}
|
||||
}
|
||||
|
||||
private static final class TServerSocketKeepAlive extends TServerSocket {
|
||||
|
||||
public TServerSocketKeepAlive(int port) throws TTransportException {
|
||||
super(port, 0);
|
||||
}
|
||||
|
||||
public TServerSocketKeepAlive(InetSocketAddress address) throws TTransportException {
|
||||
super(address, 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected TSocket acceptImpl() throws TTransportException {
|
||||
TSocket ts = super.acceptImpl();
|
||||
try {
|
||||
ts.getSocket().setKeepAlive(true);
|
||||
} catch (SocketException e) {
|
||||
throw new TTransportException(e);
|
||||
}
|
||||
return ts;
|
||||
}
|
||||
}
|
||||
|
||||
public TServer startMetaStore(String forceBindIP, int port, HiveConf conf) throws IOException {
|
||||
try {
|
||||
// Server will create new threads up to max as necessary. After an idle
|
||||
// period, it will destory threads to keep the number of threads in the
|
||||
// pool to min.
|
||||
int minWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMINTHREADS);
|
||||
int maxWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMAXTHREADS);
|
||||
boolean tcpKeepAlive = conf.getBoolVar(HiveConf.ConfVars.METASTORE_TCP_KEEP_ALIVE);
|
||||
boolean useFramedTransport = conf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_FRAMED_TRANSPORT);
|
||||
|
||||
// don't support SASL yet
|
||||
// boolean useSasl = conf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL);
|
||||
|
||||
TServerTransport serverTransport;
|
||||
if (forceBindIP != null) {
|
||||
InetSocketAddress address = new InetSocketAddress(forceBindIP, port);
|
||||
serverTransport = tcpKeepAlive ? new TServerSocketKeepAlive(address) : new TServerSocket(address);
|
||||
|
||||
} else {
|
||||
serverTransport = tcpKeepAlive ? new TServerSocketKeepAlive(port) : new TServerSocket(port);
|
||||
}
|
||||
|
||||
TProcessor processor;
|
||||
TTransportFactory transFactory;
|
||||
|
||||
IHMSHandler handler = (IHMSHandler) HiveMetaStore.newRetryingHMSHandler("new db based metaserver", conf, true);
|
||||
|
||||
if (conf.getBoolVar(HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI)) {
|
||||
transFactory = useFramedTransport
|
||||
? new ChainedTTransportFactory(new TFramedTransport.Factory(), new TUGIContainingTransport.Factory())
|
||||
: new TUGIContainingTransport.Factory();
|
||||
|
||||
processor = new TUGIBasedProcessor<>(handler);
|
||||
LOG.info("Starting DB backed MetaStore Server with SetUGI enabled");
|
||||
} else {
|
||||
transFactory = useFramedTransport ? new TFramedTransport.Factory() : new TTransportFactory();
|
||||
processor = new TSetIpAddressProcessor<>(handler);
|
||||
LOG.info("Starting DB backed MetaStore Server");
|
||||
}
|
||||
|
||||
TThreadPoolServer.Args args = new TThreadPoolServer.Args(serverTransport).processor(processor)
|
||||
.transportFactory(transFactory).protocolFactory(new TBinaryProtocol.Factory())
|
||||
.minWorkerThreads(minWorkerThreads).maxWorkerThreads(maxWorkerThreads);
|
||||
|
||||
final TServer tServer = new TThreadPoolServer(args);
|
||||
executorService.submit(tServer::serve);
|
||||
return tServer;
|
||||
} catch (Throwable x) {
|
||||
throw new IOException(x);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,23 @@
|
||||
###
|
||||
# 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.
|
||||
###
|
||||
log4j.rootLogger=WARN, A1
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.A1=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.A1.layout.ConversionPattern=[%-5p] %d %c %x - %m%n
|
||||
26
hudi-hive-sync/src/test/resources/log4j-surefire.properties
Normal file
26
hudi-hive-sync/src/test/resources/log4j-surefire.properties
Normal file
@@ -0,0 +1,26 @@
|
||||
###
|
||||
# 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.
|
||||
###
|
||||
log4j.rootLogger=WARN, A1
|
||||
log4j.category.org.apache=INFO
|
||||
log4j.category.org.apache.parquet.hadoop=WARN
|
||||
log4j.category.parquet.hadoop=WARN
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.A1=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
|
||||
Reference in New Issue
Block a user