Refactor hoodie-hive
This commit is contained in:
committed by
prazanna
parent
c192dd60b4
commit
db6150c5ef
@@ -44,6 +44,7 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
|
||||
private static double STREAM_COMPRESSION_RATIO = 0.1;
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
|
||||
|
||||
private final Path file;
|
||||
private final HoodieWrapperFileSystem fs;
|
||||
private final long maxFileSize;
|
||||
|
||||
@@ -112,6 +112,10 @@ public class HoodieAvroDataBlock implements HoodieLogBlock {
|
||||
dis.readFully(compressedSchema, 0, schemaLength);
|
||||
Schema writerSchema = new Schema.Parser().parse(HoodieAvroUtils.decompress(compressedSchema));
|
||||
|
||||
if(readerSchema == null) {
|
||||
readerSchema = writerSchema;
|
||||
}
|
||||
|
||||
GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(writerSchema, readerSchema);
|
||||
// 2. Get the total records
|
||||
int totalRecords = dis.readInt();
|
||||
|
||||
@@ -18,6 +18,14 @@ package com.uber.hoodie.common.util;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.net.URI;
|
||||
import java.nio.file.FileSystem;
|
||||
import java.nio.file.FileSystemNotFoundException;
|
||||
import java.nio.file.FileSystems;
|
||||
import java.nio.file.Path;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericDatumReader;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -29,7 +37,6 @@ import java.net.URISyntaxException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@@ -39,11 +46,6 @@ public class SchemaTestUtil {
|
||||
.parse(SchemaTestUtil.class.getResourceAsStream("/simple-test.avro"));
|
||||
}
|
||||
|
||||
public static Schema getEvolvedSchema() throws IOException {
|
||||
return new Schema.Parser()
|
||||
.parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avro"));
|
||||
}
|
||||
|
||||
public static List<IndexedRecord> generateTestRecords(int from, int limit)
|
||||
throws IOException, URISyntaxException {
|
||||
return toRecords(getSimpleSchema(), getSimpleSchema(), from, limit);
|
||||
@@ -53,11 +55,19 @@ public class SchemaTestUtil {
|
||||
int limit) throws IOException, URISyntaxException {
|
||||
GenericDatumReader<IndexedRecord> reader =
|
||||
new GenericDatumReader<>(writerSchema, readerSchema);
|
||||
try (Stream<String> stream = Files
|
||||
.lines(Paths.get(SchemaTestUtil.class.getResource("/sample.data").toURI()))) {
|
||||
// Required to register the necessary JAR:// file system
|
||||
URI resource = SchemaTestUtil.class.getClass().getResource("/sample.data").toURI();
|
||||
Path dataPath;
|
||||
if(resource.toString().contains("!")) {
|
||||
dataPath = uriToPath(resource);
|
||||
} else {
|
||||
dataPath = Paths.get(SchemaTestUtil.class.getClass().getResource("/sample.data").toURI());
|
||||
}
|
||||
|
||||
try (Stream<String> stream = Files.lines(dataPath)) {
|
||||
return stream.skip(from).limit(limit).map(s -> {
|
||||
try {
|
||||
return reader.read(null, DecoderFactory.get().jsonDecoder(readerSchema, s));
|
||||
return reader.read(null, DecoderFactory.get().jsonDecoder(writerSchema, s));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not read data from simple_data.json", e);
|
||||
}
|
||||
@@ -67,6 +77,18 @@ public class SchemaTestUtil {
|
||||
}
|
||||
}
|
||||
|
||||
static Path uriToPath(URI uri) throws IOException {
|
||||
final Map<String, String> env = new HashMap<>();
|
||||
final String[] array = uri.toString().split("!");
|
||||
FileSystem fs;
|
||||
try {
|
||||
fs = FileSystems.getFileSystem(URI.create(array[0]));
|
||||
} catch (FileSystemNotFoundException e) {
|
||||
fs = FileSystems.newFileSystem(URI.create(array[0]), env);
|
||||
}
|
||||
return fs.getPath(array[1]);
|
||||
}
|
||||
|
||||
public static List<IndexedRecord> generateHoodieTestRecords(int from, int limit)
|
||||
throws IOException, URISyntaxException {
|
||||
List<IndexedRecord> records = generateTestRecords(from, limit);
|
||||
@@ -81,4 +103,14 @@ public class SchemaTestUtil {
|
||||
Collectors.toList());
|
||||
|
||||
}
|
||||
|
||||
public static Schema getEvolvedSchema() throws IOException {
|
||||
return new Schema.Parser()
|
||||
.parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avro"));
|
||||
}
|
||||
|
||||
public static List<IndexedRecord> generateEvolvedTestRecords(int from, int limit)
|
||||
throws IOException, URISyntaxException {
|
||||
return toRecords(getSimpleSchema(), getEvolvedSchema(), from, limit);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -7,6 +7,7 @@
|
||||
{"name": "field2", "type": ["null", "string"], "default": null},
|
||||
{"name": "name", "type": ["null", "string"], "default": null},
|
||||
{"name": "favorite_number", "type": ["null", "long"], "default": null},
|
||||
{"name": "favorite_color", "type": ["null", "string"], "default": null}
|
||||
{"name": "favorite_color", "type": ["null", "string"], "default": null},
|
||||
{"name": "favorite_movie", "type": ["null", "string"], "default": null}
|
||||
]
|
||||
}
|
||||
}
|
||||
|
||||
@@ -4,7 +4,7 @@
|
||||
"name": "User",
|
||||
"fields": [
|
||||
{"name": "name", "type": "string"},
|
||||
{"name": "favorite_number", "type": "long"},
|
||||
{"name": "favorite_number", "type": "int"},
|
||||
{"name": "favorite_color", "type": "string"}
|
||||
]
|
||||
}
|
||||
|
||||
@@ -120,6 +120,10 @@
|
||||
<artifactId>mockito-all</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.twitter</groupId>
|
||||
<artifactId>parquet-avro</artifactId>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.uber.hoodie</groupId>
|
||||
@@ -138,6 +142,12 @@
|
||||
<classifier>tests</classifier>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.esotericsoftware.kryo</groupId>
|
||||
<artifactId>kryo</artifactId>
|
||||
<version>2.21</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
</dependencies>
|
||||
|
||||
|
||||
@@ -21,30 +21,45 @@ package com.uber.hoodie.hive;
|
||||
import com.beust.jcommander.Parameter;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Configs needed to sync data into Hive.
|
||||
*/
|
||||
public class HiveSyncConfig implements Serializable {
|
||||
|
||||
@Parameter(names = {"--database"}, description = "name of the target database in Hive", required = true)
|
||||
public String databaseName;
|
||||
@Parameter(names = {
|
||||
"--database"}, description = "name of the target database in Hive", required = true)
|
||||
public String databaseName;
|
||||
|
||||
@Parameter(names = {"--table"}, description = "name of the target table in Hive", required = true)
|
||||
public String tableName;
|
||||
@Parameter(names = {"--table"}, description = "name of the target table in Hive", required = true)
|
||||
public String tableName;
|
||||
|
||||
@Parameter(names = {"--user"}, description = "Hive username", required = true)
|
||||
public String hiveUser;
|
||||
@Parameter(names = {"--user"}, description = "Hive username", required = true)
|
||||
public String hiveUser;
|
||||
|
||||
@Parameter(names = {"--pass"}, description = "Hive password", required = true)
|
||||
public String hivePass;
|
||||
@Parameter(names = {"--pass"}, description = "Hive password", required = true)
|
||||
public String hivePass;
|
||||
|
||||
@Parameter(names = {"--jdbc-url"}, description = "Hive jdbc connect url", required = true)
|
||||
public String jdbcUrl;
|
||||
@Parameter(names = {"--jdbc-url"}, description = "Hive jdbc connect url", required = true)
|
||||
public String jdbcUrl;
|
||||
|
||||
@Parameter(names = {"--base-path"}, description = "Basepath of hoodie dataset to sync", required = true)
|
||||
public String basePath;
|
||||
@Parameter(names = {
|
||||
"--base-path"}, description = "Basepath of hoodie dataset to sync", required = true)
|
||||
public String basePath;
|
||||
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
@Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by")
|
||||
public List<String> partitionFields = new ArrayList<>();
|
||||
|
||||
@Parameter(names = "-partition-value-extractor", description = "Class which implements PartitionValueExtractor to extract the partition values from HDFS path")
|
||||
public String partitionValueExtractorClass = SlashEncodedDayPartitionValueExtractor.class
|
||||
.getName();
|
||||
|
||||
@Parameter(names = {
|
||||
"--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this exists to support backward compatibility. If you use hoodie 0.3.x, do not set this parameter")
|
||||
public Boolean assumeDatePartitioning = false;
|
||||
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
}
|
||||
|
||||
@@ -19,64 +19,161 @@
|
||||
package com.uber.hoodie.hive;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.uber.hoodie.hive.impl.DayBasedPartitionStrategy;
|
||||
import com.uber.hoodie.hive.impl.ParseSchemaFromDataStrategy;
|
||||
import com.uber.hoodie.hive.model.HoodieDatasetReference;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.InvalidDatasetException;
|
||||
import com.uber.hoodie.hadoop.HoodieInputFormat;
|
||||
import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat;
|
||||
import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent;
|
||||
import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent.PartitionEventType;
|
||||
import com.uber.hoodie.hive.util.SchemaUtil;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hadoop.hive.metastore.api.Partition;
|
||||
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat;
|
||||
import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import parquet.schema.MessageType;
|
||||
|
||||
/**
|
||||
* Tool to sync new data from commits, into Hive in terms of
|
||||
* Tool to sync a hoodie HDFS dataset with a hive metastore table.
|
||||
* Either use it as a api HiveSyncTool.syncHoodieTable(HiveSyncConfig)
|
||||
* or as a command line java -cp hoodie-hive.jar HiveSyncTool [args]
|
||||
*
|
||||
* - New table/partitions
|
||||
* - Updated schema for table/partitions
|
||||
* This utility will get the schema from the latest commit and will sync hive table schema
|
||||
* Also this will sync the partitions incrementally
|
||||
* (all the partitions modified since the last commit)
|
||||
*/
|
||||
@SuppressWarnings("WeakerAccess")
|
||||
public class HiveSyncTool {
|
||||
|
||||
private static Logger LOG = LoggerFactory.getLogger(HiveSyncTool.class);
|
||||
private final HoodieHiveClient hoodieHiveClient;
|
||||
private final HiveSyncConfig cfg;
|
||||
|
||||
/**
|
||||
* Sync to Hive, based on day based partitioning
|
||||
*
|
||||
* @param cfg
|
||||
*/
|
||||
public static void sync(HiveSyncConfig cfg) {
|
||||
// Configure to point to which metastore and database to connect to
|
||||
HoodieHiveConfiguration apiConfig =
|
||||
HoodieHiveConfiguration.newBuilder().hadoopConfiguration(new Configuration())
|
||||
.hivedb(cfg.databaseName)
|
||||
.hiveJdbcUrl(cfg.jdbcUrl)
|
||||
.jdbcUsername(cfg.hiveUser)
|
||||
.jdbcPassword(cfg.hivePass)
|
||||
.build();
|
||||
public HiveSyncTool(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) {
|
||||
this.hoodieHiveClient = new HoodieHiveClient(cfg, configuration, fs);
|
||||
this.cfg = cfg;
|
||||
}
|
||||
|
||||
HoodieDatasetReference datasetReference =
|
||||
new HoodieDatasetReference(cfg.tableName, cfg.basePath, cfg.databaseName);
|
||||
public void syncHoodieTable() {
|
||||
LOG.info("Trying to sync hoodie table" + cfg.tableName + " with base path " + hoodieHiveClient
|
||||
.getBasePath() + " of type " + hoodieHiveClient
|
||||
.getTableType());
|
||||
// Check if the necessary table exists
|
||||
boolean tableExists = hoodieHiveClient.doesTableExist();
|
||||
// Get the parquet schema for this dataset looking at the latest commit
|
||||
MessageType schema = hoodieHiveClient.getDataSchema();
|
||||
// Sync schema if needed
|
||||
syncSchema(tableExists, schema);
|
||||
|
||||
// initialize the strategies
|
||||
PartitionStrategy partitionStrategy = new DayBasedPartitionStrategy();
|
||||
SchemaStrategy schemaStrategy = new ParseSchemaFromDataStrategy();
|
||||
|
||||
// Creates a new dataset which reflects the state at the time of creation
|
||||
HoodieHiveDatasetSyncTask datasetSyncTask =
|
||||
HoodieHiveDatasetSyncTask.newBuilder().withReference(datasetReference)
|
||||
.withConfiguration(apiConfig).partitionStrategy(partitionStrategy)
|
||||
.schemaStrategy(schemaStrategy).build();
|
||||
|
||||
// Sync dataset
|
||||
datasetSyncTask.sync();
|
||||
LOG.info("Schema sync complete. Syncing partitions for " + cfg.tableName);
|
||||
// Get the last time we successfully synced partitions
|
||||
Optional<String> lastCommitTimeSynced = Optional.empty();
|
||||
if (tableExists) {
|
||||
lastCommitTimeSynced = hoodieHiveClient.getLastCommitTimeSynced();
|
||||
}
|
||||
LOG.info("Last commit time synced was found to be " + lastCommitTimeSynced.orElse("null"));
|
||||
List<String> writtenPartitionsSince = hoodieHiveClient
|
||||
.getPartitionsWrittenToSince(lastCommitTimeSynced);
|
||||
LOG.info("Storage partitions scan complete. Found " + writtenPartitionsSince.size());
|
||||
// Sync the partitions if needed
|
||||
syncPartitions(writtenPartitionsSince);
|
||||
|
||||
hoodieHiveClient.updateLastCommitTimeSynced();
|
||||
LOG.info("Sync complete for " + cfg.tableName);
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
hoodieHiveClient.close();
|
||||
}
|
||||
|
||||
// parse the params
|
||||
final HiveSyncConfig cfg = new HiveSyncConfig();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
|
||||
sync(cfg);
|
||||
/**
|
||||
* Get the latest schema from the last commit and check if its in sync with the hive table schema.
|
||||
* If not, evolves the table schema.
|
||||
*
|
||||
* @param tableExists - does table exist
|
||||
* @param schema - extracted schema
|
||||
*/
|
||||
private void syncSchema(boolean tableExists, MessageType schema) {
|
||||
// Check and sync schema
|
||||
if (!tableExists) {
|
||||
LOG.info("Table " + cfg.tableName + " is not found. Creating it");
|
||||
switch (hoodieHiveClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
hoodieHiveClient.createTable(schema, HoodieInputFormat.class.getName(),
|
||||
MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName());
|
||||
break;
|
||||
case MERGE_ON_READ:
|
||||
// create RT Table
|
||||
// Custom serde will not work with ALTER TABLE REPLACE COLUMNS
|
||||
// https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java#L3488
|
||||
// Need a fix to check instance of
|
||||
// hoodieHiveClient.createTable(schema, HoodieRealtimeInputFormat.class.getName(),
|
||||
// MapredParquetOutputFormat.class.getName(), HoodieParquetSerde.class.getName());
|
||||
hoodieHiveClient.createTable(schema, HoodieRealtimeInputFormat.class.getName(),
|
||||
MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName());
|
||||
// TODO - create RO Table
|
||||
break;
|
||||
default:
|
||||
LOG.error("Unknown table type " + hoodieHiveClient.getTableType());
|
||||
throw new InvalidDatasetException(hoodieHiveClient.getBasePath());
|
||||
}
|
||||
} else {
|
||||
// Check if the dataset schema has evolved
|
||||
Map<String, String> tableSchema = hoodieHiveClient.getTableSchema();
|
||||
SchemaDifference schemaDiff = SchemaUtil
|
||||
.getSchemaDifference(schema, tableSchema, cfg.partitionFields);
|
||||
if (!schemaDiff.isEmpty()) {
|
||||
LOG.info("Schema difference found for " + cfg.tableName);
|
||||
hoodieHiveClient.updateTableDefinition(schema);
|
||||
} else {
|
||||
LOG.info("No Schema difference for " + cfg.tableName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Syncs the list of storage parititions passed in (checks if the partition is in hive, if not
|
||||
* adds it or if the partition path does not match, it updates the partition path)
|
||||
*/
|
||||
private void syncPartitions(List<String> writtenPartitionsSince) {
|
||||
try {
|
||||
List<Partition> hivePartitions = hoodieHiveClient.scanTablePartitions();
|
||||
List<PartitionEvent> partitionEvents = hoodieHiveClient
|
||||
.getPartitionEvents(hivePartitions, writtenPartitionsSince);
|
||||
List<String> newPartitions = filterPartitions(partitionEvents, PartitionEventType.ADD);
|
||||
LOG.info("New Partitions " + newPartitions);
|
||||
hoodieHiveClient.addPartitionsToTable(newPartitions);
|
||||
List<String> updatePartitions = filterPartitions(partitionEvents, PartitionEventType.UPDATE);
|
||||
LOG.info("Changed Partitions " + updatePartitions);
|
||||
hoodieHiveClient.updatePartitionsToTable(updatePartitions);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieHiveSyncException("Failed to sync partitions for table " + cfg.tableName,
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
private List<String> filterPartitions(List<PartitionEvent> events, PartitionEventType eventType) {
|
||||
return events.stream()
|
||||
.filter(s -> s.eventType == eventType).map(s -> s.storagePartition).collect(
|
||||
Collectors.toList());
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
// parse the params
|
||||
final HiveSyncConfig cfg = new HiveSyncConfig();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
HiveConf hiveConf = new HiveConf();
|
||||
hiveConf.addResource(fs.getConf());
|
||||
new HiveSyncTool(cfg, hiveConf, fs).syncHoodieTable();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,607 @@
|
||||
/*
|
||||
* 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 com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
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.Reader;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.InvalidDatasetException;
|
||||
import com.uber.hoodie.hive.util.SchemaUtil;
|
||||
import java.io.IOException;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DatabaseMetaData;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.dbcp.BasicDataSource;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
|
||||
import org.apache.hadoop.hive.metastore.api.MetaException;
|
||||
import org.apache.hadoop.hive.metastore.api.Partition;
|
||||
import org.apache.hadoop.hive.metastore.api.Table;
|
||||
import org.apache.hive.jdbc.HiveDriver;
|
||||
import org.apache.thrift.TException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import parquet.format.converter.ParquetMetadataConverter;
|
||||
import parquet.hadoop.ParquetFileReader;
|
||||
import parquet.hadoop.metadata.ParquetMetadata;
|
||||
import parquet.schema.MessageType;
|
||||
|
||||
@SuppressWarnings("ConstantConditions")
|
||||
public class HoodieHiveClient {
|
||||
|
||||
private static final String HOODIE_LAST_COMMIT_TIME_SYNC = "last_commit_time_sync";
|
||||
// Make sure we have the hive JDBC driver in classpath
|
||||
private static String driverName = HiveDriver.class.getName();
|
||||
|
||||
static {
|
||||
try {
|
||||
Class.forName(driverName);
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new IllegalStateException("Could not find " + driverName + " in classpath. ", e);
|
||||
}
|
||||
}
|
||||
|
||||
private static Logger LOG = LoggerFactory.getLogger(HoodieHiveClient.class);
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
private final HoodieTableType tableType;
|
||||
private final PartitionValueExtractor partitionValueExtractor;
|
||||
private HiveMetaStoreClient client;
|
||||
private HiveSyncConfig syncConfig;
|
||||
private FileSystem fs;
|
||||
private Connection connection;
|
||||
private HoodieTimeline activeTimeline;
|
||||
|
||||
HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) {
|
||||
this.syncConfig = cfg;
|
||||
this.fs = fs;
|
||||
this.metaClient = new HoodieTableMetaClient(fs, cfg.basePath, true);
|
||||
this.tableType = metaClient.getTableType();
|
||||
|
||||
LOG.info("Creating hive connection " + cfg.jdbcUrl);
|
||||
createHiveConnection();
|
||||
try {
|
||||
this.client = new HiveMetaStoreClient(configuration);
|
||||
} catch (MetaException e) {
|
||||
throw new HoodieHiveSyncException("Failed to create HiveMetaStoreClient", e);
|
||||
}
|
||||
|
||||
try {
|
||||
this.partitionValueExtractor = (PartitionValueExtractor) Class
|
||||
.forName(cfg.partitionValueExtractorClass).newInstance();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieHiveSyncException(
|
||||
"Failed to initialize PartitionValueExtractor class " + cfg.partitionValueExtractorClass,
|
||||
e);
|
||||
}
|
||||
|
||||
activeTimeline = metaClient.getActiveTimeline().getCommitsAndCompactionsTimeline()
|
||||
.filterCompletedInstants();
|
||||
}
|
||||
|
||||
public HoodieTimeline getActiveTimeline() {
|
||||
return activeTimeline;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add the (NEW) partitons to the table
|
||||
*/
|
||||
void addPartitionsToTable(List<String> partitionsToAdd) {
|
||||
if (partitionsToAdd.isEmpty()) {
|
||||
LOG.info("No partitions to add for " + syncConfig.tableName);
|
||||
return;
|
||||
}
|
||||
LOG.info("Adding partitions " + partitionsToAdd.size() + " to table " + syncConfig.tableName);
|
||||
String sql = constructAddPartitions(partitionsToAdd);
|
||||
updateHiveSQL(sql);
|
||||
}
|
||||
|
||||
/**
|
||||
* Partition path has changed - update the path for te following partitions
|
||||
*/
|
||||
void updatePartitionsToTable(List<String> changedPartitions) {
|
||||
if (changedPartitions.isEmpty()) {
|
||||
LOG.info("No partitions to change for " + syncConfig.tableName);
|
||||
return;
|
||||
}
|
||||
LOG.info("Changing partitions " + changedPartitions.size() + " on " + syncConfig.tableName);
|
||||
List<String> sqls = constructChangePartitions(changedPartitions);
|
||||
for (String sql : sqls) {
|
||||
updateHiveSQL(sql);
|
||||
}
|
||||
}
|
||||
|
||||
private String constructAddPartitions(List<String> partitions) {
|
||||
StringBuilder alterSQL = new StringBuilder("ALTER TABLE ");
|
||||
alterSQL.append(syncConfig.databaseName).append(".").append(syncConfig.tableName)
|
||||
.append(" ADD IF NOT EXISTS ");
|
||||
for (String partition : partitions) {
|
||||
|
||||
StringBuilder partBuilder = new StringBuilder();
|
||||
List<String> partitionValues = partitionValueExtractor
|
||||
.extractPartitionValuesInPath(partition);
|
||||
Preconditions.checkArgument(syncConfig.partitionFields.size() == partitionValues.size(),
|
||||
"Partition key parts " + syncConfig.partitionFields
|
||||
+ " does not match with partition values " + partitionValues
|
||||
+ ". Check partition strategy. ");
|
||||
for (int i = 0; i < syncConfig.partitionFields.size(); i++) {
|
||||
partBuilder.append(syncConfig.partitionFields.get(i)).append("=").append("'")
|
||||
.append(partitionValues.get(i)).append("'");
|
||||
}
|
||||
|
||||
String fullPartitionPath = new Path(syncConfig.basePath, partition).toString();
|
||||
alterSQL.append(" PARTITION (").append(partBuilder.toString()).append(") LOCATION '")
|
||||
.append(fullPartitionPath).append("' ");
|
||||
}
|
||||
return alterSQL.toString();
|
||||
}
|
||||
|
||||
private List<String> constructChangePartitions(List<String> partitions) {
|
||||
List<String> changePartitions = Lists.newArrayList();
|
||||
String alterTable = "ALTER TABLE " + syncConfig.databaseName + "." + syncConfig.tableName;
|
||||
for (String partition : partitions) {
|
||||
StringBuilder partBuilder = new StringBuilder();
|
||||
List<String> partitionValues = partitionValueExtractor
|
||||
.extractPartitionValuesInPath(partition);
|
||||
Preconditions.checkArgument(syncConfig.partitionFields.size() == partitionValues.size(),
|
||||
"Partition key parts " + syncConfig.partitionFields
|
||||
+ " does not match with partition values " + partitionValues
|
||||
+ ". Check partition strategy. ");
|
||||
for (int i = 0; i < syncConfig.partitionFields.size(); i++) {
|
||||
partBuilder.append(syncConfig.partitionFields.get(i)).append("=").append("'")
|
||||
.append(partitionValues.get(i)).append("'");
|
||||
}
|
||||
|
||||
String fullPartitionPath = new Path(syncConfig.basePath, partition).toString();
|
||||
String changePartition =
|
||||
alterTable + " PARTITION (" + partBuilder.toString() + ") SET LOCATION '"
|
||||
+ "hdfs://nameservice1" + fullPartitionPath + "'";
|
||||
changePartitions.add(changePartition);
|
||||
}
|
||||
return changePartitions;
|
||||
}
|
||||
|
||||
/**
|
||||
* Iterate over the storage partitions and find if there are any new partitions that need
|
||||
* to be added or updated. Generate a list of PartitionEvent based on the changes required.
|
||||
*/
|
||||
List<PartitionEvent> getPartitionEvents(List<Partition> tablePartitions,
|
||||
List<String> partitionStoragePartitions) {
|
||||
Map<String, String> paths = Maps.newHashMap();
|
||||
for (Partition tablePartition : tablePartitions) {
|
||||
List<String> hivePartitionValues = tablePartition.getValues();
|
||||
Collections.sort(hivePartitionValues);
|
||||
String fullTablePartitionPath = Path
|
||||
.getPathWithoutSchemeAndAuthority(new Path(tablePartition.getSd().getLocation())).toUri()
|
||||
.getPath();
|
||||
paths.put(String.join(", ", hivePartitionValues), fullTablePartitionPath);
|
||||
}
|
||||
|
||||
List<PartitionEvent> events = Lists.newArrayList();
|
||||
for (String storagePartition : partitionStoragePartitions) {
|
||||
String fullStoragePartitionPath = new Path(syncConfig.basePath, storagePartition).toString();
|
||||
// Check if the partition values or if hdfs path is the same
|
||||
List<String> storagePartitionValues = partitionValueExtractor
|
||||
.extractPartitionValuesInPath(storagePartition);
|
||||
Collections.sort(storagePartitionValues);
|
||||
String storageValue = String.join(", ", storagePartitionValues);
|
||||
if (!paths.containsKey(storageValue)) {
|
||||
events.add(PartitionEvent.newPartitionAddEvent(storagePartition));
|
||||
} else if (!paths.get(storageValue).equals(fullStoragePartitionPath)) {
|
||||
events.add(PartitionEvent.newPartitionUpdateEvent(storagePartition));
|
||||
}
|
||||
}
|
||||
return events;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Scan table partitions
|
||||
*/
|
||||
List<Partition> scanTablePartitions() throws TException {
|
||||
return client
|
||||
.listPartitions(syncConfig.databaseName, syncConfig.tableName, (short) -1);
|
||||
}
|
||||
|
||||
void updateTableDefinition(MessageType newSchema) {
|
||||
try {
|
||||
String newSchemaStr = SchemaUtil.generateSchemaString(newSchema);
|
||||
// Cascade clause should not be present for non-partitioned tables
|
||||
String cascadeClause = syncConfig.partitionFields.size() > 0 ? " cascade" : "";
|
||||
StringBuilder sqlBuilder = new StringBuilder("ALTER TABLE ").append("`")
|
||||
.append(syncConfig.databaseName).append(".").append(syncConfig.tableName).append("`")
|
||||
.append(" REPLACE COLUMNS(")
|
||||
.append(newSchemaStr).append(" )").append(cascadeClause);
|
||||
LOG.info("Creating table with " + sqlBuilder);
|
||||
updateHiveSQL(sqlBuilder.toString());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieHiveSyncException("Failed to update table for " + syncConfig.tableName, e);
|
||||
}
|
||||
}
|
||||
|
||||
void createTable(MessageType storageSchema,
|
||||
String inputFormatClass, String outputFormatClass, String serdeClass) {
|
||||
try {
|
||||
String createSQLQuery = SchemaUtil
|
||||
.generateCreateDDL(storageSchema, syncConfig, inputFormatClass,
|
||||
outputFormatClass, serdeClass);
|
||||
LOG.info("Creating table with " + createSQLQuery);
|
||||
updateHiveSQL(createSQLQuery);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieHiveSyncException("Failed to create table " + syncConfig.tableName, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the table schema
|
||||
*/
|
||||
Map<String, String> getTableSchema() {
|
||||
if (!doesTableExist()) {
|
||||
throw new IllegalArgumentException(
|
||||
"Failed to get schema for table " + syncConfig.tableName + " does not exist");
|
||||
}
|
||||
Map<String, String> schema = Maps.newHashMap();
|
||||
ResultSet result = null;
|
||||
try {
|
||||
DatabaseMetaData databaseMetaData = connection.getMetaData();
|
||||
result = databaseMetaData
|
||||
.getColumns(null, syncConfig.databaseName, syncConfig.tableName, null);
|
||||
while (result.next()) {
|
||||
String columnName = result.getString(4);
|
||||
String columnType = result.getString(6);
|
||||
schema.put(columnName, columnType);
|
||||
}
|
||||
return schema;
|
||||
} catch (SQLException e) {
|
||||
throw new HoodieHiveSyncException(
|
||||
"Failed to get table schema for " + syncConfig.tableName, e);
|
||||
} finally {
|
||||
closeQuietly(result, null);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the schema for a hoodie dataset.
|
||||
* Depending on the type of table, read from any file written in the latest commit.
|
||||
* We will assume that the schema has not changed within a single atomic write.
|
||||
*
|
||||
* @return Parquet schema for this dataset
|
||||
*/
|
||||
@SuppressWarnings("WeakerAccess")
|
||||
public MessageType getDataSchema() {
|
||||
try {
|
||||
switch (tableType) {
|
||||
case COPY_ON_WRITE:
|
||||
// If this is COW, get the last commit and read the schema from a file written in the last commit
|
||||
HoodieInstant lastCommit = activeTimeline.lastInstant()
|
||||
.orElseThrow(() -> new InvalidDatasetException(syncConfig.basePath));
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(activeTimeline.getInstantDetails(lastCommit).get());
|
||||
String filePath = commitMetadata.getFileIdAndFullPaths().values().stream().findAny()
|
||||
.orElseThrow(() -> new IllegalArgumentException(
|
||||
"Could not find any data file written for commit " + lastCommit
|
||||
+ ", could not get schema for dataset " + metaClient.getBasePath()));
|
||||
return readSchemaFromDataFile(new Path(filePath));
|
||||
case MERGE_ON_READ:
|
||||
// If this is MOR, depending on whether the latest commit is a delta commit or compaction commit
|
||||
// Get a datafile written and get the schema from that file
|
||||
Optional<HoodieInstant> lastCompactionCommit = metaClient.getActiveTimeline()
|
||||
.getCompactionTimeline().filterCompletedInstants().lastInstant();
|
||||
LOG.info("Found the last compaction commit as " + lastCompactionCommit);
|
||||
|
||||
Optional<HoodieInstant> lastDeltaCommitAfterCompaction = Optional.empty();
|
||||
if (lastCompactionCommit.isPresent()) {
|
||||
lastDeltaCommitAfterCompaction = metaClient.getActiveTimeline()
|
||||
.getDeltaCommitTimeline()
|
||||
.filterCompletedInstants()
|
||||
.findInstantsAfter(lastCompactionCommit.get().getTimestamp(), Integer.MAX_VALUE).lastInstant();
|
||||
}
|
||||
LOG.info("Found the last delta commit after last compaction as "
|
||||
+ lastDeltaCommitAfterCompaction);
|
||||
|
||||
if (lastDeltaCommitAfterCompaction.isPresent()) {
|
||||
HoodieInstant lastDeltaCommit = lastDeltaCommitAfterCompaction.get();
|
||||
// read from the log file wrote
|
||||
commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(activeTimeline.getInstantDetails(lastDeltaCommit).get());
|
||||
filePath = commitMetadata.getFileIdAndFullPaths().values().stream().filter(s -> s.contains(
|
||||
HoodieLogFile.DELTA_EXTENSION)).findAny()
|
||||
.orElseThrow(() -> new IllegalArgumentException(
|
||||
"Could not find any data file written for commit " + lastDeltaCommit
|
||||
+ ", could not get schema for dataset " + metaClient.getBasePath()));
|
||||
return readSchemaFromLogFile(lastCompactionCommit, new Path(filePath));
|
||||
} else {
|
||||
return readSchemaFromLastCompaction(lastCompactionCommit);
|
||||
}
|
||||
default:
|
||||
LOG.error("Unknown table type " + tableType);
|
||||
throw new InvalidDatasetException(syncConfig.basePath);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieHiveSyncException(
|
||||
"Failed to get dataset schema for " + syncConfig.tableName, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Read schema from a data file from the last compaction commit done.
|
||||
*
|
||||
* @param lastCompactionCommitOpt
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
|
||||
private MessageType readSchemaFromLastCompaction(Optional<HoodieInstant> lastCompactionCommitOpt)
|
||||
throws IOException {
|
||||
HoodieInstant lastCompactionCommit = lastCompactionCommitOpt.orElseThrow(
|
||||
() -> new HoodieHiveSyncException(
|
||||
"Could not read schema from last compaction, no compaction commits found on path "
|
||||
+ syncConfig.basePath));
|
||||
|
||||
// Read from the compacted file wrote
|
||||
HoodieCompactionMetadata compactionMetadata = HoodieCompactionMetadata
|
||||
.fromBytes(activeTimeline.getInstantDetails(lastCompactionCommit).get());
|
||||
String filePath = compactionMetadata.getFileIdAndFullPaths().values().stream().findAny()
|
||||
.orElseThrow(() -> new IllegalArgumentException(
|
||||
"Could not find any data file written for compaction " + lastCompactionCommit
|
||||
+ ", could not get schema for dataset " + metaClient.getBasePath()));
|
||||
return readSchemaFromDataFile(new Path(filePath));
|
||||
}
|
||||
|
||||
/**
|
||||
* Read the schema from the log file on path
|
||||
*
|
||||
* @param lastCompactionCommitOpt
|
||||
* @param path
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
|
||||
private MessageType readSchemaFromLogFile(Optional<HoodieInstant> lastCompactionCommitOpt,
|
||||
Path path) throws IOException {
|
||||
Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null);
|
||||
HoodieAvroDataBlock lastBlock = null;
|
||||
while (reader.hasNext()) {
|
||||
HoodieLogBlock block = reader.next();
|
||||
if (block instanceof HoodieAvroDataBlock) {
|
||||
lastBlock = (HoodieAvroDataBlock) block;
|
||||
}
|
||||
}
|
||||
if (lastBlock != null) {
|
||||
return new parquet.avro.AvroSchemaConverter().convert(lastBlock.getSchema());
|
||||
}
|
||||
// Fall back to read the schema from last compaction
|
||||
LOG.info("Falling back to read the schema from last compaction " + lastCompactionCommitOpt);
|
||||
return readSchemaFromLastCompaction(lastCompactionCommitOpt);
|
||||
}
|
||||
|
||||
/**
|
||||
* Read the parquet schema from a parquet File
|
||||
*/
|
||||
private MessageType readSchemaFromDataFile(Path parquetFilePath) throws IOException {
|
||||
LOG.info("Reading schema from " + parquetFilePath);
|
||||
if (!fs.exists(parquetFilePath)) {
|
||||
throw new IllegalArgumentException(
|
||||
"Failed to read schema from data file " + parquetFilePath
|
||||
+ ". File does not exist.");
|
||||
}
|
||||
ParquetMetadata fileFooter =
|
||||
ParquetFileReader.readFooter(fs.getConf(), parquetFilePath, ParquetMetadataConverter.NO_FILTER);
|
||||
return fileFooter.getFileMetaData().getSchema();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if the configured table exists
|
||||
*/
|
||||
boolean doesTableExist() {
|
||||
try {
|
||||
return client.tableExists(syncConfig.databaseName, syncConfig.tableName);
|
||||
} catch (TException e) {
|
||||
throw new HoodieHiveSyncException(
|
||||
"Failed to check if table exists " + syncConfig.tableName, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Execute a update in hive metastore with this SQL
|
||||
*
|
||||
* @param s SQL to execute
|
||||
*/
|
||||
void updateHiveSQL(String s) {
|
||||
Statement stmt = null;
|
||||
try {
|
||||
stmt = connection.createStatement();
|
||||
LOG.info("Executing SQL " + s);
|
||||
stmt.execute(s);
|
||||
} catch (SQLException e) {
|
||||
throw new HoodieHiveSyncException("Failed in executing SQL " + s, e);
|
||||
} finally {
|
||||
closeQuietly(null, stmt);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private void createHiveConnection() {
|
||||
if (connection == null) {
|
||||
BasicDataSource ds = new BasicDataSource();
|
||||
ds.setDriverClassName(driverName);
|
||||
ds.setUrl(getHiveJdbcUrlWithDefaultDBName());
|
||||
ds.setUsername(syncConfig.hiveUser);
|
||||
ds.setPassword(syncConfig.hivePass);
|
||||
LOG.info("Getting Hive Connection from Datasource " + ds);
|
||||
try {
|
||||
this.connection = ds.getConnection();
|
||||
} catch (SQLException e) {
|
||||
throw new HoodieHiveSyncException(
|
||||
"Cannot create hive connection " + getHiveJdbcUrlWithDefaultDBName(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private String getHiveJdbcUrlWithDefaultDBName() {
|
||||
String hiveJdbcUrl = syncConfig.jdbcUrl;
|
||||
String urlAppend = null;
|
||||
// If the hive url contains addition properties like ;transportMode=http;httpPath=hs2
|
||||
if (hiveJdbcUrl.contains(";")) {
|
||||
urlAppend = hiveJdbcUrl.substring(hiveJdbcUrl.indexOf(";"));
|
||||
hiveJdbcUrl = hiveJdbcUrl.substring(0, hiveJdbcUrl.indexOf(";"));
|
||||
}
|
||||
if (!hiveJdbcUrl.endsWith("/")) {
|
||||
hiveJdbcUrl = hiveJdbcUrl + "/";
|
||||
}
|
||||
return hiveJdbcUrl + syncConfig.databaseName + (urlAppend == null ? "" : urlAppend);
|
||||
}
|
||||
|
||||
private static void closeQuietly(ResultSet resultSet, Statement stmt) {
|
||||
try {
|
||||
if (stmt != null) {
|
||||
stmt.close();
|
||||
}
|
||||
if (resultSet != null) {
|
||||
resultSet.close();
|
||||
}
|
||||
} catch (SQLException e) {
|
||||
LOG.error("Could not close the resultset opened ", e);
|
||||
}
|
||||
}
|
||||
|
||||
public String getBasePath() {
|
||||
return metaClient.getBasePath();
|
||||
}
|
||||
|
||||
HoodieTableType getTableType() {
|
||||
return tableType;
|
||||
}
|
||||
|
||||
public FileSystem getFs() {
|
||||
return fs;
|
||||
}
|
||||
|
||||
Optional<String> getLastCommitTimeSynced() {
|
||||
// Get the last commit time from the TBLproperties
|
||||
try {
|
||||
Table database = client.getTable(syncConfig.databaseName, syncConfig.tableName);
|
||||
return Optional
|
||||
.ofNullable(database.getParameters().getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, null));
|
||||
} catch (Exception e) {
|
||||
throw new HoodieHiveSyncException(
|
||||
"Failed to get the last commit time synced from the database", e);
|
||||
}
|
||||
}
|
||||
|
||||
void close() {
|
||||
try {
|
||||
if (connection != null) {
|
||||
connection.close();
|
||||
}
|
||||
if(client != null) {
|
||||
client.close();
|
||||
}
|
||||
} catch (SQLException e) {
|
||||
LOG.error("Could not close connection ", e);
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
|
||||
List<String> getPartitionsWrittenToSince(Optional<String> lastCommitTimeSynced) {
|
||||
if (!lastCommitTimeSynced.isPresent()) {
|
||||
LOG.info("Last commit time synced is not known, listing all partitions");
|
||||
try {
|
||||
return FSUtils
|
||||
.getAllPartitionPaths(fs, syncConfig.basePath, syncConfig.assumeDatePartitioning);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to list all partitions in " + syncConfig.basePath, e);
|
||||
}
|
||||
} else {
|
||||
LOG.info("Last commit time synced is " + lastCommitTimeSynced.get()
|
||||
+ ", Getting commits since then");
|
||||
|
||||
HoodieTimeline timelineToSync = activeTimeline
|
||||
.findInstantsAfter(lastCommitTimeSynced.get(), Integer.MAX_VALUE);
|
||||
return timelineToSync.getInstants().map(s -> {
|
||||
try {
|
||||
return HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(s).get());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Failed to get partitions written since " + lastCommitTimeSynced, e);
|
||||
}
|
||||
}).flatMap(s -> s.getPartitionToWriteStats().keySet().stream()).distinct()
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
|
||||
void updateLastCommitTimeSynced() {
|
||||
// Set the last commit time from the TBLproperties
|
||||
String lastCommitSynced = activeTimeline.lastInstant().get().getTimestamp();
|
||||
try {
|
||||
Table table = client.getTable(syncConfig.databaseName, syncConfig.tableName);
|
||||
table.putToParameters(HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitSynced);
|
||||
client.alter_table(syncConfig.databaseName, syncConfig.tableName, table, true);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieHiveSyncException(
|
||||
"Failed to get update last commit time synced to " + lastCommitSynced, e);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Partition Event captures any partition that needs to be added or updated
|
||||
*/
|
||||
static class PartitionEvent {
|
||||
|
||||
public enum PartitionEventType {ADD, UPDATE}
|
||||
|
||||
PartitionEventType eventType;
|
||||
String storagePartition;
|
||||
|
||||
PartitionEvent(
|
||||
PartitionEventType eventType, String storagePartition) {
|
||||
this.eventType = eventType;
|
||||
this.storagePartition = storagePartition;
|
||||
}
|
||||
|
||||
static PartitionEvent newPartitionAddEvent(String storagePartition) {
|
||||
return new PartitionEvent(PartitionEventType.ADD, storagePartition);
|
||||
}
|
||||
|
||||
static PartitionEvent newPartitionUpdateEvent(String storagePartition) {
|
||||
return new PartitionEvent(PartitionEventType.UPDATE, storagePartition);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,119 +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 org.apache.hadoop.conf.Configuration;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Configurations for registering a hoodie dataset into hive metastore
|
||||
*/
|
||||
public class HoodieHiveConfiguration {
|
||||
private final String hiveJdbcUrl;
|
||||
private final String dbName;
|
||||
private final String hiveUsername;
|
||||
private final String hivePassword;
|
||||
private final Configuration configuration;
|
||||
|
||||
private HoodieHiveConfiguration(String hiveJdbcUrl, String defaultDatabaseName,
|
||||
String hiveUsername, String hivePassword, Configuration configuration) {
|
||||
this.hiveJdbcUrl = hiveJdbcUrl;
|
||||
this.dbName = defaultDatabaseName;
|
||||
this.hiveUsername = hiveUsername;
|
||||
this.hivePassword = hivePassword;
|
||||
this.configuration = configuration;
|
||||
}
|
||||
|
||||
public String getHiveJdbcUrl() {
|
||||
return hiveJdbcUrl;
|
||||
}
|
||||
|
||||
public String getDbName() {
|
||||
return dbName;
|
||||
}
|
||||
|
||||
public String getHiveUsername() {
|
||||
return hiveUsername;
|
||||
}
|
||||
|
||||
public String getHivePassword() {
|
||||
return hivePassword;
|
||||
}
|
||||
|
||||
public Configuration getConfiguration() {
|
||||
return configuration;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieHiveConfiguration{");
|
||||
sb.append("hiveJdbcUrl='").append(hiveJdbcUrl).append('\'');
|
||||
sb.append(", dbName='").append(dbName).append('\'');
|
||||
sb.append(", hiveUsername='").append(hiveUsername).append('\'');
|
||||
sb.append(", hivePassword='").append(hivePassword).append('\'');
|
||||
sb.append(", configuration=").append(configuration);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public static Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
private static Logger LOG = LoggerFactory.getLogger(Builder.class);
|
||||
private String hiveJdbcUrl;
|
||||
private String dbName;
|
||||
private String jdbcUsername;
|
||||
private String jdbcPassword;
|
||||
private Configuration configuration;
|
||||
|
||||
public Builder hiveJdbcUrl(String hiveJdbcUrl) {
|
||||
this.hiveJdbcUrl = hiveJdbcUrl;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hivedb(String hiveDatabase) {
|
||||
this.dbName = hiveDatabase;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder jdbcUsername(String jdbcUsername) {
|
||||
this.jdbcUsername = jdbcUsername;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder jdbcPassword(String jdbcPassword) {
|
||||
this.jdbcPassword = jdbcPassword;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hadoopConfiguration(Configuration configuration) {
|
||||
this.configuration = configuration;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHiveConfiguration build() {
|
||||
HoodieHiveConfiguration config =
|
||||
new HoodieHiveConfiguration(hiveJdbcUrl, dbName, jdbcUsername, jdbcPassword,
|
||||
configuration);
|
||||
LOG.info("Hoodie Hive Configuration - " + config);
|
||||
return config;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,182 +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.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.uber.hoodie.hive.client.HoodieFSClient;
|
||||
import com.uber.hoodie.hive.client.HoodieHiveClient;
|
||||
import com.uber.hoodie.hive.model.HoodieDatasetReference;
|
||||
import com.uber.hoodie.hive.model.StoragePartition;
|
||||
import com.uber.hoodie.hive.model.TablePartition;
|
||||
import org.apache.commons.lang.ArrayUtils;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Represents a Hive External Dataset.
|
||||
* Contains metadata for storage and table partitions.
|
||||
*/
|
||||
public class HoodieHiveDatasetSyncTask {
|
||||
private static Logger LOG = LoggerFactory.getLogger(HoodieHiveDatasetSyncTask.class);
|
||||
private final HoodieHiveSchemaSyncTask schemaSyncTask;
|
||||
private final List<StoragePartition> newPartitions;
|
||||
private final List<StoragePartition> changedPartitions;
|
||||
|
||||
public HoodieHiveDatasetSyncTask(HoodieHiveSchemaSyncTask schemaSyncTask,
|
||||
List<StoragePartition> newPartitions, List<StoragePartition> changedPartitions) {
|
||||
this.schemaSyncTask = schemaSyncTask;
|
||||
this.newPartitions = ImmutableList.copyOf(newPartitions);
|
||||
this.changedPartitions = ImmutableList.copyOf(changedPartitions);
|
||||
}
|
||||
|
||||
public HoodieHiveSchemaSyncTask getSchemaSyncTask() {
|
||||
return schemaSyncTask;
|
||||
}
|
||||
|
||||
public List<StoragePartition> getNewPartitions() {
|
||||
return newPartitions;
|
||||
}
|
||||
|
||||
public List<StoragePartition> getChangedPartitions() {
|
||||
return changedPartitions;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sync this dataset
|
||||
* 1. If any schema difference is found, then sync the table schema
|
||||
* 2. If any new partitions are found, adds partitions to the table (which uses the table schema by default)
|
||||
* 3. If any partition path has changed, modify the partition to the new path (which does not change the partition schema)
|
||||
*/
|
||||
public void sync() {
|
||||
LOG.info("Starting Sync for " + schemaSyncTask.getReference());
|
||||
try {
|
||||
// First sync the table schema
|
||||
schemaSyncTask.sync();
|
||||
|
||||
// Add all the new partitions
|
||||
schemaSyncTask.getHiveClient()
|
||||
.addPartitionsToTable(schemaSyncTask.getReference(), newPartitions,
|
||||
schemaSyncTask.getPartitionStrategy());
|
||||
// Update all the changed partitions
|
||||
schemaSyncTask.getHiveClient()
|
||||
.updatePartitionsToTable(schemaSyncTask.getReference(), changedPartitions,
|
||||
schemaSyncTask.getPartitionStrategy());
|
||||
} catch (Exception e) {
|
||||
throw new HoodieHiveDatasetException(
|
||||
"Failed to sync dataset " + schemaSyncTask.getReference(), e);
|
||||
}
|
||||
LOG.info("Sync for " + schemaSyncTask.getReference() + " complete.");
|
||||
}
|
||||
|
||||
public static Builder newBuilder(HoodieHiveDatasetSyncTask dataset) {
|
||||
return newBuilder().withConfiguration(dataset.schemaSyncTask.getConf())
|
||||
.withReference(dataset.schemaSyncTask.getReference())
|
||||
.withFSClient(dataset.schemaSyncTask.getFsClient())
|
||||
.withHiveClient(dataset.schemaSyncTask.getHiveClient())
|
||||
.schemaStrategy(dataset.schemaSyncTask.getSchemaStrategy())
|
||||
.partitionStrategy(dataset.schemaSyncTask.getPartitionStrategy());
|
||||
}
|
||||
|
||||
public static Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
private static Logger LOG = LoggerFactory.getLogger(Builder.class);
|
||||
private HoodieHiveConfiguration configuration;
|
||||
private HoodieDatasetReference datasetReference;
|
||||
private SchemaStrategy schemaStrategy;
|
||||
private PartitionStrategy partitionStrategy;
|
||||
private HoodieHiveClient hiveClient;
|
||||
private HoodieFSClient fsClient;
|
||||
|
||||
public Builder withReference(HoodieDatasetReference reference) {
|
||||
this.datasetReference = reference;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withConfiguration(HoodieHiveConfiguration configuration) {
|
||||
this.configuration = configuration;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder schemaStrategy(SchemaStrategy schemaStrategy) {
|
||||
this.schemaStrategy = schemaStrategy;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder partitionStrategy(PartitionStrategy partitionStrategy) {
|
||||
if(partitionStrategy != null) {
|
||||
LOG.info("Partitioning the dataset with keys " + ArrayUtils
|
||||
.toString(partitionStrategy.getHivePartitionFieldNames()));
|
||||
}
|
||||
this.partitionStrategy = partitionStrategy;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withHiveClient(HoodieHiveClient hiveClient) {
|
||||
this.hiveClient = hiveClient;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withFSClient(HoodieFSClient fsClient) {
|
||||
this.fsClient = fsClient;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHiveDatasetSyncTask build() {
|
||||
LOG.info("Building dataset for " + datasetReference);
|
||||
HoodieHiveSchemaSyncTask schemaSyncTask =
|
||||
HoodieHiveSchemaSyncTask.newBuilder().withReference(datasetReference)
|
||||
.withConfiguration(configuration).schemaStrategy(schemaStrategy)
|
||||
.partitionStrategy(partitionStrategy).withHiveClient(hiveClient)
|
||||
.withFSClient(fsClient).build();
|
||||
|
||||
List<StoragePartition> storagePartitions = Lists.newArrayList();
|
||||
List<String> storagePartitionPaths = schemaSyncTask.getPartitionStrategy()
|
||||
.scanAllPartitions(schemaSyncTask.getReference(), schemaSyncTask.getFsClient());
|
||||
for (String path : storagePartitionPaths) {
|
||||
storagePartitions.add(new StoragePartition(schemaSyncTask.getReference(),
|
||||
schemaSyncTask.getPartitionStrategy(), path));
|
||||
}
|
||||
LOG.info("Storage partitions scan complete. Found " + storagePartitions.size());
|
||||
|
||||
List<StoragePartition> newPartitions;
|
||||
List<StoragePartition> changedPartitions;
|
||||
|
||||
// Check if table exists
|
||||
if (schemaSyncTask.getHiveClient().checkTableExists(schemaSyncTask.getReference())) {
|
||||
List<TablePartition> partitions =
|
||||
schemaSyncTask.getHiveClient().scanPartitions(schemaSyncTask.getReference());
|
||||
LOG.info("Table partition scan complete. Found " + partitions.size());
|
||||
newPartitions = schemaSyncTask.getFsClient()
|
||||
.getUnregisteredStoragePartitions(partitions, storagePartitions);
|
||||
changedPartitions = schemaSyncTask.getFsClient()
|
||||
.getChangedStoragePartitions(partitions, storagePartitions);
|
||||
} else {
|
||||
newPartitions = storagePartitions;
|
||||
changedPartitions = Lists.newArrayList();
|
||||
}
|
||||
return new HoodieHiveDatasetSyncTask(schemaSyncTask, newPartitions, changedPartitions);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,243 +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.google.common.base.Objects;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.uber.hoodie.hadoop.HoodieInputFormat;
|
||||
import com.uber.hoodie.hive.impl.DayBasedPartitionStrategy;
|
||||
import com.uber.hoodie.hive.client.HoodieFSClient;
|
||||
import com.uber.hoodie.hive.client.HoodieHiveClient;
|
||||
import com.uber.hoodie.hive.impl.ParseSchemaFromDataStrategy;
|
||||
import com.uber.hoodie.hive.client.SchemaUtil;
|
||||
import com.uber.hoodie.hive.model.HoodieDatasetReference;
|
||||
import com.uber.hoodie.hive.model.SchemaDifference;
|
||||
import org.apache.commons.lang.ArrayUtils;
|
||||
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import parquet.schema.MessageType;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Represents the Schema sync task for the dataset.
|
||||
* Execute sync() on this task to sync up the HDFS dataset schema and hive table schema
|
||||
*/
|
||||
public class HoodieHiveSchemaSyncTask {
|
||||
private static Logger LOG = LoggerFactory.getLogger(HoodieHiveSchemaSyncTask.class);
|
||||
|
||||
private static final String DEFAULT_INPUTFORMAT = HoodieInputFormat.class.getName();
|
||||
private static final String DEFAULT_OUTPUTFORMAT = MapredParquetOutputFormat.class.getName();
|
||||
|
||||
private final HoodieDatasetReference reference;
|
||||
private final MessageType storageSchema;
|
||||
private final Map<String, String> tableSchema;
|
||||
private final PartitionStrategy partitionStrategy;
|
||||
private final SchemaStrategy schemaStrategy;
|
||||
private final HoodieHiveClient hiveClient;
|
||||
private final HoodieHiveConfiguration conf;
|
||||
private final HoodieFSClient fsClient;
|
||||
|
||||
public HoodieHiveSchemaSyncTask(HoodieDatasetReference datasetReference,
|
||||
MessageType schemaInferred, Map<String, String> fieldsSchema,
|
||||
PartitionStrategy partitionStrategy, SchemaStrategy schemaStrategy,
|
||||
HoodieHiveConfiguration configuration, HoodieHiveClient hiveClient,
|
||||
HoodieFSClient fsClient) {
|
||||
this.reference = datasetReference;
|
||||
this.storageSchema = schemaInferred;
|
||||
this.tableSchema = fieldsSchema;
|
||||
this.partitionStrategy = partitionStrategy;
|
||||
this.schemaStrategy = schemaStrategy;
|
||||
this.hiveClient = hiveClient;
|
||||
this.conf = configuration;
|
||||
this.fsClient = fsClient;
|
||||
}
|
||||
|
||||
public SchemaDifference getSchemaDifference() {
|
||||
return SchemaUtil.getSchemaDifference(storageSchema, tableSchema,
|
||||
partitionStrategy.getHivePartitionFieldNames());
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the table schema is present. If not, creates one.
|
||||
* If already exists, computes the schema difference and if there is any difference
|
||||
* it generates a alter table and syncs up the schema to hive metastore.
|
||||
*/
|
||||
public void sync() {
|
||||
try {
|
||||
// Check if the table needs to be created
|
||||
if (tableSchema.isEmpty()) {
|
||||
// create the database
|
||||
LOG.info("Schema not found. Creating for " + reference);
|
||||
hiveClient.createTable(storageSchema, reference,
|
||||
partitionStrategy.getHivePartitionFieldNames(), DEFAULT_INPUTFORMAT,
|
||||
DEFAULT_OUTPUTFORMAT);
|
||||
} else {
|
||||
if (!getSchemaDifference().isEmpty()) {
|
||||
LOG.info("Schema sync required for " + reference);
|
||||
hiveClient.updateTableDefinition(reference,
|
||||
partitionStrategy.getHivePartitionFieldNames(), storageSchema);
|
||||
} else {
|
||||
LOG.info("Schema sync not required for " + reference);
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new HoodieHiveDatasetException("Failed to sync dataset " + reference,
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
public static Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public MessageType getStorageSchema() {
|
||||
return storageSchema;
|
||||
}
|
||||
|
||||
public Map<String, String> getTableSchema() {
|
||||
return tableSchema;
|
||||
}
|
||||
|
||||
public PartitionStrategy getPartitionStrategy() {
|
||||
return partitionStrategy;
|
||||
}
|
||||
|
||||
public SchemaStrategy getSchemaStrategy() {
|
||||
return schemaStrategy;
|
||||
}
|
||||
|
||||
public HoodieHiveClient getHiveClient() {
|
||||
return hiveClient;
|
||||
}
|
||||
|
||||
public HoodieHiveConfiguration getConf() {
|
||||
return conf;
|
||||
}
|
||||
|
||||
public HoodieDatasetReference getReference() {
|
||||
return reference;
|
||||
}
|
||||
|
||||
public HoodieFSClient getFsClient() {
|
||||
return fsClient;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o)
|
||||
return true;
|
||||
if (o == null || getClass() != o.getClass())
|
||||
return false;
|
||||
HoodieHiveSchemaSyncTask that = (HoodieHiveSchemaSyncTask) o;
|
||||
return Objects.equal(storageSchema, that.storageSchema) && Objects
|
||||
.equal(tableSchema, that.tableSchema);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(storageSchema, tableSchema);
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
private static Logger LOG = LoggerFactory.getLogger(Builder.class);
|
||||
private HoodieHiveConfiguration configuration;
|
||||
private HoodieDatasetReference datasetReference;
|
||||
private SchemaStrategy schemaStrategy;
|
||||
private PartitionStrategy partitionStrategy;
|
||||
private HoodieHiveClient hiveClient;
|
||||
private HoodieFSClient fsClient;
|
||||
|
||||
public Builder withReference(HoodieDatasetReference reference) {
|
||||
this.datasetReference = reference;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withConfiguration(HoodieHiveConfiguration configuration) {
|
||||
this.configuration = configuration;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder schemaStrategy(SchemaStrategy schemaStrategy) {
|
||||
this.schemaStrategy = schemaStrategy;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder partitionStrategy(PartitionStrategy partitionStrategy) {
|
||||
if(partitionStrategy != null) {
|
||||
LOG.info("Partitioning the dataset with keys " + ArrayUtils
|
||||
.toString(partitionStrategy.getHivePartitionFieldNames()));
|
||||
}
|
||||
this.partitionStrategy = partitionStrategy;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withHiveClient(HoodieHiveClient hiveClient) {
|
||||
this.hiveClient = hiveClient;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withFSClient(HoodieFSClient fsClient) {
|
||||
this.fsClient = fsClient;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHiveSchemaSyncTask build() {
|
||||
LOG.info("Building dataset schema for " + datasetReference);
|
||||
createDefaults();
|
||||
|
||||
MessageType schemaInferred =
|
||||
schemaStrategy.getDatasetSchema(datasetReference, fsClient);
|
||||
LOG.info("Storage Schema inferred for dataset " + datasetReference);
|
||||
LOG.debug("Inferred Storage Schema " + schemaInferred);
|
||||
|
||||
Map<String, String> fieldsSchema;
|
||||
if (!hiveClient.checkTableExists(datasetReference)) {
|
||||
fieldsSchema = Maps.newHashMap();
|
||||
} else {
|
||||
fieldsSchema = hiveClient.getTableSchema(datasetReference);
|
||||
}
|
||||
LOG.info("Table Schema inferred for dataset " + datasetReference);
|
||||
LOG.debug("Inferred Table Schema " + fieldsSchema);
|
||||
|
||||
return new HoodieHiveSchemaSyncTask(datasetReference, schemaInferred, fieldsSchema,
|
||||
partitionStrategy, schemaStrategy, configuration, hiveClient, fsClient);
|
||||
}
|
||||
|
||||
private void createDefaults() {
|
||||
if (partitionStrategy == null) {
|
||||
LOG.info("Partition strategy is not set. Selecting the default strategy");
|
||||
partitionStrategy = new DayBasedPartitionStrategy();
|
||||
}
|
||||
if (schemaStrategy == null) {
|
||||
LOG.info(
|
||||
"Schema strategy not specified. Selecting the default based on the dataset type");
|
||||
schemaStrategy = new ParseSchemaFromDataStrategy();
|
||||
}
|
||||
if (fsClient == null) {
|
||||
LOG.info("Creating a new FS Client as none has been passed in");
|
||||
fsClient = new HoodieFSClient(configuration);
|
||||
}
|
||||
if (hiveClient == null) {
|
||||
LOG.info("Creating a new Hive Client as none has been passed in");
|
||||
hiveClient = new HoodieHiveClient(configuration);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -16,21 +16,21 @@
|
||||
|
||||
package com.uber.hoodie.hive;
|
||||
|
||||
public class HoodieHiveDatasetException extends RuntimeException {
|
||||
public class HoodieHiveSyncException extends RuntimeException {
|
||||
|
||||
public HoodieHiveDatasetException() {
|
||||
public HoodieHiveSyncException() {
|
||||
super();
|
||||
}
|
||||
|
||||
public HoodieHiveDatasetException(String message) {
|
||||
public HoodieHiveSyncException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public HoodieHiveDatasetException(String message, Throwable t) {
|
||||
public HoodieHiveSyncException(String message, Throwable t) {
|
||||
super(message, t);
|
||||
}
|
||||
|
||||
public HoodieHiveDatasetException(Throwable t) {
|
||||
public HoodieHiveSyncException(Throwable t) {
|
||||
super(t);
|
||||
}
|
||||
|
||||
@@ -1,59 +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.HoodieFSClient;
|
||||
import com.uber.hoodie.hive.model.HoodieDatasetReference;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Abstraction to define HDFS partition strategies.
|
||||
* Strategy provides hookups to map partitions on to physical layout
|
||||
*
|
||||
* @see SchemaStrategy
|
||||
*/
|
||||
public interface PartitionStrategy {
|
||||
/**
|
||||
* Scans the file system for all partitions and returns String[] which are the available partitions, relative to
|
||||
* the base path
|
||||
*
|
||||
* @param basePath
|
||||
* @param fsClient
|
||||
* @return
|
||||
*/
|
||||
List<String> scanAllPartitions(HoodieDatasetReference basePath, HoodieFSClient fsClient);
|
||||
|
||||
/**
|
||||
* Get the list of hive field names the dataset will be partitioned on.
|
||||
* The field name should be present in the storage schema.
|
||||
*
|
||||
* @return List of partitions field names
|
||||
*/
|
||||
String[] getHivePartitionFieldNames();
|
||||
|
||||
/**
|
||||
* Convert a Partition path (returned in scanAllPartitions) to values for column names returned in getHivePartitionFieldNames
|
||||
* e.g. 2016/12/12/ will return [2016, 12, 12]
|
||||
*
|
||||
* @param partitionPath storage path
|
||||
* @return List of partitions field values
|
||||
*/
|
||||
String[] convertPartitionToValues(HoodieDatasetReference metadata, String partitionPath);
|
||||
}
|
||||
@@ -0,0 +1,31 @@
|
||||
/*
|
||||
* 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 java.util.List;
|
||||
|
||||
/**
|
||||
* HDFS Path contain hive partition values for the keys it is partitioned on.
|
||||
* This mapping is not straight forward and requires a pluggable implementation to extract the partition value from HDFS path.
|
||||
*
|
||||
* e.g. Hive table partitioned by datestr=yyyy-mm-dd and hdfs path /app/hoodie/dataset1/YYYY=[yyyy]/MM=[mm]/DD=[dd]
|
||||
*/
|
||||
public interface PartitionValueExtractor {
|
||||
List<String> extractPartitionValuesInPath(String partitionPath);
|
||||
}
|
||||
@@ -14,7 +14,7 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.hive.model;
|
||||
package com.uber.hoodie.hive;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
@@ -1,31 +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.HoodieFSClient;
|
||||
import com.uber.hoodie.hive.model.HoodieDatasetReference;
|
||||
import parquet.schema.MessageType;
|
||||
|
||||
/**
|
||||
* Abstraction to get the Parquet schema for a {@link HoodieDatasetReference}
|
||||
* If you are managing the schemas externally, connect to the system and get the schema.
|
||||
*
|
||||
* @see PartitionStrategy
|
||||
*/
|
||||
public interface SchemaStrategy {
|
||||
MessageType getDatasetSchema(HoodieDatasetReference metadata, HoodieFSClient fsClient);
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* 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 com.beust.jcommander.internal.Lists;
|
||||
import java.util.List;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.format.DateTimeFormat;
|
||||
import org.joda.time.format.DateTimeFormatter;
|
||||
/**
|
||||
* HDFS Path contain hive partition values for the keys it is partitioned on.
|
||||
* This mapping is not straight forward and requires a pluggable implementation to extract the partition value from HDFS path.
|
||||
*
|
||||
* This implementation extracts datestr=yyyy-mm-dd from path of type /yyyy/mm/dd
|
||||
*/
|
||||
public class SlashEncodedDayPartitionValueExtractor implements PartitionValueExtractor {
|
||||
|
||||
private final DateTimeFormatter dtfOut;
|
||||
|
||||
public SlashEncodedDayPartitionValueExtractor() {
|
||||
this.dtfOut = DateTimeFormat.forPattern("yyyy-MM-dd");
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> extractPartitionValuesInPath(String partitionPath) {
|
||||
// partition path is expected to be in this format yyyy/mm/dd
|
||||
String[] splits = partitionPath.split("/");
|
||||
if (splits.length != 3) {
|
||||
throw new IllegalArgumentException(
|
||||
"Partition path " + partitionPath + " is not in the form yyyy/mm/dd ");
|
||||
}
|
||||
// Get the partition part and remove the / as well at the end
|
||||
int year = Integer.parseInt(splits[0]);
|
||||
int mm = Integer.parseInt(splits[1]);
|
||||
int dd = Integer.parseInt(splits[2]);
|
||||
DateTime dateTime = new DateTime(year, mm, dd, 0, 0);
|
||||
return Lists.newArrayList(dtfOut.print(dateTime));
|
||||
}
|
||||
}
|
||||
@@ -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.client;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.hive.HoodieHiveConfiguration;
|
||||
import com.uber.hoodie.hive.HoodieHiveDatasetException;
|
||||
import com.uber.hoodie.hive.model.HoodieDatasetReference;
|
||||
import com.uber.hoodie.hive.model.StoragePartition;
|
||||
import com.uber.hoodie.hive.model.TablePartition;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import parquet.hadoop.ParquetFileReader;
|
||||
import parquet.hadoop.metadata.ParquetMetadata;
|
||||
import parquet.schema.MessageType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Client to access HDFS
|
||||
*/
|
||||
public class HoodieFSClient {
|
||||
final public static String PARQUET_EXTENSION = ".parquet";
|
||||
final public static String PARQUET_EXTENSION_ZIPPED = ".parquet.gz";
|
||||
private final static Logger LOG = LoggerFactory.getLogger(HoodieFSClient.class);
|
||||
private final HoodieHiveConfiguration conf;
|
||||
|
||||
private final FileSystem fs;
|
||||
|
||||
public HoodieFSClient(HoodieHiveConfiguration configuration) {
|
||||
this.conf = configuration;
|
||||
try {
|
||||
this.fs = FileSystem.get(configuration.getConfiguration());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieHiveDatasetException(
|
||||
"Could not initialize file system from configuration", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Read the parquet schema from a parquet File
|
||||
*
|
||||
* @param parquetFilePath
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
public MessageType readSchemaFromDataFile(Path parquetFilePath) throws IOException {
|
||||
LOG.info("Reading schema from " + parquetFilePath);
|
||||
|
||||
if (!fs.exists(parquetFilePath)) {
|
||||
throw new IllegalArgumentException(
|
||||
"Failed to read schema from data file " + parquetFilePath
|
||||
+ ". File does not exist.");
|
||||
}
|
||||
ParquetMetadata fileFooter =
|
||||
ParquetFileReader.readFooter(conf.getConfiguration(), parquetFilePath);
|
||||
return fileFooter.getFileMetaData().getSchema();
|
||||
}
|
||||
|
||||
/**
|
||||
* Find the last data file under the partition path.
|
||||
*
|
||||
* @param metadata
|
||||
* @param partitionPathString
|
||||
* @return
|
||||
*/
|
||||
public Path lastDataFileForDataset(HoodieDatasetReference metadata,
|
||||
String partitionPathString) {
|
||||
try {
|
||||
Path partitionPath = new Path(partitionPathString);
|
||||
if (!fs.exists(partitionPath)) {
|
||||
throw new HoodieHiveDatasetException(
|
||||
"Partition path " + partitionPath + " not found in Dataset " + metadata);
|
||||
}
|
||||
|
||||
RemoteIterator<LocatedFileStatus> files = fs.listFiles(partitionPath, true);
|
||||
// Iterate over the list. List is generally is listed in chronological order becasue of the date partitions
|
||||
// Get the latest schema
|
||||
Path returnPath = null;
|
||||
while (files.hasNext()) {
|
||||
Path path = files.next().getPath();
|
||||
if (path.getName().endsWith(PARQUET_EXTENSION) || path.getName()
|
||||
.endsWith(PARQUET_EXTENSION_ZIPPED)) {
|
||||
if(returnPath == null || path.toString().compareTo(returnPath.toString()) > 0) {
|
||||
returnPath = path;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (returnPath != null) {
|
||||
return returnPath;
|
||||
}
|
||||
throw new HoodieHiveDatasetException(
|
||||
"No data file found in path " + partitionPath + " for dataset " + metadata);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieHiveDatasetException(
|
||||
"Failed to get data file in path " + partitionPathString + " for dataset "
|
||||
+ metadata, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of storage partitions which does not have its equivalent hive partitions
|
||||
*
|
||||
* @param tablePartitions
|
||||
* @param storagePartitions
|
||||
* @return
|
||||
*/
|
||||
public List<StoragePartition> getUnregisteredStoragePartitions(
|
||||
List<TablePartition> tablePartitions, List<StoragePartition> storagePartitions) {
|
||||
Set<String> paths = Sets.newHashSet();
|
||||
for (TablePartition tablePartition : tablePartitions) {
|
||||
paths.add(tablePartition.getLocation().toUri().getPath());
|
||||
}
|
||||
List<StoragePartition> missing = Lists.newArrayList();
|
||||
for (StoragePartition storagePartition : storagePartitions) {
|
||||
String hdfsPath = storagePartition.getPartitionPath().toUri().getPath();
|
||||
if (!paths.contains(hdfsPath)) {
|
||||
missing.add(storagePartition);
|
||||
}
|
||||
}
|
||||
return missing;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of storage partitions which does not have its equivalent hive partitions
|
||||
*
|
||||
* @param tablePartitions
|
||||
* @param storagePartitions
|
||||
* @return
|
||||
*/
|
||||
public List<StoragePartition> getChangedStoragePartitions(
|
||||
List<TablePartition> tablePartitions, List<StoragePartition> storagePartitions) {
|
||||
Map<String, String> paths = Maps.newHashMap();
|
||||
for (TablePartition tablePartition : tablePartitions) {
|
||||
String[] partitionKeyValueStr = tablePartition.getPartitionFieldValues();
|
||||
Arrays.sort(partitionKeyValueStr);
|
||||
paths.put(Arrays.toString(partitionKeyValueStr), tablePartition.getLocation().toUri().getPath());
|
||||
}
|
||||
|
||||
List<StoragePartition> changed = Lists.newArrayList();
|
||||
for (StoragePartition storagePartition : storagePartitions) {
|
||||
String[] partitionKeyValues = storagePartition.getPartitionFieldValues();
|
||||
Arrays.sort(partitionKeyValues);
|
||||
String partitionKeyValueStr = Arrays.toString(partitionKeyValues);
|
||||
String hdfsPath = storagePartition.getPartitionPath().toUri().getPath();
|
||||
if (paths.containsKey(partitionKeyValueStr) && !paths.get(partitionKeyValueStr).equals(hdfsPath)) {
|
||||
changed.add(storagePartition);
|
||||
}
|
||||
}
|
||||
return changed;
|
||||
}
|
||||
|
||||
public int calculateStorageHash(FileStatus[] paths) {
|
||||
return Objects.hashCode(paths);
|
||||
}
|
||||
|
||||
public FileSystem getFs() {
|
||||
return fs;
|
||||
}
|
||||
}
|
||||
@@ -1,365 +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.client;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.uber.hoodie.hive.HoodieHiveConfiguration;
|
||||
import com.uber.hoodie.hive.HoodieHiveDatasetException;
|
||||
import com.uber.hoodie.hive.PartitionStrategy;
|
||||
import com.uber.hoodie.hive.model.HoodieDatasetReference;
|
||||
import com.uber.hoodie.hive.model.SchemaDifference;
|
||||
import com.uber.hoodie.hive.model.StoragePartition;
|
||||
import com.uber.hoodie.hive.model.TablePartition;
|
||||
import org.apache.commons.dbcp.BasicDataSource;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
|
||||
import org.apache.hadoop.hive.metastore.api.Partition;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import parquet.schema.MessageType;
|
||||
|
||||
import javax.sql.DataSource;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DatabaseMetaData;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Client to access Hive
|
||||
*/
|
||||
public class HoodieHiveClient implements Closeable {
|
||||
private static Logger LOG = LoggerFactory.getLogger(HoodieHiveClient.class);
|
||||
private static String driverName = "org.apache.hive.jdbc.HiveDriver";
|
||||
|
||||
static {
|
||||
try {
|
||||
Class.forName(driverName);
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new IllegalStateException("Could not find " + driverName + " in classpath. ", e);
|
||||
}
|
||||
}
|
||||
|
||||
private final HoodieHiveConfiguration configuration;
|
||||
private Connection connection;
|
||||
private HiveConf hiveConf;
|
||||
|
||||
public HoodieHiveClient(HoodieHiveConfiguration configuration) {
|
||||
this.configuration = configuration;
|
||||
this.hiveConf = new HiveConf();
|
||||
this.hiveConf.addResource(configuration.getConfiguration());
|
||||
try {
|
||||
this.connection = getConnection();
|
||||
} catch (SQLException e) {
|
||||
throw new HoodieHiveDatasetException("Failed to connect to hive metastore ", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Scan all the partitions for the given {@link HoodieDatasetReference} with the given {@link PartitionStrategy}
|
||||
*
|
||||
* @param metadata
|
||||
* @return
|
||||
*/
|
||||
public List<TablePartition> scanPartitions(HoodieDatasetReference metadata) {
|
||||
if (!checkTableExists(metadata)) {
|
||||
throw new IllegalArgumentException(
|
||||
"Failed to scan partitions as table " + metadata.getDatabaseTableName()
|
||||
+ " does not exist");
|
||||
}
|
||||
List<TablePartition> partitions = Lists.newArrayList();
|
||||
HiveMetaStoreClient client = null;
|
||||
try {
|
||||
client = new HiveMetaStoreClient(hiveConf);
|
||||
List<Partition> hivePartitions = client
|
||||
.listPartitions(metadata.getDatabaseName(), metadata.getTableName(), (short) -1);
|
||||
for (Partition partition : hivePartitions) {
|
||||
partitions.add(new TablePartition(metadata, partition));
|
||||
}
|
||||
return partitions;
|
||||
} catch (Exception e) {
|
||||
throw new HoodieHiveDatasetException("Failed to scan partitions for " + metadata, e);
|
||||
} finally {
|
||||
if (client != null) {
|
||||
client.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if table exists
|
||||
*
|
||||
* @param metadata
|
||||
* @return
|
||||
*/
|
||||
public boolean checkTableExists(HoodieDatasetReference metadata) {
|
||||
ResultSet resultSet = null;
|
||||
try {
|
||||
Connection conn = getConnection();
|
||||
resultSet = conn.getMetaData()
|
||||
.getTables(null, metadata.getDatabaseName(), metadata.getTableName(), null);
|
||||
return resultSet.next();
|
||||
} catch (SQLException e) {
|
||||
throw new HoodieHiveDatasetException("Failed to check if table exists " + metadata, e);
|
||||
} finally {
|
||||
closeQuietly(resultSet, null);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Update the hive metastore pointed to by {@link HoodieDatasetReference} with the difference
|
||||
* in schema {@link SchemaDifference}
|
||||
*
|
||||
* @param metadata
|
||||
* @param hivePartitionFieldNames
|
||||
* @param newSchema @return
|
||||
*/
|
||||
public boolean updateTableDefinition(HoodieDatasetReference metadata,
|
||||
String[] hivePartitionFieldNames, MessageType newSchema) {
|
||||
try {
|
||||
String newSchemaStr = SchemaUtil.generateSchemaString(newSchema);
|
||||
// Cascade clause should not be present for non-partitioned tables
|
||||
String cascadeClause = hivePartitionFieldNames.length > 0 ? " cascade" : "";
|
||||
StringBuilder sqlBuilder = new StringBuilder("ALTER TABLE ").append("`")
|
||||
.append(metadata.getDatabaseTableName()).append("`").append(" REPLACE COLUMNS(")
|
||||
.append(newSchemaStr).append(" )").append(cascadeClause);
|
||||
LOG.info("Creating table with " + sqlBuilder);
|
||||
return updateHiveSQL(sqlBuilder.toString());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieHiveDatasetException("Failed to update table for " + metadata, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Execute a update in hive metastore with this SQL
|
||||
*
|
||||
* @param s SQL to execute
|
||||
* @return
|
||||
*/
|
||||
public boolean updateHiveSQL(String s) {
|
||||
Statement stmt = null;
|
||||
try {
|
||||
Connection conn = getConnection();
|
||||
stmt = conn.createStatement();
|
||||
LOG.info("Executing SQL " + s);
|
||||
return stmt.execute(s);
|
||||
} catch (SQLException e) {
|
||||
throw new HoodieHiveDatasetException("Failed in executing SQL " + s, e);
|
||||
} finally {
|
||||
closeQuietly(null, stmt);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the table schema
|
||||
*
|
||||
* @param datasetReference
|
||||
* @return
|
||||
*/
|
||||
public Map<String, String> getTableSchema(HoodieDatasetReference datasetReference) {
|
||||
if (!checkTableExists(datasetReference)) {
|
||||
throw new IllegalArgumentException(
|
||||
"Failed to get schema as table " + datasetReference.getDatabaseTableName()
|
||||
+ " does not exist");
|
||||
}
|
||||
Map<String, String> schema = Maps.newHashMap();
|
||||
ResultSet result = null;
|
||||
try {
|
||||
Connection connection = getConnection();
|
||||
DatabaseMetaData databaseMetaData = connection.getMetaData();
|
||||
result = databaseMetaData.getColumns(null, datasetReference.getDatabaseName(),
|
||||
datasetReference.getTableName(), null);
|
||||
while (result.next()) {
|
||||
String columnName = result.getString(4);
|
||||
String columnType = result.getString(6);
|
||||
schema.put(columnName, columnType);
|
||||
}
|
||||
return schema;
|
||||
} catch (SQLException e) {
|
||||
throw new HoodieHiveDatasetException(
|
||||
"Failed to get table schema for " + datasetReference, e);
|
||||
} finally {
|
||||
closeQuietly(result, null);
|
||||
}
|
||||
}
|
||||
|
||||
public void addPartitionsToTable(HoodieDatasetReference datasetReference,
|
||||
List<StoragePartition> partitionsToAdd, PartitionStrategy strategy) {
|
||||
if (partitionsToAdd.isEmpty()) {
|
||||
LOG.info("No partitions to add for " + datasetReference);
|
||||
return;
|
||||
}
|
||||
LOG.info("Adding partitions " + partitionsToAdd.size() + " to dataset " + datasetReference);
|
||||
String sql = constructAddPartitions(datasetReference, partitionsToAdd, strategy);
|
||||
updateHiveSQL(sql);
|
||||
}
|
||||
|
||||
public void updatePartitionsToTable(HoodieDatasetReference datasetReference,
|
||||
List<StoragePartition> changedPartitions, PartitionStrategy partitionStrategy) {
|
||||
if (changedPartitions.isEmpty()) {
|
||||
LOG.info("No partitions to change for " + datasetReference);
|
||||
return;
|
||||
}
|
||||
LOG.info(
|
||||
"Changing partitions " + changedPartitions.size() + " on dataset " + datasetReference);
|
||||
List<String> sqls =
|
||||
constructChangePartitions(datasetReference, changedPartitions, partitionStrategy);
|
||||
for (String sql : sqls) {
|
||||
updateHiveSQL(sql);
|
||||
}
|
||||
}
|
||||
|
||||
public void createTable(MessageType storageSchema, HoodieDatasetReference metadata,
|
||||
String[] partitionKeys, String inputFormatClass, String outputFormatClass) {
|
||||
try {
|
||||
String createSQLQuery = SchemaUtil
|
||||
.generateCreateDDL(storageSchema, metadata, partitionKeys, inputFormatClass,
|
||||
outputFormatClass);
|
||||
LOG.info("Creating table with " + createSQLQuery);
|
||||
updateHiveSQL(createSQLQuery);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieHiveDatasetException("Failed to create table for " + metadata, e);
|
||||
}
|
||||
}
|
||||
|
||||
private static void closeQuietly(ResultSet resultSet, Statement stmt) {
|
||||
try {
|
||||
if (stmt != null)
|
||||
stmt.close();
|
||||
if (resultSet != null)
|
||||
resultSet.close();
|
||||
} catch (SQLException e) {
|
||||
LOG.error("Could not close the resultset opened ", e);
|
||||
}
|
||||
}
|
||||
|
||||
private Connection getConnection() throws SQLException {
|
||||
int count = 0;
|
||||
int maxTries = 3;
|
||||
if (connection == null) {
|
||||
Configuration conf = configuration.getConfiguration();
|
||||
DataSource ds = getDatasource();
|
||||
LOG.info("Getting Hive Connection from Datasource " + ds);
|
||||
while (true) {
|
||||
try {
|
||||
this.connection = ds.getConnection();
|
||||
break;
|
||||
} catch (SQLException e) {
|
||||
if (++count == maxTries)
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
return connection;
|
||||
}
|
||||
|
||||
private DataSource getDatasource() {
|
||||
BasicDataSource ds = new BasicDataSource();
|
||||
ds.setDriverClassName(driverName);
|
||||
ds.setUrl(getHiveJdbcUrlWithDefaultDBName());
|
||||
ds.setUsername(configuration.getHiveUsername());
|
||||
ds.setPassword(configuration.getHivePassword());
|
||||
return ds;
|
||||
}
|
||||
|
||||
public String getHiveJdbcUrlWithDefaultDBName() {
|
||||
String hiveJdbcUrl = configuration.getHiveJdbcUrl();
|
||||
String urlAppend = null;
|
||||
// If the hive url contains addition properties like ;transportMode=http;httpPath=hs2
|
||||
if (hiveJdbcUrl.contains(";")) {
|
||||
urlAppend = hiveJdbcUrl.substring(hiveJdbcUrl.indexOf(";"));
|
||||
hiveJdbcUrl = hiveJdbcUrl.substring(0, hiveJdbcUrl.indexOf(";"));
|
||||
}
|
||||
if (!hiveJdbcUrl.endsWith("/")) {
|
||||
hiveJdbcUrl = hiveJdbcUrl + "/";
|
||||
}
|
||||
return hiveJdbcUrl + configuration.getDbName() + (urlAppend == null ? "" : urlAppend);
|
||||
}
|
||||
|
||||
private static List<String> constructChangePartitions(HoodieDatasetReference metadata,
|
||||
List<StoragePartition> partitions, PartitionStrategy partitionStrategy) {
|
||||
String[] partitionFieldNames = partitionStrategy.getHivePartitionFieldNames();
|
||||
|
||||
List<String> changePartitions = Lists.newArrayList();
|
||||
String alterTable = "ALTER TABLE " + metadata.getDatabaseTableName();
|
||||
for (StoragePartition partition : partitions) {
|
||||
StringBuilder partBuilder = new StringBuilder();
|
||||
String[] partitionValues = partition.getPartitionFieldValues();
|
||||
Preconditions.checkArgument(partitionFieldNames.length == partitionValues.length,
|
||||
"Partition key parts " + Arrays.toString(partitionFieldNames)
|
||||
+ " does not match with partition values " + Arrays.toString(partitionValues)
|
||||
+ ". Check partition strategy. ");
|
||||
for (int i = 0; i < partitionFieldNames.length; i++) {
|
||||
partBuilder.append(partitionFieldNames[i]).append("=").append("'")
|
||||
.append(partitionValues[i]).append("'");
|
||||
}
|
||||
String changePartition =
|
||||
alterTable + " PARTITION (" + partBuilder.toString() + ") SET LOCATION '"
|
||||
+ "hdfs://nameservice1" + partition.getPartitionPath() + "'";
|
||||
changePartitions.add(changePartition);
|
||||
}
|
||||
return changePartitions;
|
||||
}
|
||||
|
||||
private static String constructAddPartitions(HoodieDatasetReference metadata,
|
||||
List<StoragePartition> partitions, PartitionStrategy partitionStrategy) {
|
||||
return constructAddPartitions(metadata.getDatabaseTableName(), partitions,
|
||||
partitionStrategy);
|
||||
}
|
||||
|
||||
private static String constructAddPartitions(String newDbTableName,
|
||||
List<StoragePartition> partitions, PartitionStrategy partitionStrategy) {
|
||||
String[] partitionFieldNames = partitionStrategy.getHivePartitionFieldNames();
|
||||
StringBuilder alterSQL = new StringBuilder("ALTER TABLE ");
|
||||
alterSQL.append(newDbTableName).append(" ADD IF NOT EXISTS ");
|
||||
for (StoragePartition partition : partitions) {
|
||||
StringBuilder partBuilder = new StringBuilder();
|
||||
String[] partitionValues = partition.getPartitionFieldValues();
|
||||
Preconditions.checkArgument(partitionFieldNames.length == partitionValues.length,
|
||||
"Partition key parts " + Arrays.toString(partitionFieldNames)
|
||||
+ " does not match with partition values " + Arrays.toString(partitionValues)
|
||||
+ ". Check partition strategy. ");
|
||||
for (int i = 0; i < partitionFieldNames.length; i++) {
|
||||
partBuilder.append(partitionFieldNames[i]).append("=").append("'")
|
||||
.append(partitionValues[i]).append("'");
|
||||
}
|
||||
alterSQL.append(" PARTITION (").append(partBuilder.toString()).append(") LOCATION '")
|
||||
.append(partition.getPartitionPath()).append("' ");
|
||||
}
|
||||
|
||||
return alterSQL.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (connection != null) {
|
||||
try {
|
||||
connection.close();
|
||||
} catch (SQLException e) {
|
||||
LOG.error("Could not close the connection opened ", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,39 +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.example;
|
||||
|
||||
import com.uber.hoodie.hive.HiveSyncTool;
|
||||
import com.uber.hoodie.hive.HiveSyncConfig;
|
||||
|
||||
/**
|
||||
* Example showing how to sync the dataset, written by `HoodieClientExample`
|
||||
*/
|
||||
public class HoodieHiveSyncExample {
|
||||
|
||||
public static void main(String[] args) {
|
||||
|
||||
HiveSyncConfig cfg = new HiveSyncConfig();
|
||||
cfg.databaseName = "default";
|
||||
cfg.tableName = "uber_trips";
|
||||
cfg.basePath = "/tmp/hoodie/sample-table/";
|
||||
cfg.hiveUser = "hive";
|
||||
cfg.hivePass = "hive";
|
||||
cfg.jdbcUrl = "jdbc:hive2://localhost:10010/";
|
||||
|
||||
HiveSyncTool.sync(cfg);
|
||||
}
|
||||
}
|
||||
@@ -1,76 +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.impl;
|
||||
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.hive.HoodieHiveDatasetException;
|
||||
import com.uber.hoodie.hive.PartitionStrategy;
|
||||
import com.uber.hoodie.hive.client.HoodieFSClient;
|
||||
import com.uber.hoodie.hive.model.HoodieDatasetReference;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.format.DateTimeFormat;
|
||||
import org.joda.time.format.DateTimeFormatter;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Simple day based partitions.
|
||||
* Storage is of this format yyyy/mm/dd
|
||||
* Table is partitioned by dateStringFieldName=MM/dd/yyyy
|
||||
*/
|
||||
public class DayBasedPartitionStrategy implements PartitionStrategy {
|
||||
private Logger LOG = LoggerFactory.getLogger(DayBasedPartitionStrategy.class);
|
||||
private final String dateStringFieldName;
|
||||
private final DateTimeFormatter dtfOut;
|
||||
|
||||
public DayBasedPartitionStrategy() {
|
||||
this.dateStringFieldName = "datestr";
|
||||
this.dtfOut = DateTimeFormat.forPattern("yyyy-MM-dd");
|
||||
}
|
||||
|
||||
@Override public List<String> scanAllPartitions(HoodieDatasetReference ref, HoodieFSClient fsClient) {
|
||||
try {
|
||||
return FSUtils.getAllPartitionPaths(fsClient.getFs(), ref.getBaseDatasetPath(), true);
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieHiveDatasetException(
|
||||
"IOException when listing partitions under dataset " + ref , ioe);
|
||||
}
|
||||
}
|
||||
|
||||
@Override public String[] getHivePartitionFieldNames() {
|
||||
return new String[] {dateStringFieldName};
|
||||
}
|
||||
|
||||
@Override
|
||||
public String[] convertPartitionToValues(HoodieDatasetReference metadata, String partitionPath) {
|
||||
//yyyy/mm/dd
|
||||
String[] splits = partitionPath.split("/");
|
||||
if (splits.length != 3) {
|
||||
throw new IllegalArgumentException(
|
||||
"Partition path " + partitionPath + " is not in the form yyyy/mm/dd ");
|
||||
}
|
||||
// Get the partition part and remove the / as well at the end
|
||||
int year = Integer.parseInt(splits[0]);
|
||||
int mm = Integer.parseInt(splits[1]);
|
||||
int dd = Integer.parseInt(splits[2]);
|
||||
DateTime dateTime = new DateTime(year, mm, dd, 0, 0);
|
||||
return new String[] {dtfOut.print(dateTime)};
|
||||
}
|
||||
}
|
||||
@@ -1,43 +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.impl;
|
||||
|
||||
import com.uber.hoodie.hive.HoodieHiveDatasetException;
|
||||
import com.uber.hoodie.hive.SchemaStrategy;
|
||||
import com.uber.hoodie.hive.client.HoodieFSClient;
|
||||
import com.uber.hoodie.hive.model.HoodieDatasetReference;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import parquet.schema.MessageType;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Schema strategy to read the parquet schema from any of the data file
|
||||
*/
|
||||
public class ParseSchemaFromDataStrategy implements SchemaStrategy {
|
||||
@Override
|
||||
public MessageType getDatasetSchema(HoodieDatasetReference metadata, HoodieFSClient fsClient) {
|
||||
Path anyDataFile = fsClient.lastDataFileForDataset(metadata, metadata.getBaseDatasetPath());
|
||||
try {
|
||||
return fsClient.readSchemaFromDataFile(anyDataFile);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieHiveDatasetException(
|
||||
"Could not read schema for " + metadata + ", tried to read schema from "
|
||||
+ anyDataFile, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,79 +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.model;
|
||||
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* A reference to a Dataset. Each dataset will have a hadoop configuration, table name,
|
||||
* base path in HDFS. {@link HoodieDatasetReference} is immutable.
|
||||
*/
|
||||
public class HoodieDatasetReference {
|
||||
private String tableName;
|
||||
private String baseDatasetPath;
|
||||
private String databaseName;
|
||||
|
||||
public HoodieDatasetReference(String tableName, String baseDatasetPath, String databaseName) {
|
||||
this.tableName = tableName;
|
||||
this.baseDatasetPath = baseDatasetPath;
|
||||
this.databaseName = databaseName;
|
||||
}
|
||||
|
||||
public String getDatabaseTableName() {
|
||||
return databaseName + "." + tableName;
|
||||
}
|
||||
|
||||
public String getTableName() {
|
||||
return tableName;
|
||||
}
|
||||
|
||||
public String getBaseDatasetPath() {
|
||||
return baseDatasetPath;
|
||||
}
|
||||
|
||||
public String getDatabaseName() {
|
||||
return databaseName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o)
|
||||
return true;
|
||||
if (o == null || getClass() != o.getClass())
|
||||
return false;
|
||||
HoodieDatasetReference that = (HoodieDatasetReference) o;
|
||||
return Objects.equals(tableName, that.tableName) &&
|
||||
Objects.equals(baseDatasetPath, that.baseDatasetPath) &&
|
||||
Objects.equals(databaseName, that.databaseName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(tableName, baseDatasetPath, databaseName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieDatasetReference{");
|
||||
sb.append("tableName='").append(tableName).append('\'');
|
||||
sb.append(", baseDatasetPath='").append(baseDatasetPath).append('\'');
|
||||
sb.append(", databaseName='").append(databaseName).append('\'');
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
@@ -1,51 +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.model;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import com.uber.hoodie.hive.PartitionStrategy;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
public class StoragePartition {
|
||||
private static Logger LOG = LoggerFactory.getLogger(StoragePartition.class);
|
||||
private final PartitionStrategy partitionStrategy;
|
||||
private final String partitionPath;
|
||||
private final HoodieDatasetReference metadata;
|
||||
|
||||
public StoragePartition(HoodieDatasetReference metadata, PartitionStrategy partitionStrategy, String partitionPath) {
|
||||
this.metadata = metadata;
|
||||
this.partitionPath = partitionPath;
|
||||
this.partitionStrategy = partitionStrategy;
|
||||
}
|
||||
|
||||
public String[] getPartitionFieldValues() {
|
||||
return partitionStrategy.convertPartitionToValues(metadata, partitionPath);
|
||||
}
|
||||
|
||||
public Path getPartitionPath() {
|
||||
return new Path(metadata.getBaseDatasetPath(), partitionPath);
|
||||
//return Path.getPathWithoutSchemeAndAuthority(new Path(metadata.getBaseDatasetPath(), partitionPath));
|
||||
}
|
||||
|
||||
@Override public String toString() {
|
||||
return Objects.toStringHelper(this).add("partitionPath", partitionPath)
|
||||
.add("metadata", metadata).toString();
|
||||
}
|
||||
}
|
||||
@@ -1,38 +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.model;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.metastore.api.Partition;
|
||||
|
||||
public class TablePartition {
|
||||
private final HoodieDatasetReference metadata;
|
||||
private final Partition partition;
|
||||
|
||||
public TablePartition(HoodieDatasetReference metadata, Partition partition) {
|
||||
this.metadata = metadata;
|
||||
this.partition = partition;
|
||||
}
|
||||
|
||||
public Path getLocation() {
|
||||
return Path.getPathWithoutSchemeAndAuthority(new Path(partition.getSd().getLocation()));
|
||||
}
|
||||
|
||||
public String[] getPartitionFieldValues() {
|
||||
return partition.getValues().toArray(new String[partition.getValuesSize()]);
|
||||
}
|
||||
}
|
||||
@@ -14,7 +14,7 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.hive.client;
|
||||
package com.uber.hoodie.hive.util;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
|
||||
@@ -14,15 +14,13 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.hive.client;
|
||||
package com.uber.hoodie.hive.util;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.hive.HoodieHiveDatasetException;
|
||||
import com.uber.hoodie.hive.model.HoodieDatasetReference;
|
||||
import com.uber.hoodie.hive.model.SchemaDifference;
|
||||
import org.apache.commons.lang.ArrayUtils;
|
||||
import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
|
||||
import com.uber.hoodie.hive.HiveSyncConfig;
|
||||
import com.uber.hoodie.hive.HoodieHiveSyncException;
|
||||
import com.uber.hoodie.hive.SchemaDifference;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import parquet.schema.DecimalMetadata;
|
||||
@@ -52,12 +50,12 @@ public class SchemaUtil {
|
||||
* @return
|
||||
*/
|
||||
public static SchemaDifference getSchemaDifference(MessageType storageSchema,
|
||||
Map<String, String> tableSchema, String[] partitionKeys) {
|
||||
Map<String, String> tableSchema, List<String> partitionKeys) {
|
||||
Map<String, String> newTableSchema;
|
||||
try {
|
||||
newTableSchema = convertParquetSchemaToHiveSchema(storageSchema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieHiveDatasetException("Failed to convert parquet schema to hive schema",
|
||||
throw new HoodieHiveSyncException("Failed to convert parquet schema to hive schema",
|
||||
e);
|
||||
}
|
||||
LOG.info("Getting schema difference for " + tableSchema + "\r\n\r\n" + newTableSchema);
|
||||
@@ -68,14 +66,13 @@ public class SchemaUtil {
|
||||
for (Map.Entry<String, String> field : tableSchema.entrySet()) {
|
||||
String fieldName = field.getKey().toLowerCase();
|
||||
String tickSurroundedFieldName = tickSurround(fieldName);
|
||||
if (!isFieldExistsInSchema(newTableSchema, tickSurroundedFieldName) && !ArrayUtils
|
||||
.contains(partitionKeys, fieldName)) {
|
||||
if (!isFieldExistsInSchema(newTableSchema, tickSurroundedFieldName) && !partitionKeys.contains(fieldName)) {
|
||||
schemaDiffBuilder.deleteTableColumn(fieldName);
|
||||
} else {
|
||||
// check type
|
||||
String tableColumnType = field.getValue();
|
||||
if (!isFieldExistsInSchema(newTableSchema, tickSurroundedFieldName)) {
|
||||
if (ArrayUtils.contains(partitionKeys, fieldName)) {
|
||||
if (partitionKeys.contains(fieldName)) {
|
||||
// Partition key does not have to be part of the storage schema
|
||||
continue;
|
||||
}
|
||||
@@ -93,7 +90,7 @@ public class SchemaUtil {
|
||||
if (!tableColumnType.equalsIgnoreCase(expectedType)) {
|
||||
// check for incremental datasets, the schema type change is allowed as per evolution rules
|
||||
if (!isSchemaTypeUpdateAllowed(tableColumnType, expectedType)) {
|
||||
throw new HoodieHiveDatasetException(
|
||||
throw new HoodieHiveSyncException(
|
||||
"Could not convert field Type from " + tableColumnType + " to "
|
||||
+ expectedType + " for field " + fieldName);
|
||||
}
|
||||
@@ -401,27 +398,27 @@ public class SchemaUtil {
|
||||
}
|
||||
|
||||
public static String generateCreateDDL(MessageType storageSchema,
|
||||
HoodieDatasetReference metadata, String[] partitionKeys, String inputFormatClass,
|
||||
String outputFormatClass) throws IOException {
|
||||
HiveSyncConfig config, String inputFormatClass,
|
||||
String outputFormatClass, String serdeClass) throws IOException {
|
||||
Map<String, String> hiveSchema = convertParquetSchemaToHiveSchema(storageSchema);
|
||||
String columns = generateSchemaString(storageSchema);
|
||||
|
||||
StringBuilder partitionFields = new StringBuilder();
|
||||
for (String partitionKey : partitionKeys) {
|
||||
for (String partitionKey : config.partitionFields) {
|
||||
partitionFields.append(partitionKey).append(" ")
|
||||
.append(getPartitionKeyType(hiveSchema, partitionKey));
|
||||
}
|
||||
|
||||
StringBuilder sb = new StringBuilder("CREATE EXTERNAL TABLE IF NOT EXISTS ");
|
||||
sb = sb.append(metadata.getDatabaseTableName());
|
||||
sb = sb.append(config.databaseName).append(".").append(config.tableName);
|
||||
sb = sb.append("( ").append(columns).append(")");
|
||||
if (partitionKeys.length > 0) {
|
||||
if (!config.partitionFields.isEmpty()) {
|
||||
sb = sb.append(" PARTITIONED BY (").append(partitionFields).append(")");
|
||||
}
|
||||
sb = sb.append(" ROW FORMAT SERDE '").append(ParquetHiveSerDe.class.getName()).append("'");
|
||||
sb = sb.append(" ROW FORMAT SERDE '").append(serdeClass).append("'");
|
||||
sb = sb.append(" STORED AS INPUTFORMAT '").append(inputFormatClass).append("'");
|
||||
sb = sb.append(" OUTPUTFORMAT '").append(outputFormatClass).append("' LOCATION '")
|
||||
.append(metadata.getBaseDatasetPath()).append("'");
|
||||
.append(config.basePath).append("'");
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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"));
|
||||
}
|
||||
|
||||
}
|
||||
@@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
353
hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java
Normal file
353
hoodie-hive/src/test/java/com/uber/hoodie/hive/TestUtil.java
Normal 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();
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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());
|
||||
}
|
||||
}
|
||||
@@ -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);
|
||||
}
|
||||
}
|
||||
@@ -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,6 +0,0 @@
|
||||
message m {
|
||||
required int32 nation_key;
|
||||
required binary name;
|
||||
required int32 region_key;
|
||||
required binary comment_col;
|
||||
}
|
||||
@@ -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,7 +0,0 @@
|
||||
message m {
|
||||
required int32 nation_key;
|
||||
required binary name;
|
||||
required int64 region_key;
|
||||
required binary comment_col;
|
||||
optional binary desc;
|
||||
}
|
||||
5
pom.xml
5
pom.xml
@@ -410,6 +410,11 @@
|
||||
<artifactId>parquet-hive-bundle</artifactId>
|
||||
<version>1.5.0</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.twitter</groupId>
|
||||
<artifactId>parquet-avro</artifactId>
|
||||
<version>1.5.0-cdh5.7.2</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.parquet</groupId>
|
||||
|
||||
Reference in New Issue
Block a user