1
0

Refactor hoodie-hive

This commit is contained in:
Prasanna Rajaperumal
2017-05-19 23:47:27 -07:00
committed by prazanna
parent c192dd60b4
commit db6150c5ef
40 changed files with 1614 additions and 2296 deletions

View File

@@ -1,186 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.hive;
import com.uber.hoodie.hive.client.SchemaUtil;
import com.uber.hoodie.hive.model.HoodieDatasetReference;
import com.uber.hoodie.hive.model.SchemaDifference;
import com.uber.hoodie.hive.util.TestUtil;
import org.joda.time.DateTime;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.runners.model.InitializationError;
import parquet.schema.MessageType;
import parquet.schema.OriginalType;
import parquet.schema.PrimitiveType;
import java.io.IOException;
import static org.junit.Assert.assertEquals;
public class DatasetSchemaTest {
@Before
public void setUp() throws IOException, InterruptedException {
TestUtil.setUp();
}
@Test
public void testSchemaDiff() throws IOException, InitializationError {
HoodieDatasetReference metadata = TestUtil
.createDataset("test1", "/tmp/hdfs/DatasetSchemaTest/testSchema/", 5, "/nation.schema");
HoodieHiveSchemaSyncTask schema =
HoodieHiveSchemaSyncTask.newBuilder().withReference(metadata)
.withConfiguration(TestUtil.hDroneConfiguration).build();
SchemaDifference diff = schema.getSchemaDifference();
assertEquals("There should be 4 columns to be added", 4, diff.getAddColumnTypes().size());
assertEquals("No update columns expected", 0, diff.getUpdateColumnTypes().size());
assertEquals("No delete columns expected", 0, diff.getDeleteColumns().size());
schema.sync();
schema = HoodieHiveSchemaSyncTask.newBuilder().withReference(metadata)
.withConfiguration(TestUtil.hDroneConfiguration).build();
diff = schema.getSchemaDifference();
assertEquals("After sync, there should not be any new columns to add", 0,
diff.getAddColumnTypes().size());
assertEquals("After sync, there should not be any new columns to update", 0,
diff.getUpdateColumnTypes().size());
assertEquals("After sync, there should not be any new columns to delete", 0,
diff.getDeleteColumns().size());
}
@Test
public void testSchemaEvolution() throws IOException, InitializationError {
int initialPartitionsCount = 5;
HoodieDatasetReference metadata = TestUtil
.createDataset("test1", "/tmp/hdfs/DatasetSchemaTest/testSchema/",
initialPartitionsCount, "/nation.schema");
HoodieHiveSchemaSyncTask schema =
HoodieHiveSchemaSyncTask.newBuilder().withReference(metadata)
.withConfiguration(TestUtil.hDroneConfiguration).build();
schema.sync();
schema = HoodieHiveSchemaSyncTask.newBuilder().withReference(metadata)
.withConfiguration(TestUtil.hDroneConfiguration).build();
SchemaDifference diff = schema.getSchemaDifference();
assertEquals("After sync, diff should be empty", true, diff.isEmpty());
int newSchemaversion = 2;
int newPartitionsCount = 2;
TestUtil.evolveDataset(metadata, newPartitionsCount, "/nation_evolved.schema",
DateTime.now().getMillis(), newSchemaversion);
schema = HoodieHiveSchemaSyncTask.newBuilder().withReference(metadata)
.withConfiguration(TestUtil.hDroneConfiguration).build();
diff = schema.getSchemaDifference();
assertEquals("Schema has evolved, there should be a diff", false, diff.isEmpty());
assertEquals("Schema has evolved, there should be 1 column to add", 1,
diff.getAddColumnTypes().size());
assertEquals("Schema has evolved, there should be 1 column to update", 1,
diff.getUpdateColumnTypes().size());
assertEquals(0, diff.getDeleteColumns().size());
}
/**
* 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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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);
}
}

View File

@@ -1,99 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.hive;
import com.uber.hoodie.hive.client.HoodieHiveClient;
import com.uber.hoodie.hive.model.HoodieDatasetReference;
import com.uber.hoodie.hive.util.TestUtil;
import org.joda.time.DateTime;
import org.junit.Before;
import org.junit.Test;
import org.junit.runners.model.InitializationError;
import parquet.schema.MessageType;
import java.io.IOException;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
public class HDroneDatasetTest {
private HoodieHiveClient hiveClient;
@Before
public void setUp() throws IOException, InterruptedException {
TestUtil.setUp();
hiveClient = new HoodieHiveClient(TestUtil.hDroneConfiguration);
}
@Test
public void testDatasetCreation() throws IOException, InitializationError {
HoodieDatasetReference metadata = TestUtil
.createDataset("test1", "/tmp/hdfs/DatasetSchemaTest/testSchema/", 5, "/nation.schema");
HoodieHiveDatasetSyncTask dataset =
HoodieHiveDatasetSyncTask.newBuilder().withReference(metadata)
.withConfiguration(TestUtil.hDroneConfiguration).build();
assertEquals("There should be 5 new partitions", 5, dataset.getNewPartitions().size());
assertEquals("There should not be any changed partitions", 0,
dataset.getChangedPartitions().size());
assertFalse("Table should not exist", hiveClient.checkTableExists(metadata));
dataset.sync();
dataset = HoodieHiveDatasetSyncTask.newBuilder().withReference(metadata)
.withConfiguration(TestUtil.hDroneConfiguration).build();
assertTrue("Table should exist after flush", hiveClient.checkTableExists(metadata));
assertEquals("After flush, There should not be any new partitions to flush", 0,
dataset.getNewPartitions().size());
assertEquals("After flush, There should not be any modified partitions to flush", 0,
dataset.getChangedPartitions().size());
assertEquals("Table Schema should have 5 fields", 5,
hiveClient.getTableSchema(metadata).size());
}
@Test
public void testDatasetEvolution() throws IOException, InitializationError {
int initialPartitionsCount = 5;
HoodieDatasetReference metadata = TestUtil
.createDataset("test1", "/tmp/hdfs/DatasetSchemaTest/testSchema/",
initialPartitionsCount, "/nation.schema");
HoodieHiveDatasetSyncTask dataset =
HoodieHiveDatasetSyncTask.newBuilder().withReference(metadata)
.withConfiguration(TestUtil.hDroneConfiguration).build();
dataset.sync();
dataset = HoodieHiveDatasetSyncTask.newBuilder(dataset).build();
int newSchemaversion = 2;
int newPartitionsCount = 2;
TestUtil.evolveDataset(metadata, newPartitionsCount, "/nation_evolved.schema",
DateTime.now().getMillis(), newSchemaversion);
dataset = HoodieHiveDatasetSyncTask.newBuilder(dataset).build();
assertEquals("There should be " + newPartitionsCount + " partitions to be added",
newPartitionsCount, dataset.getNewPartitions().size());
dataset.sync();
dataset = HoodieHiveDatasetSyncTask.newBuilder(dataset).build();
MessageType newDatasetSchema = dataset.getSchemaSyncTask().getStorageSchema();
MessageType expectedSchema = TestUtil.readSchema("/nation_evolved.schema");
assertEquals("Table schema should be evolved schema", expectedSchema, newDatasetSchema);
assertEquals("Table schema should have 6 fields", 6,
hiveClient.getTableSchema(metadata).size());
assertEquals("Valid Evolution should be reflected", "BIGINT",
hiveClient.getTableSchema(metadata).get("region_key"));
}
}

View File

@@ -0,0 +1,308 @@
/*
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.hive;
import static org.junit.Assert.*;
import com.uber.hoodie.common.util.SchemaTestUtil;
import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent;
import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent.PartitionEventType;
import com.uber.hoodie.hive.util.SchemaUtil;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.List;
import java.util.Optional;
import org.apache.hadoop.hive.metastore.api.Partition;
import org.apache.thrift.TException;
import org.joda.time.DateTime;
import org.junit.Before;
import org.junit.Test;
import org.junit.runners.model.InitializationError;
import parquet.schema.MessageType;
import parquet.schema.OriginalType;
import parquet.schema.PrimitiveType;
@SuppressWarnings("ConstantConditions")
public class HiveSyncToolTest {
@Before
public void setUp() throws IOException, InterruptedException, URISyntaxException {
TestUtil.setUp();
}
@Before
public void teardown() throws IOException, InterruptedException {
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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 =
parquet.schema.Types.buildMessage().optionalGroup().as(parquet.schema.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 IOException, InitializationError, URISyntaxException, TException, InterruptedException {
String commitTime = "100";
TestUtil.createCOWDataset(commitTime, 5);
HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig,
TestUtil.getHiveConf(), TestUtil.fileSystem);
assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + " should not exist initially",
hiveClient.doesTableExist());
// 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());
assertEquals("Hive Schema should match the dataset schema + partition field",
hiveClient.getTableSchema().size(),
hiveClient.getDataSchema().getColumns().size() + 1);
assertEquals("Table partitions should match the number of partitions we wrote", 5,
hiveClient.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES",
commitTime,
hiveClient.getLastCommitTimeSynced().get());
}
@Test
public void testSyncIncremental()
throws IOException, InitializationError, URISyntaxException, TException, InterruptedException {
String commitTime1 = "100";
TestUtil.createCOWDataset(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().size());
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES",
commitTime1,
hiveClient.getLastCommitTimeSynced().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(Optional.of(commitTime1));
assertEquals("We should have one partition written after 100 commit", 1,
writtenPartitionsSince.size());
List<Partition> hivePartitions = hiveClient.scanTablePartitions();
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().size());
assertEquals("The last commit that was sycned should be 101",
commitTime2,
hiveClient.getLastCommitTimeSynced().get());
}
@Test
public void testSyncIncrementalWithSchemaEvolution()
throws IOException, InitializationError, URISyntaxException, TException, InterruptedException {
String commitTime1 = "100";
TestUtil.createCOWDataset(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().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().size());
assertEquals("Hive Schema has evolved - Field favorite_number has evolved from int to long",
"BIGINT",
hiveClient.getTableSchema().get("favorite_number"));
assertTrue("Hive Schema has evolved - Field favorite_movie was added",
hiveClient.getTableSchema().containsKey("favorite_movie"));
// Sync should add the one partition
assertEquals("The one partition we wrote should be added to hive", 6,
hiveClient.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be 101",
commitTime2,
hiveClient.getLastCommitTimeSynced().get());
}
@Test
public void testSyncMergeOnRead()
throws IOException, InitializationError, URISyntaxException, TException, InterruptedException {
String commitTime = "100";
String deltaCommitTime = "101";
TestUtil.createMORDataset(commitTime, deltaCommitTime, 5);
HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig,
TestUtil.getHiveConf(), TestUtil.fileSystem);
assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + " should not exist initially",
hiveClient.doesTableExist());
// 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());
assertEquals("Hive Schema should match the dataset schema + partition field",
hiveClient.getTableSchema().size(), SchemaTestUtil.getSimpleSchema().getFields().size() + 1);
assertEquals("Table partitions should match the number of partitions we wrote", 5,
hiveClient.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES",
deltaCommitTime,
hiveClient.getLastCommitTimeSynced().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();
hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig,
TestUtil.getHiveConf(), TestUtil.fileSystem);
assertEquals("Hive Schema should match the evolved dataset schema + partition field",
hiveClient.getTableSchema().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().size());
assertEquals("The last commit that was sycned should be 103",
deltaCommitTime2,
hiveClient.getLastCommitTimeSynced().get());
}
}

View File

@@ -0,0 +1,353 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.hive;
import static com.uber.hoodie.common.model.HoodieTestUtils.DEFAULT_TASK_PARTITIONID;
import static org.junit.Assert.fail;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.minicluster.HdfsTestService;
import com.uber.hoodie.common.minicluster.ZookeeperTestService;
import com.uber.hoodie.common.model.CompactionWriteStat;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieDeltaWriteStat;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
import com.uber.hoodie.hive.util.HiveTestService;
import java.io.File;
import java.io.IOException;
import java.net.URISyntaxException;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
import java.util.List;
import java.util.Map.Entry;
import java.util.Set;
import java.util.UUID;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.io.FileUtils;
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;
@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 = Sets.newHashSet();
public static void setUp() throws IOException, InterruptedException, URISyntaxException {
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/HiveSyncToolTest/";
hiveSyncConfig.assumeDatePartitioning = true;
hiveSyncConfig.partitionFields = Lists.newArrayList("datestr");
dtfOut = DateTimeFormat.forPattern("yyyy/MM/dd");
clear();
}
static void clear() throws IOException {
fileSystem.delete(new Path(hiveSyncConfig.basePath), true);
HoodieTableMetaClient
.initTableType(fileSystem, hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE,
hiveSyncConfig.tableName);
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 createCOWDataset(String commitTime, int numberOfPartitions)
throws IOException, InitializationError, URISyntaxException, InterruptedException {
Path path = new Path(hiveSyncConfig.basePath);
FileUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
HoodieTableMetaClient
.initTableType(fileSystem, hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE,
hiveSyncConfig.tableName);
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 createMORDataset(String commitTime, String deltaCommitTime, int numberOfPartitions)
throws IOException, InitializationError, URISyntaxException, InterruptedException {
Path path = new Path(hiveSyncConfig.basePath);
FileUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
HoodieTableMetaClient
.initTableType(fileSystem, hiveSyncConfig.basePath, HoodieTableType.MERGE_ON_READ,
hiveSyncConfig.tableName);
boolean result = fileSystem.mkdirs(path);
checkResult(result);
DateTime dateTime = DateTime.now();
HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, true, dateTime, commitTime);
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName);
HoodieCompactionMetadata compactionMetadata = new HoodieCompactionMetadata();
commitMetadata.getPartitionToWriteStats()
.forEach((key, value) -> value.stream().map(k -> new CompactionWriteStat(k, key, 0, 0, 0))
.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, InterruptedException {
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);
HoodieCompactionMetadata compactionMetadata = new HoodieCompactionMetadata();
commitMetadata.getPartitionToWriteStats()
.forEach((key, value) -> value.stream().map(k -> new CompactionWriteStat(k, key, 0, 0, 0))
.forEach(l -> compactionMetadata.addWriteStat(key, l)));
createCompactionCommitFile(compactionMetadata, commitTime);
HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(), isLogSchemaSimple);
createDeltaCommitFile(deltaMetadata, deltaCommitTime);
}
private static HoodieCommitMetadata createLogFiles(
HashMap<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.getFullPath());
HoodieDataFile dataFile = new HoodieDataFile(fileSystem.getFileStatus(path));
HoodieLogFile logFile = generateLogData(path, isLogSchemaSimple);
HoodieDeltaWriteStat writeStat = new HoodieDeltaWriteStat();
writeStat.setFileId(dataFile.getFileId());
writeStat.setFullPath(logFile.getPath().toString());
commitMetadata.addWriteStat(partitionPath, writeStat);
}
}
return commitMetadata;
}
private static HoodieCommitMetadata createPartitions(int numberOfPartitions,
boolean isParquetSchemaSimple, DateTime startFrom, String commitTime)
throws IOException, URISyntaxException, InterruptedException {
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, InterruptedException {
List<HoodieWriteStat> writeStats = Lists.newArrayList();
for (int i = 0; i < 5; i++) {
// Create 5 files
String fileId = UUID.randomUUID().toString();
Path filePath = new Path(partPath.toString() + "/" + FSUtils
.makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileId));
generateParquetData(filePath, isParquetSchemaSimple);
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setFileId(fileId);
writeStat.setFullPath(filePath.toString());
writeStats.add(writeStat);
}
return writeStats;
}
@SuppressWarnings({"unchecked", "deprecation"})
private static void generateParquetData(Path filePath, boolean isParquetSchemaSimple)
throws IOException, URISyntaxException, InterruptedException {
Schema schema = (isParquetSchemaSimple ? SchemaTestUtil.getSimpleSchema()
: SchemaTestUtil.getEvolvedSchema());
org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema);
BloomFilter filter = new BloomFilter(1000, 0.0001);
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());
HoodieDataFile dataFile = new HoodieDataFile(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));
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, schema);
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(
HoodieCompactionMetadata commitMetadata, String commitTime)
throws IOException {
byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8);
Path fullPath = new Path(
hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeCompactionFileName(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();
}
}

View File

@@ -1,44 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.hive.util;
import org.apache.hadoop.fs.Path;
import parquet.hadoop.ParquetWriter;
import parquet.hadoop.metadata.CompressionCodecName;
import parquet.schema.MessageType;
import java.io.IOException;
import java.util.List;
public class CsvParquetWriter extends ParquetWriter<List<String>> {
public CsvParquetWriter(Path file, MessageType schema) throws IOException {
this(file, schema, false);
}
public CsvParquetWriter(Path file, MessageType schema, boolean enableDictionary)
throws IOException {
this(file, schema, CompressionCodecName.UNCOMPRESSED, enableDictionary);
}
public CsvParquetWriter(Path file, MessageType schema, CompressionCodecName codecName,
boolean enableDictionary) throws IOException {
super(file, new CsvWriteSupport(schema), codecName,
DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE, enableDictionary, false);
}
}

View File

@@ -1,94 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.hive.util;
import org.apache.hadoop.conf.Configuration;
import parquet.column.ColumnDescriptor;
import parquet.hadoop.api.WriteSupport;
import parquet.io.ParquetEncodingException;
import parquet.io.api.Binary;
import parquet.io.api.RecordConsumer;
import parquet.schema.MessageType;
import java.util.HashMap;
import java.util.List;
public class CsvWriteSupport extends WriteSupport<List<String>> {
MessageType schema;
RecordConsumer recordConsumer;
List<ColumnDescriptor> cols;
// TODO: support specifying encodings and compression
public CsvWriteSupport(MessageType schema) {
this.schema = schema;
this.cols = schema.getColumns();
}
@Override public WriteContext init(Configuration config) {
return new WriteContext(schema, new HashMap<String, String>());
}
@Override public void prepareForWrite(RecordConsumer r) {
recordConsumer = r;
}
@Override public void write(List<String> values) {
if (values.size() != cols.size()) {
throw new ParquetEncodingException("Invalid input data. Expecting " +
cols.size() + " columns. Input had " + values.size() + " columns (" + cols + ") : "
+ values);
}
recordConsumer.startMessage();
for (int i = 0; i < cols.size(); ++i) {
String val = values.get(i);
// val.length() == 0 indicates a NULL value.
if (val.length() > 0) {
recordConsumer.startField(cols.get(i).getPath()[0], i);
switch (cols.get(i).getType()) {
case BOOLEAN:
recordConsumer.addBoolean(Boolean.parseBoolean(val));
break;
case FLOAT:
recordConsumer.addFloat(Float.parseFloat(val));
break;
case DOUBLE:
recordConsumer.addDouble(Double.parseDouble(val));
break;
case INT32:
recordConsumer.addInteger(Integer.parseInt(val));
break;
case INT64:
recordConsumer.addLong(Long.parseLong(val));
break;
case BINARY:
recordConsumer.addBinary(stringToBinary(val));
break;
default:
throw new ParquetEncodingException(
"Unsupported column type: " + cols.get(i).getType());
}
recordConsumer.endField(cols.get(i).getPath()[0], i);
}
}
recordConsumer.endMessage();
}
private Binary stringToBinary(Object value) {
return Binary.fromString(value.toString());
}
}

View File

@@ -1,201 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.hive.util;
import com.google.common.collect.Sets;
import com.uber.hoodie.common.minicluster.HdfsTestService;
import com.uber.hoodie.common.minicluster.ZookeeperTestService;
import com.uber.hoodie.hive.HoodieHiveConfiguration;
import com.uber.hoodie.hive.client.HoodieHiveClient;
import com.uber.hoodie.hive.model.HoodieDatasetReference;
import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
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.hive.service.server.HiveServer2;
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 parquet.schema.MessageType;
import parquet.schema.MessageTypeParser;
import java.io.BufferedReader;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.util.Arrays;
import java.util.Set;
import java.util.regex.Pattern;
public class TestUtil {
private static MiniDFSCluster dfsCluster;
private static ZooKeeperServer zkServer;
private static HiveServer2 hiveServer;
public static Configuration configuration;
public static HoodieHiveConfiguration hDroneConfiguration;
private static DateTimeFormatter dtfOut;
public static final String CSV_DELIMITER = "|";
private static FileSystem fileSystem;
private static Set<String> createdTablesSet = Sets.newHashSet();
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();
}
hDroneConfiguration =
HoodieHiveConfiguration.newBuilder().hiveJdbcUrl("jdbc:hive2://127.0.0.1:9999/")
.hivedb("hdrone_test").jdbcUsername("").jdbcPassword("")
.hadoopConfiguration(hiveServer.getHiveConf()).build();
dtfOut = DateTimeFormat.forPattern("yyyy/MM/dd");
HoodieHiveClient client = new HoodieHiveClient(hDroneConfiguration);
for (String tableName : createdTablesSet) {
client.updateHiveSQL("drop table if exists " + tableName);
}
createdTablesSet.clear();
client.updateHiveSQL(
"drop database if exists " + hDroneConfiguration.getDbName());
client.updateHiveSQL("create database " + hDroneConfiguration.getDbName());
fileSystem = FileSystem.get(configuration);
}
public static void shutdown() {
if (hiveServer != null) {
hiveServer.stop();
}
if (dfsCluster != null) {
dfsCluster.shutdown();
}
if (zkServer != null) {
zkServer.shutdown();
}
}
public static HoodieDatasetReference createDataset(String tableName, String hdfsPath, int numberOfPartitions,
String schemaFile) throws IOException, InitializationError {
Path path = new Path(hdfsPath);
FileUtils.deleteDirectory(new File(hdfsPath));
boolean result = fileSystem.mkdirs(path);
checkResult(result);
HoodieDatasetReference metadata =
new HoodieDatasetReference(tableName, path.toString(),
hDroneConfiguration.getDbName());
DateTime dateTime = DateTime.now();
createPartitions(metadata, numberOfPartitions, schemaFile, dateTime, 1);
createdTablesSet.add(metadata.getDatabaseTableName());
return metadata;
}
private static void createPartitions(HoodieDatasetReference metadata, int numberOfPartitions,
String schemaFile, DateTime startFrom, int schemaVersion) throws IOException {
startFrom = startFrom.withTimeAtStartOfDay();
for (int i = 0; i < numberOfPartitions; i++) {
Path partPath = new Path(metadata.getBaseDatasetPath() + "/" + dtfOut.print(startFrom));
fileSystem.makeQualified(partPath);
fileSystem.mkdirs(partPath);
createTestData(partPath, schemaFile, schemaVersion);
startFrom = startFrom.minusDays(1);
}
}
private static void createTestData(Path partPath, String schemaFile, int schemaVersion)
throws IOException {
for (int i = 0; i < 5; i++) {
// Create 5 files
Path filePath =
new Path(partPath.toString() + "/" + getParquetFilePath(schemaVersion, i));
generateParquetData(filePath, schemaFile);
}
}
private static String getParquetFilePath(int version, int iteration) {
return "test.topic.name@sjc1@SV_" + version + "@" + iteration + ".parquet";
}
public static MessageType readSchema(String schemaFile) throws IOException {
return MessageTypeParser
.parseMessageType(IOUtils.toString(TestUtil.class.getResourceAsStream(schemaFile)));
}
public static void generateParquetData(Path filePath, String schemaFile) throws IOException {
MessageType schema = readSchema(schemaFile);
CsvParquetWriter writer = new CsvParquetWriter(filePath, schema);
BufferedReader br = new BufferedReader(
new InputStreamReader(TestUtil.class.getResourceAsStream(getDataFile(schemaFile))));
String line;
try {
while ((line = br.readLine()) != null) {
String[] fields = line.split(Pattern.quote(CSV_DELIMITER));
writer.write(Arrays.asList(fields));
}
writer.close();
} finally {
br.close();
}
InputStreamReader io = null;
FSDataOutputStream hdfsPath = null;
try {
io = new FileReader(filePath.toString());
hdfsPath = fileSystem.create(filePath);
IOUtils.copy(io, hdfsPath);
} finally {
if (io != null) {
io.close();
}
if (hdfsPath != null) {
hdfsPath.close();
}
}
}
private static String getDataFile(String schemaFile) {
return schemaFile.replaceAll(".schema", ".csv");
}
private static void checkResult(boolean result) throws InitializationError {
if (!result) {
throw new InitializationError("Could not initialize");
}
}
public static void evolveDataset(HoodieDatasetReference metadata, int newPartitionCount,
String newSchema, Long startFrom, int schemaVersion) throws IOException {
createPartitions(metadata, newPartitionCount, newSchema,
new DateTime(startFrom).plusDays(newPartitionCount + 1), schemaVersion);
}
}

View File

@@ -1,25 +0,0 @@
0|ALGERIA|0| haggle. carefully final deposits detect slyly agai
1|ARGENTINA|1|al foxes promise slyly according to the regular accounts. bold requests alon
2|BRAZIL|1|y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special
3|CANADA|1|eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold
4|EGYPT|4|y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d
5|ETHIOPIA|0|ven packages wake quickly. regu
6|FRANCE|3|refully final requests. regular, ironi
7|GERMANY|3|l platelets. regular accounts x-ray: unusual, regular acco
8|INDIA|2|ss excuses cajole slyly across the packages. deposits print aroun
9|INDONESIA|2| slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull
10|IRAN|4|efully alongside of the slyly final dependencies.
11|IRAQ|4|nic deposits boost atop the quickly final requests? quickly regula
12|JAPAN|2|ously. final, express gifts cajole a
13|JORDAN|4|ic deposits are blithely about the carefully regular pa
14|KENYA|0| pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t
15|MOROCCO|0|rns. blithely bold courts among the closely regular packages use furiously bold platelets?
16|MOZAMBIQUE|0|s. ironic, unusual asymptotes wake blithely r
17|PERU|1|platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun
18|CHINA|2|c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos
19|ROMANIA|3|ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account
20|SAUDI ARABIA|4|ts. silent requests haggle. closely express packages sleep across the blithely
21|VIETNAM|2|hely enticingly express accounts. even, final
22|RUSSIA|3| requests against the platelets use never according to the quickly regular pint
23|UNITED KINGDOM|3|eans boost carefully special requests. accounts are. carefull
24|UNITED STATES|1|y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be
1 0 ALGERIA 0 haggle. carefully final deposits detect slyly agai
2 1 ARGENTINA 1 al foxes promise slyly according to the regular accounts. bold requests alon
3 2 BRAZIL 1 y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special
4 3 CANADA 1 eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold
5 4 EGYPT 4 y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d
6 5 ETHIOPIA 0 ven packages wake quickly. regu
7 6 FRANCE 3 refully final requests. regular, ironi
8 7 GERMANY 3 l platelets. regular accounts x-ray: unusual, regular acco
9 8 INDIA 2 ss excuses cajole slyly across the packages. deposits print aroun
10 9 INDONESIA 2 slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull
11 10 IRAN 4 efully alongside of the slyly final dependencies.
12 11 IRAQ 4 nic deposits boost atop the quickly final requests? quickly regula
13 12 JAPAN 2 ously. final, express gifts cajole a
14 13 JORDAN 4 ic deposits are blithely about the carefully regular pa
15 14 KENYA 0 pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t
16 15 MOROCCO 0 rns. blithely bold courts among the closely regular packages use furiously bold platelets?
17 16 MOZAMBIQUE 0 s. ironic, unusual asymptotes wake blithely r
18 17 PERU 1 platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun
19 18 CHINA 2 c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos
20 19 ROMANIA 3 ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account
21 20 SAUDI ARABIA 4 ts. silent requests haggle. closely express packages sleep across the blithely
22 21 VIETNAM 2 hely enticingly express accounts. even, final
23 22 RUSSIA 3 requests against the platelets use never according to the quickly regular pint
24 23 UNITED KINGDOM 3 eans boost carefully special requests. accounts are. carefull
25 24 UNITED STATES 1 y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be

View File

@@ -1,6 +0,0 @@
message m {
required int32 nation_key;
required binary name;
required int32 region_key;
required binary comment_col;
}

View File

@@ -1,25 +0,0 @@
0|ALGERIA|0| haggle. carefully final deposits detect slyly agai|desc0
1|ARGENTINA|1|al foxes promise slyly according to the regular accounts. bold requests alon|desc1
2|BRAZIL|1|y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special |desc2
3|CANADA|1|eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold|desc3
4|EGYPT|4|y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d|desc4
5|ETHIOPIA|0|ven packages wake quickly. regu|desc5
6|FRANCE|3|refully final requests. regular, ironi|desc6
7|GERMANY|3|l platelets. regular accounts x-ray: unusual, regular acco|desc7
8|INDIA|2|ss excuses cajole slyly across the packages. deposits print aroun|desc8
9|INDONESIA|2| slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull|desc9
10|IRAN|4|efully alongside of the slyly final dependencies. |desc10
11|IRAQ|4|nic deposits boost atop the quickly final requests? quickly regula|desc11
12|JAPAN|2|ously. final, express gifts cajole a|desc12
13|JORDAN|4|ic deposits are blithely about the carefully regular pa|desc13
14|KENYA|0| pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t|desc14
15|MOROCCO|0|rns. blithely bold courts among the closely regular packages use furiously bold platelets?|desc15
16|MOZAMBIQUE|0|s. ironic, unusual asymptotes wake blithely r|desc16
17|PERU|1|platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun|desc17
18|CHINA|2|c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos|desc18
19|ROMANIA|3|ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account|desc19
20|SAUDI ARABIA|4|ts. silent requests haggle. closely express packages sleep across the blithely|desc20
21|VIETNAM|2|hely enticingly express accounts. even, final |desc21
22|RUSSIA|3| requests against the platelets use never according to the quickly regular pint|desc22
23|UNITED KINGDOM|3|eans boost carefully special requests. accounts are. carefull|desc23
24|UNITED STATES|1|y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be|desc24
1 0 ALGERIA 0 haggle. carefully final deposits detect slyly agai desc0
2 1 ARGENTINA 1 al foxes promise slyly according to the regular accounts. bold requests alon desc1
3 2 BRAZIL 1 y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special desc2
4 3 CANADA 1 eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold desc3
5 4 EGYPT 4 y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d desc4
6 5 ETHIOPIA 0 ven packages wake quickly. regu desc5
7 6 FRANCE 3 refully final requests. regular, ironi desc6
8 7 GERMANY 3 l platelets. regular accounts x-ray: unusual, regular acco desc7
9 8 INDIA 2 ss excuses cajole slyly across the packages. deposits print aroun desc8
10 9 INDONESIA 2 slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull desc9
11 10 IRAN 4 efully alongside of the slyly final dependencies. desc10
12 11 IRAQ 4 nic deposits boost atop the quickly final requests? quickly regula desc11
13 12 JAPAN 2 ously. final, express gifts cajole a desc12
14 13 JORDAN 4 ic deposits are blithely about the carefully regular pa desc13
15 14 KENYA 0 pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t desc14
16 15 MOROCCO 0 rns. blithely bold courts among the closely regular packages use furiously bold platelets? desc15
17 16 MOZAMBIQUE 0 s. ironic, unusual asymptotes wake blithely r desc16
18 17 PERU 1 platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun desc17
19 18 CHINA 2 c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos desc18
20 19 ROMANIA 3 ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account desc19
21 20 SAUDI ARABIA 4 ts. silent requests haggle. closely express packages sleep across the blithely desc20
22 21 VIETNAM 2 hely enticingly express accounts. even, final desc21
23 22 RUSSIA 3 requests against the platelets use never according to the quickly regular pint desc22
24 23 UNITED KINGDOM 3 eans boost carefully special requests. accounts are. carefull desc23
25 24 UNITED STATES 1 y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be desc24

View File

@@ -1,7 +0,0 @@
message m {
required int32 nation_key;
required binary name;
required int64 region_key;
required binary comment_col;
optional binary desc;
}