Add hoodie-hive module to support hive registration of hoodie datasets
This commit is contained in:
@@ -0,0 +1,119 @@
|
||||
/*
|
||||
* 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;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,40 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
public class HoodieHiveDatasetException extends RuntimeException {
|
||||
|
||||
public HoodieHiveDatasetException() {
|
||||
super();
|
||||
}
|
||||
|
||||
public HoodieHiveDatasetException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public HoodieHiveDatasetException(String message, Throwable t) {
|
||||
super(message, t);
|
||||
}
|
||||
|
||||
public HoodieHiveDatasetException(Throwable t) {
|
||||
super(t);
|
||||
}
|
||||
|
||||
protected static String format(String message, Object... args) {
|
||||
return String.format(String.valueOf(message), (Object[]) args);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,181 @@
|
||||
/*
|
||||
* 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.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();
|
||||
FileStatus[] storagePartitionPaths = schemaSyncTask.getPartitionStrategy()
|
||||
.scanAllPartitions(schemaSyncTask.getReference(), schemaSyncTask.getFsClient());
|
||||
for (FileStatus fileStatus : storagePartitionPaths) {
|
||||
storagePartitions.add(new StoragePartition(schemaSyncTask.getReference(),
|
||||
schemaSyncTask.getPartitionStrategy(), fileStatus));
|
||||
}
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,243 @@
|
||||
/*
|
||||
* 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* 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 FileStatus[] which are the available partitions
|
||||
*
|
||||
* @param basePath
|
||||
* @param fsClient
|
||||
* @return
|
||||
*/
|
||||
FileStatus[] 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. /data/topic/2016/12/12/ will return [2016, 12, 12]
|
||||
*
|
||||
* @param partition storage path
|
||||
* @return List of partitions field values
|
||||
*/
|
||||
String[] convertPartitionToValues(HoodieDatasetReference metadata, Path partition);
|
||||
}
|
||||
@@ -0,0 +1,31 @@
|
||||
/*
|
||||
* 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,48 @@
|
||||
/*
|
||||
* 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.collect.Maps;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
||||
public class ColumnNameXLator {
|
||||
private static Map<String, String> xformMap = Maps.newHashMap();
|
||||
|
||||
public static String translateNestedColumn(String colName) {
|
||||
Map.Entry entry;
|
||||
for (Iterator i$ = xformMap.entrySet().iterator(); i$.hasNext();
|
||||
colName = colName.replaceAll((String) entry.getKey(), (String) entry.getValue())) {
|
||||
entry = (Map.Entry) i$.next();
|
||||
}
|
||||
|
||||
return colName;
|
||||
}
|
||||
|
||||
public static String translateColumn(String colName) {
|
||||
return colName;
|
||||
}
|
||||
|
||||
public static String translate(String colName, boolean nestedColumn) {
|
||||
return !nestedColumn ? translateColumn(colName) : translateNestedColumn(colName);
|
||||
}
|
||||
|
||||
static {
|
||||
xformMap.put("\\$", "_dollar_");
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,206 @@
|
||||
/*
|
||||
* 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)) {
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds all the files/directories that match the pattern under the {@link HoodieDatasetReference} basePath
|
||||
*
|
||||
* @param metadata
|
||||
* @param pattern
|
||||
* @return
|
||||
*/
|
||||
public FileStatus[] getDirectoriesMatchingPattern(HoodieDatasetReference metadata, String pattern) {
|
||||
try {
|
||||
Path path = new Path(metadata.getBaseDatasetPath() + pattern);
|
||||
FileStatus[] status = fs.globStatus(path);
|
||||
List<FileStatus> returns = Lists.newArrayList();
|
||||
for(FileStatus st:status) {
|
||||
if(!st.getPath().toString().contains(".distcp")) {
|
||||
// Ignore temporary directories created by distcp
|
||||
returns.add(st);
|
||||
}
|
||||
}
|
||||
return returns.toArray(new FileStatus[returns.size()]);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieHiveDatasetException(
|
||||
"IOException when reading directories under dataset " + metadata + " with pattern "
|
||||
+ pattern, 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);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,365 @@
|
||||
/*
|
||||
* 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,436 @@
|
||||
/*
|
||||
* 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.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 org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import parquet.schema.DecimalMetadata;
|
||||
import parquet.schema.GroupType;
|
||||
import parquet.schema.MessageType;
|
||||
import parquet.schema.OriginalType;
|
||||
import parquet.schema.PrimitiveType;
|
||||
import parquet.schema.Type;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Schema Utilities
|
||||
*/
|
||||
public class SchemaUtil {
|
||||
private static Logger LOG = LoggerFactory.getLogger(SchemaUtil.class);
|
||||
|
||||
/**
|
||||
* Get the schema difference between the storage schema and hive table schema
|
||||
*
|
||||
* @param storageSchema
|
||||
* @param tableSchema
|
||||
* @param partitionKeys
|
||||
* @return
|
||||
*/
|
||||
public static SchemaDifference getSchemaDifference(MessageType storageSchema,
|
||||
Map<String, String> tableSchema, String[] partitionKeys) {
|
||||
Map<String, String> newTableSchema;
|
||||
try {
|
||||
newTableSchema = convertParquetSchemaToHiveSchema(storageSchema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieHiveDatasetException("Failed to convert parquet schema to hive schema",
|
||||
e);
|
||||
}
|
||||
LOG.info("Getting schema difference for " + tableSchema + "\r\n\r\n" + newTableSchema);
|
||||
SchemaDifference.Builder schemaDiffBuilder =
|
||||
SchemaDifference.newBuilder(storageSchema, tableSchema);
|
||||
Set<String> tableColumns = Sets.newHashSet();
|
||||
|
||||
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)) {
|
||||
schemaDiffBuilder.deleteTableColumn(fieldName);
|
||||
} else {
|
||||
// check type
|
||||
String tableColumnType = field.getValue();
|
||||
if (!isFieldExistsInSchema(newTableSchema, tickSurroundedFieldName)) {
|
||||
if (ArrayUtils.contains(partitionKeys, fieldName)) {
|
||||
// Partition key does not have to be part of the storage schema
|
||||
continue;
|
||||
}
|
||||
// We will log this and continue. Hive schema is a superset of all parquet schemas
|
||||
LOG.warn("Ignoring table column " + fieldName
|
||||
+ " as its not present in the parquet schema");
|
||||
continue;
|
||||
}
|
||||
tableColumnType = tableColumnType.replaceAll("\\s+", "");
|
||||
|
||||
String expectedType = getExpectedType(newTableSchema, tickSurroundedFieldName);
|
||||
expectedType = expectedType.replaceAll("\\s+", "");
|
||||
expectedType = expectedType.replaceAll("`", "");
|
||||
|
||||
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(
|
||||
"Could not convert field Type from " + tableColumnType + " to "
|
||||
+ expectedType + " for field " + fieldName);
|
||||
}
|
||||
schemaDiffBuilder.updateTableColumn(fieldName,
|
||||
getExpectedType(newTableSchema, tickSurroundedFieldName));
|
||||
}
|
||||
}
|
||||
tableColumns.add(tickSurroundedFieldName);
|
||||
}
|
||||
|
||||
for (Map.Entry<String, String> entry : newTableSchema.entrySet()) {
|
||||
if (!tableColumns.contains(entry.getKey().toLowerCase())) {
|
||||
schemaDiffBuilder.addTableColumn(entry.getKey(), entry.getValue());
|
||||
}
|
||||
}
|
||||
LOG.info("Difference between schemas: " + schemaDiffBuilder.build().toString());
|
||||
|
||||
return schemaDiffBuilder.build();
|
||||
}
|
||||
|
||||
private static String getExpectedType(Map<String, String> newTableSchema, String fieldName) {
|
||||
for (Map.Entry<String, String> entry : newTableSchema.entrySet()) {
|
||||
if (entry.getKey().toLowerCase().equals(fieldName)) {
|
||||
return entry.getValue();
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private static boolean isFieldExistsInSchema(Map<String, String> newTableSchema,
|
||||
String fieldName) {
|
||||
for (String entry : newTableSchema.keySet()) {
|
||||
if (entry.toLowerCase().equals(fieldName)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns equivalent Hive table schema read from a parquet file
|
||||
*
|
||||
* @param messageType : Parquet Schema
|
||||
* @return : Hive Table schema read from parquet file MAP<String,String>
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Map<String, String> convertParquetSchemaToHiveSchema(MessageType messageType)
|
||||
throws IOException {
|
||||
Map<String, String> schema = Maps.newLinkedHashMap();
|
||||
List<Type> parquetFields = messageType.getFields();
|
||||
for (Type parquetType : parquetFields) {
|
||||
StringBuilder result = new StringBuilder();
|
||||
String key = parquetType.getName();
|
||||
if (parquetType.isRepetition(Type.Repetition.REPEATED)) {
|
||||
result.append(createHiveArray(parquetType, ""));
|
||||
} else {
|
||||
result.append(convertField(parquetType));
|
||||
}
|
||||
|
||||
schema.put(hiveCompatibleFieldName(key, false), result.toString());
|
||||
}
|
||||
return schema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert one field data type of parquet schema into an equivalent Hive
|
||||
* schema
|
||||
*
|
||||
* @param parquetType : Single paruet field
|
||||
* @return : Equivalent sHive schema
|
||||
*/
|
||||
private static String convertField(final Type parquetType) {
|
||||
StringBuilder field = new StringBuilder();
|
||||
if (parquetType.isPrimitive()) {
|
||||
final PrimitiveType.PrimitiveTypeName parquetPrimitiveTypeName =
|
||||
parquetType.asPrimitiveType().getPrimitiveTypeName();
|
||||
final OriginalType originalType = parquetType.getOriginalType();
|
||||
if (originalType == OriginalType.DECIMAL) {
|
||||
final DecimalMetadata decimalMetadata =
|
||||
parquetType.asPrimitiveType().getDecimalMetadata();
|
||||
return field.append("DECIMAL(").append(decimalMetadata.getPrecision()).
|
||||
append(" , ").append(decimalMetadata.getScale()).append(")").toString();
|
||||
}
|
||||
// TODO - fix the method naming here
|
||||
return parquetPrimitiveTypeName
|
||||
.convert(new PrimitiveType.PrimitiveTypeNameConverter<String, RuntimeException>() {
|
||||
@Override
|
||||
public String convertBOOLEAN(
|
||||
PrimitiveType.PrimitiveTypeName primitiveTypeName) {
|
||||
return "boolean";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String convertINT32(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
|
||||
return "int";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String convertINT64(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
|
||||
return "bigint";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String convertINT96(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
|
||||
return "timestamp-millis";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String convertFLOAT(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
|
||||
return "float";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String convertDOUBLE(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
|
||||
return "double";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String convertFIXED_LEN_BYTE_ARRAY(
|
||||
PrimitiveType.PrimitiveTypeName primitiveTypeName) {
|
||||
return "binary";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String convertBINARY(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
|
||||
if (originalType == OriginalType.UTF8
|
||||
|| originalType == OriginalType.ENUM) {
|
||||
return "string";
|
||||
} else {
|
||||
return "binary";
|
||||
}
|
||||
}
|
||||
});
|
||||
} else {
|
||||
GroupType parquetGroupType = parquetType.asGroupType();
|
||||
OriginalType originalType = parquetGroupType.getOriginalType();
|
||||
if (originalType != null) {
|
||||
switch (originalType) {
|
||||
case LIST:
|
||||
if (parquetGroupType.getFieldCount() != 1) {
|
||||
throw new UnsupportedOperationException(
|
||||
"Invalid list type " + parquetGroupType);
|
||||
}
|
||||
Type elementType = parquetGroupType.getType(0);
|
||||
if (!elementType.isRepetition(Type.Repetition.REPEATED)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"Invalid list type " + parquetGroupType);
|
||||
}
|
||||
return createHiveArray(elementType, parquetGroupType.getName());
|
||||
case MAP:
|
||||
if (parquetGroupType.getFieldCount() != 1 || parquetGroupType.getType(0)
|
||||
.isPrimitive()) {
|
||||
throw new UnsupportedOperationException(
|
||||
"Invalid map type " + parquetGroupType);
|
||||
}
|
||||
GroupType mapKeyValType = parquetGroupType.getType(0).asGroupType();
|
||||
if (!mapKeyValType.isRepetition(Type.Repetition.REPEATED) ||
|
||||
!mapKeyValType.getOriginalType().equals(OriginalType.MAP_KEY_VALUE) ||
|
||||
mapKeyValType.getFieldCount() != 2) {
|
||||
throw new UnsupportedOperationException(
|
||||
"Invalid map type " + parquetGroupType);
|
||||
}
|
||||
Type keyType = mapKeyValType.getType(0);
|
||||
if (!keyType.isPrimitive() ||
|
||||
!keyType.asPrimitiveType().getPrimitiveTypeName()
|
||||
.equals(PrimitiveType.PrimitiveTypeName.BINARY) ||
|
||||
!keyType.getOriginalType().equals(OriginalType.UTF8)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"Map key type must be binary (UTF8): " + keyType);
|
||||
}
|
||||
Type valueType = mapKeyValType.getType(1);
|
||||
return createHiveMap(convertField(keyType), convertField(valueType));
|
||||
case ENUM:
|
||||
case UTF8:
|
||||
return "string";
|
||||
case MAP_KEY_VALUE:
|
||||
// MAP_KEY_VALUE was supposed to be used to annotate key and
|
||||
// value group levels in a
|
||||
// MAP. However, that is always implied by the structure of
|
||||
// MAP. Hence, PARQUET-113
|
||||
// dropped the requirement for having MAP_KEY_VALUE.
|
||||
default:
|
||||
throw new UnsupportedOperationException(
|
||||
"Cannot convert Parquet type " + parquetType);
|
||||
}
|
||||
} else {
|
||||
// if no original type then it's a record
|
||||
return createHiveStruct(parquetGroupType.getFields());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return a 'struct' Hive schema from a list of Parquet fields
|
||||
*
|
||||
* @param parquetFields : list of parquet fields
|
||||
* @return : Equivalent 'struct' Hive schema
|
||||
*/
|
||||
private static String createHiveStruct(List<Type> parquetFields) {
|
||||
StringBuilder struct = new StringBuilder();
|
||||
struct.append("STRUCT< ");
|
||||
for (Type field : parquetFields) {
|
||||
//TODO: struct field name is only translated to support special char($)
|
||||
//We will need to extend it to other collection type
|
||||
struct.append(hiveCompatibleFieldName(field.getName(), true)).append(" : ");
|
||||
struct.append(convertField(field)).append(", ");
|
||||
}
|
||||
struct.delete(struct.length() - 2, struct.length()); // Remove the last
|
||||
// ", "
|
||||
struct.append(">");
|
||||
String finalStr = struct.toString();
|
||||
// Struct cannot have - in them. userstore_udr_entities has uuid in struct. This breaks the schema.
|
||||
// HDrone sync should not fail because of this.
|
||||
finalStr = finalStr.replaceAll("-", "_");
|
||||
return finalStr;
|
||||
}
|
||||
|
||||
|
||||
private static String hiveCompatibleFieldName(String fieldName, boolean isNested) {
|
||||
String result = fieldName;
|
||||
if (isNested) {
|
||||
result = ColumnNameXLator.translateNestedColumn(fieldName);
|
||||
}
|
||||
return tickSurround(result);
|
||||
}
|
||||
|
||||
private static String tickSurround(String result) {
|
||||
if (!result.startsWith("`")) {
|
||||
result = "`" + result;
|
||||
}
|
||||
if (!result.endsWith("`")) {
|
||||
result = result + "`";
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a 'Map' schema from Parquet map field
|
||||
*
|
||||
* @param keyType
|
||||
* @param valueType
|
||||
* @return
|
||||
*/
|
||||
private static String createHiveMap(String keyType, String valueType) {
|
||||
return "MAP< " + keyType + ", " + valueType + ">";
|
||||
}
|
||||
|
||||
/**
|
||||
* Create an Array Hive schema from equivalent parquet list type
|
||||
*
|
||||
* @param elementType
|
||||
* @param elementName
|
||||
* @return
|
||||
*/
|
||||
private static String createHiveArray(Type elementType, String elementName) {
|
||||
StringBuilder array = new StringBuilder();
|
||||
array.append("ARRAY< ");
|
||||
if (elementType.isPrimitive()) {
|
||||
array.append(convertField(elementType));
|
||||
} else {
|
||||
final GroupType groupType = elementType.asGroupType();
|
||||
final List<Type> groupFields = groupType.getFields();
|
||||
if (groupFields.size() > 1 || (groupFields.size() == 1 && (
|
||||
elementType.getName().equals("array") || elementType.getName()
|
||||
.equals(elementName + "_tuple")))) {
|
||||
array.append(convertField(elementType));
|
||||
} else {
|
||||
array.append(convertField(groupType.getFields().get(0)));
|
||||
}
|
||||
}
|
||||
array.append(">");
|
||||
return array.toString();
|
||||
}
|
||||
|
||||
public static boolean isSchemaTypeUpdateAllowed(String prevType, String newType) {
|
||||
if (prevType == null || prevType.trim().isEmpty() ||
|
||||
newType == null || newType.trim().isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
prevType = prevType.toLowerCase();
|
||||
newType = newType.toLowerCase();
|
||||
if (prevType.equals(newType)) {
|
||||
return true;
|
||||
} else if (prevType.equalsIgnoreCase("int") && newType.equalsIgnoreCase("bigint")) {
|
||||
return true;
|
||||
} else if (prevType.equalsIgnoreCase("float") && newType.equalsIgnoreCase("double")) {
|
||||
return true;
|
||||
} else if (prevType.contains("struct") && newType.toLowerCase().contains("struct")) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
public static String generateSchemaString(MessageType storageSchema) throws IOException {
|
||||
Map<String, String> hiveSchema = convertParquetSchemaToHiveSchema(storageSchema);
|
||||
StringBuilder columns = new StringBuilder();
|
||||
for (Map.Entry<String, String> hiveSchemaEntry : hiveSchema.entrySet()) {
|
||||
columns.append(hiveSchemaEntry.getKey()).append(" ");
|
||||
columns.append(hiveSchemaEntry.getValue()).append(", ");
|
||||
}
|
||||
// Remove the last ", "
|
||||
columns.delete(columns.length() - 2, columns.length());
|
||||
return columns.toString();
|
||||
}
|
||||
|
||||
public static String generateCreateDDL(MessageType storageSchema,
|
||||
HoodieDatasetReference metadata, String[] partitionKeys, String inputFormatClass,
|
||||
String outputFormatClass) throws IOException {
|
||||
Map<String, String> hiveSchema = convertParquetSchemaToHiveSchema(storageSchema);
|
||||
String columns = generateSchemaString(storageSchema);
|
||||
|
||||
StringBuilder partitionFields = new StringBuilder();
|
||||
for (String partitionKey : partitionKeys) {
|
||||
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("( ").append(columns).append(")");
|
||||
if (partitionKeys.length > 0) {
|
||||
sb = sb.append(" PARTITIONED BY (").append(partitionFields).append(")");
|
||||
}
|
||||
sb = sb.append(" ROW FORMAT SERDE '").append(ParquetHiveSerDe.class.getName()).append("'");
|
||||
sb = sb.append(" STORED AS INPUTFORMAT '").append(inputFormatClass).append("'");
|
||||
sb = sb.append(" OUTPUTFORMAT '").append(outputFormatClass).append("' LOCATION '")
|
||||
.append(metadata.getBaseDatasetPath()).append("'");
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
private static String getPartitionKeyType(Map<String, String> hiveSchema, String partitionKey) {
|
||||
if (hiveSchema.containsKey(partitionKey)) {
|
||||
return hiveSchema.get(partitionKey);
|
||||
}
|
||||
// Default the unknown partition fields to be String
|
||||
// TODO - all partition fields should be part of the schema. datestr is treated as special. Dont do that
|
||||
return "String";
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* 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.HoodieHiveConfiguration;
|
||||
import com.uber.hoodie.hive.HoodieHiveDatasetSyncTask;
|
||||
import com.uber.hoodie.hive.PartitionStrategy;
|
||||
import com.uber.hoodie.hive.SchemaStrategy;
|
||||
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;
|
||||
|
||||
/**
|
||||
* Example showing basic usage of Hoodie Hive API
|
||||
*/
|
||||
public class HoodieDatasetExample {
|
||||
public static void main(String[] args) {
|
||||
// Configure to point to which metastore and database to connect to
|
||||
HoodieHiveConfiguration apiConfig =
|
||||
HoodieHiveConfiguration.newBuilder().hadoopConfiguration(new Configuration())
|
||||
.hivedb("tmp").hiveJdbcUrl("jdbc:hive2://localhost:10010/").jdbcUsername("hive")
|
||||
.jdbcPassword("hive").build();
|
||||
|
||||
HoodieDatasetReference datasetReference =
|
||||
new HoodieDatasetReference("clickstream", "hdfs:///data/tables/user.clickstream",
|
||||
"raw");
|
||||
|
||||
// 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();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,78 @@
|
||||
/*
|
||||
* 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.PartitionStrategy;
|
||||
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 org.joda.time.DateTime;
|
||||
import org.joda.time.format.DateTimeFormat;
|
||||
import org.joda.time.format.DateTimeFormatter;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* 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 FileStatus[] scanAllPartitions(HoodieDatasetReference ref, HoodieFSClient fsClient) {
|
||||
return fsClient.getDirectoriesMatchingPattern(ref, "/*/*/*");
|
||||
}
|
||||
|
||||
@Override public String[] getHivePartitionFieldNames() {
|
||||
return new String[] {dateStringFieldName};
|
||||
}
|
||||
|
||||
@Override
|
||||
public String[] convertPartitionToValues(HoodieDatasetReference metadata, Path partition) {
|
||||
//yyyy/mm/dd
|
||||
String basePath = metadata.getBaseDatasetPath();
|
||||
String partitionPath = partition.toUri().getPath();
|
||||
if (!partitionPath.contains(basePath)) {
|
||||
throw new IllegalArgumentException(
|
||||
"Partition path " + partitionPath + " is not part of the dataset " + metadata);
|
||||
}
|
||||
// Get the partition part and remove the / as well at the end
|
||||
String partitionPart = partitionPath.substring(basePath.length() + 1);
|
||||
LOG.info("Extracting parts from " + partitionPart);
|
||||
int year = extractPart(partitionPart, 0);
|
||||
int mm = extractPart(partitionPart, 1);
|
||||
int dd = extractPart(partitionPart, 2);
|
||||
DateTime dateTime = new DateTime(year, mm, dd, 0, 0);
|
||||
return new String[] {dtfOut.print(dateTime)};
|
||||
}
|
||||
|
||||
private int extractPart(String pathString, int index) {
|
||||
String[] parts = pathString.split("/");
|
||||
String part = parts[index];
|
||||
return Integer.parseInt(part);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,79 @@
|
||||
/*
|
||||
* 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();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,108 @@
|
||||
/*
|
||||
* 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.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import parquet.schema.MessageType;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* Represents the schema difference between the storage schema and hive table schema
|
||||
*/
|
||||
public class SchemaDifference {
|
||||
private final MessageType storageSchema;
|
||||
private final Map<String, String> tableSchema;
|
||||
private final List<String> deleteColumns;
|
||||
private final Map<String, String> updateColumnTypes;
|
||||
private final Map<String, String> addColumnTypes;
|
||||
|
||||
private SchemaDifference(MessageType storageSchema, Map<String, String> tableSchema,
|
||||
List<String> deleteColumns, Map<String, String> updateColumnTypes, Map<String, String> addColumnTypes) {
|
||||
this.storageSchema = storageSchema;
|
||||
this.tableSchema = tableSchema;
|
||||
this.deleteColumns = ImmutableList.copyOf(deleteColumns);
|
||||
this.updateColumnTypes = ImmutableMap.copyOf(updateColumnTypes);
|
||||
this.addColumnTypes = ImmutableMap.copyOf(addColumnTypes);
|
||||
}
|
||||
|
||||
public List<String> getDeleteColumns() {
|
||||
return deleteColumns;
|
||||
}
|
||||
|
||||
public Map<String, String> getUpdateColumnTypes() {
|
||||
return updateColumnTypes;
|
||||
}
|
||||
|
||||
public Map<String, String> getAddColumnTypes() {
|
||||
return addColumnTypes;
|
||||
}
|
||||
|
||||
@Override public String toString() {
|
||||
return Objects.toStringHelper(this).add("deleteColumns", deleteColumns)
|
||||
.add("updateColumnTypes", updateColumnTypes).add("addColumnTypes", addColumnTypes)
|
||||
.toString();
|
||||
}
|
||||
|
||||
public static Builder newBuilder(MessageType storageSchema, Map<String, String> tableSchema) {
|
||||
return new Builder(storageSchema, tableSchema);
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
return deleteColumns.isEmpty() && updateColumnTypes.isEmpty() && addColumnTypes.isEmpty();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
private final MessageType storageSchema;
|
||||
private final Map<String, String> tableSchema;
|
||||
private List<String> deleteColumns;
|
||||
private Map<String, String> updateColumnTypes;
|
||||
private Map<String, String> addColumnTypes;
|
||||
|
||||
public Builder(MessageType storageSchema, Map<String, String> tableSchema) {
|
||||
this.storageSchema = storageSchema;
|
||||
this.tableSchema = tableSchema;
|
||||
deleteColumns = Lists.newArrayList();
|
||||
updateColumnTypes = Maps.newHashMap();
|
||||
addColumnTypes = Maps.newHashMap();
|
||||
}
|
||||
|
||||
public Builder deleteTableColumn(String column) {
|
||||
deleteColumns.add(column);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder updateTableColumn(String column, String storageColumnType) {
|
||||
updateColumnTypes.put(column, storageColumnType);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder addTableColumn(String name, String type) {
|
||||
addColumnTypes.put(name, type);
|
||||
return this;
|
||||
}
|
||||
|
||||
public SchemaDifference build() {
|
||||
return new SchemaDifference(storageSchema, tableSchema, deleteColumns, updateColumnTypes, addColumnTypes);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,51 @@
|
||||
/*
|
||||
* 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 Path partitionPath;
|
||||
private final HoodieDatasetReference metadata;
|
||||
|
||||
public StoragePartition(HoodieDatasetReference metadata, PartitionStrategy partitionStrategy,
|
||||
FileStatus input) {
|
||||
this.metadata = metadata;
|
||||
this.partitionPath = Path.getPathWithoutSchemeAndAuthority(input.getPath());
|
||||
this.partitionStrategy = partitionStrategy;
|
||||
}
|
||||
|
||||
public String[] getPartitionFieldValues() {
|
||||
return partitionStrategy.convertPartitionToValues(metadata, partitionPath);
|
||||
}
|
||||
|
||||
public Path getPartitionPath() {
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
@Override public String toString() {
|
||||
return Objects.toStringHelper(this).add("partitionPath", partitionPath)
|
||||
.add("metadata", metadata).toString();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,38 @@
|
||||
/*
|
||||
* 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()]);
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user