1
0

[HUDI-3730] Improve meta sync class design and hierarchies (#5854)

* [HUDI-3730] Improve meta sync class design and hierarchies (#5754)
* Implements class design proposed in RFC-55

Co-authored-by: jian.feng <fengjian428@gmial.com>
Co-authored-by: jian.feng <jian.feng@shopee.com>
This commit is contained in:
Shiyan Xu
2022-07-03 04:17:25 -05:00
committed by GitHub
parent c00ea84985
commit c0e1587966
86 changed files with 2977 additions and 2877 deletions

View File

@@ -19,113 +19,121 @@
package org.apache.hudi.gcp.bigquery;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.config.ConfigProperty;
import org.apache.hudi.sync.common.HoodieSyncConfig;
import com.beust.jcommander.Parameter;
import com.beust.jcommander.ParametersDelegate;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Properties;
/**
* Configs needed to sync data into BigQuery.
*/
public class BigQuerySyncConfig implements Serializable {
public class BigQuerySyncConfig extends HoodieSyncConfig implements Serializable {
public static String BIGQUERY_SYNC_PROJECT_ID = "hoodie.gcp.bigquery.sync.project_id";
public static String BIGQUERY_SYNC_DATASET_NAME = "hoodie.gcp.bigquery.sync.dataset_name";
public static String BIGQUERY_SYNC_DATASET_LOCATION = "hoodie.gcp.bigquery.sync.dataset_location";
public static String BIGQUERY_SYNC_TABLE_NAME = "hoodie.gcp.bigquery.sync.table_name";
public static String BIGQUERY_SYNC_SOURCE_URI = "hoodie.gcp.bigquery.sync.source_uri";
public static String BIGQUERY_SYNC_SOURCE_URI_PREFIX = "hoodie.gcp.bigquery.sync.source_uri_prefix";
public static String BIGQUERY_SYNC_SYNC_BASE_PATH = "hoodie.gcp.bigquery.sync.base_path";
public static String BIGQUERY_SYNC_PARTITION_FIELDS = "hoodie.gcp.bigquery.sync.partition_fields";
public static String BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA = "hoodie.gcp.bigquery.sync.use_file_listing_from_metadata";
public static String BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING = "hoodie.gcp.bigquery.sync.assume_date_partitioning";
public static final ConfigProperty<String> BIGQUERY_SYNC_PROJECT_ID = ConfigProperty
.key("hoodie.gcp.bigquery.sync.project_id")
.noDefaultValue()
.withDocumentation("Name of the target project in BigQuery");
@Parameter(names = {"--project-id"}, description = "name of the target project in BigQuery", required = true)
public String projectId;
@Parameter(names = {"--dataset-name"}, description = "name of the target dataset in BigQuery", required = true)
public String datasetName;
@Parameter(names = {"--dataset-location"}, description = "location of the target dataset in BigQuery", required = true)
public String datasetLocation;
@Parameter(names = {"--table-name"}, description = "name of the target table in BigQuery", required = true)
public String tableName;
@Parameter(names = {"--source-uri"}, description = "name of the source uri gcs path of the table", required = true)
public String sourceUri;
@Parameter(names = {"--source-uri-prefix"}, description = "name of the source uri gcs path prefix of the table", required = true)
public String sourceUriPrefix;
@Parameter(names = {"--base-path"}, description = "Base path of the hoodie table to sync", required = true)
public String basePath;
@Parameter(names = {"--partitioned-by"}, description = "Comma-delimited partition fields. Default to non-partitioned.")
public List<String> partitionFields = new ArrayList<>();
@Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata")
public Boolean useFileListingFromMetadata = false;
@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;
public static final ConfigProperty<String> BIGQUERY_SYNC_DATASET_NAME = ConfigProperty
.key("hoodie.gcp.bigquery.sync.dataset_name")
.noDefaultValue()
.withDocumentation("Name of the target dataset in BigQuery");
public static BigQuerySyncConfig copy(BigQuerySyncConfig cfg) {
BigQuerySyncConfig newConfig = new BigQuerySyncConfig();
newConfig.projectId = cfg.projectId;
newConfig.datasetName = cfg.datasetName;
newConfig.datasetLocation = cfg.datasetLocation;
newConfig.tableName = cfg.tableName;
newConfig.sourceUri = cfg.sourceUri;
newConfig.sourceUriPrefix = cfg.sourceUriPrefix;
newConfig.basePath = cfg.basePath;
newConfig.partitionFields = cfg.partitionFields;
newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata;
newConfig.assumeDatePartitioning = cfg.assumeDatePartitioning;
newConfig.help = cfg.help;
return newConfig;
public static final ConfigProperty<String> BIGQUERY_SYNC_DATASET_LOCATION = ConfigProperty
.key("hoodie.gcp.bigquery.sync.dataset_location")
.noDefaultValue()
.withDocumentation("Location of the target dataset in BigQuery");
public static final ConfigProperty<String> BIGQUERY_SYNC_TABLE_NAME = ConfigProperty
.key("hoodie.gcp.bigquery.sync.table_name")
.noDefaultValue()
.withDocumentation("Name of the target table in BigQuery");
public static final ConfigProperty<String> BIGQUERY_SYNC_SOURCE_URI = ConfigProperty
.key("hoodie.gcp.bigquery.sync.source_uri")
.noDefaultValue()
.withDocumentation("Name of the source uri gcs path of the table");
public static final ConfigProperty<String> BIGQUERY_SYNC_SOURCE_URI_PREFIX = ConfigProperty
.key("hoodie.gcp.bigquery.sync.source_uri_prefix")
.noDefaultValue()
.withDocumentation("Name of the source uri gcs path prefix of the table");
public static final ConfigProperty<String> BIGQUERY_SYNC_SYNC_BASE_PATH = ConfigProperty
.key("hoodie.gcp.bigquery.sync.base_path")
.noDefaultValue()
.withDocumentation("Base path of the hoodie table to sync");
public static final ConfigProperty<String> BIGQUERY_SYNC_PARTITION_FIELDS = ConfigProperty
.key("hoodie.gcp.bigquery.sync.partition_fields")
.noDefaultValue()
.withDocumentation("Comma-delimited partition fields. Default to non-partitioned.");
public static final ConfigProperty<Boolean> BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA = ConfigProperty
.key("hoodie.gcp.bigquery.sync.use_file_listing_from_metadata")
.defaultValue(false)
.withDocumentation("Fetch file listing from Hudi's metadata");
public static final ConfigProperty<Boolean> BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING = ConfigProperty
.key("hoodie.gcp.bigquery.sync.assume_date_partitioning")
.defaultValue(false)
.withDocumentation("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 BigQuerySyncConfig(Properties props) {
super(props);
}
public TypedProperties toProps() {
TypedProperties properties = new TypedProperties();
properties.put(BIGQUERY_SYNC_PROJECT_ID, projectId);
properties.put(BIGQUERY_SYNC_DATASET_NAME, datasetName);
properties.put(BIGQUERY_SYNC_DATASET_LOCATION, datasetLocation);
properties.put(BIGQUERY_SYNC_TABLE_NAME, tableName);
properties.put(BIGQUERY_SYNC_SOURCE_URI, sourceUri);
properties.put(BIGQUERY_SYNC_SOURCE_URI_PREFIX, sourceUriPrefix);
properties.put(BIGQUERY_SYNC_SYNC_BASE_PATH, basePath);
properties.put(BIGQUERY_SYNC_PARTITION_FIELDS, String.join(",", partitionFields));
properties.put(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA, useFileListingFromMetadata);
properties.put(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING, assumeDatePartitioning);
return properties;
}
public static class BigQuerySyncConfigParams {
public static BigQuerySyncConfig fromProps(TypedProperties props) {
BigQuerySyncConfig config = new BigQuerySyncConfig();
config.projectId = props.getString(BIGQUERY_SYNC_PROJECT_ID);
config.datasetName = props.getString(BIGQUERY_SYNC_DATASET_NAME);
config.datasetLocation = props.getString(BIGQUERY_SYNC_DATASET_LOCATION);
config.tableName = props.getString(BIGQUERY_SYNC_TABLE_NAME);
config.sourceUri = props.getString(BIGQUERY_SYNC_SOURCE_URI);
config.sourceUriPrefix = props.getString(BIGQUERY_SYNC_SOURCE_URI_PREFIX);
config.basePath = props.getString(BIGQUERY_SYNC_SYNC_BASE_PATH);
config.partitionFields = props.getStringList(BIGQUERY_SYNC_PARTITION_FIELDS, ",", Collections.emptyList());
config.useFileListingFromMetadata = props.getBoolean(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA, false);
config.assumeDatePartitioning = props.getBoolean(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING, false);
return config;
}
@ParametersDelegate()
public final HoodieSyncConfigParams hoodieSyncConfigParams = new HoodieSyncConfigParams();
@Override
public String toString() {
return "BigQuerySyncConfig{projectId='" + projectId
+ "', datasetName='" + datasetName
+ "', datasetLocation='" + datasetLocation
+ "', tableName='" + tableName
+ "', sourceUri='" + sourceUri
+ "', sourceUriPrefix='" + sourceUriPrefix
+ "', basePath='" + basePath + "'"
+ ", partitionFields=" + partitionFields
+ "', useFileListingFromMetadata='" + useFileListingFromMetadata
+ "', assumeDataPartitioning='" + assumeDatePartitioning
+ "', help=" + help + "}";
@Parameter(names = {"--project-id"}, description = "Name of the target project in BigQuery", required = true)
public String projectId;
@Parameter(names = {"--dataset-name"}, description = "Name of the target dataset in BigQuery", required = true)
public String datasetName;
@Parameter(names = {"--dataset-location"}, description = "Location of the target dataset in BigQuery", required = true)
public String datasetLocation;
@Parameter(names = {"--table-name"}, description = "Name of the target table in BigQuery", required = true)
public String tableName;
@Parameter(names = {"--source-uri"}, description = "Name of the source uri gcs path of the table", required = true)
public String sourceUri;
@Parameter(names = {"--source-uri-prefix"}, description = "Name of the source uri gcs path prefix of the table", required = true)
public String sourceUriPrefix;
@Parameter(names = {"--base-path"}, description = "Base path of the hoodie table to sync", required = true)
public String basePath;
@Parameter(names = {"--partitioned-by"}, description = "Comma-delimited partition fields. Default to non-partitioned.")
public List<String> partitionFields = new ArrayList<>();
@Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata")
public boolean useFileListingFromMetadata = false;
@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;
public boolean isHelp() {
return hoodieSyncConfigParams.isHelp();
}
public Properties toProps() {
final Properties props = hoodieSyncConfigParams.toProps();
props.setProperty(BIGQUERY_SYNC_PROJECT_ID.key(), projectId);
props.setProperty(BIGQUERY_SYNC_DATASET_NAME.key(), datasetName);
props.setProperty(BIGQUERY_SYNC_DATASET_LOCATION.key(), datasetLocation);
props.setProperty(BIGQUERY_SYNC_TABLE_NAME.key(), tableName);
props.setProperty(BIGQUERY_SYNC_SOURCE_URI.key(), sourceUri);
props.setProperty(BIGQUERY_SYNC_SOURCE_URI_PREFIX.key(), sourceUriPrefix);
props.setProperty(BIGQUERY_SYNC_SYNC_BASE_PATH.key(), basePath);
props.setProperty(BIGQUERY_SYNC_PARTITION_FIELDS.key(), String.join(",", partitionFields));
props.setProperty(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), String.valueOf(useFileListingFromMetadata));
props.setProperty(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING.key(), String.valueOf(assumeDatePartitioning));
return props;
}
}
}

View File

@@ -19,19 +19,26 @@
package org.apache.hudi.gcp.bigquery;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.sync.common.AbstractSyncTool;
import org.apache.hudi.sync.common.HoodieSyncTool;
import org.apache.hudi.sync.common.util.ManifestFileWriter;
import com.beust.jcommander.JCommander;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.util.Properties;
import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING;
import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_NAME;
import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_PARTITION_FIELDS;
import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_SOURCE_URI;
import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_SOURCE_URI_PREFIX;
import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_SYNC_BASE_PATH;
import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_TABLE_NAME;
import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA;
/**
* Tool to sync a hoodie table with a big query table. Either use it as an api
* BigQuerySyncTool.syncHoodieTable(BigQuerySyncConfig) or as a command line java -cp hoodie-hive.jar BigQuerySyncTool [args]
@@ -40,26 +47,28 @@ import org.apache.log4j.Logger;
*
* @Experimental
*/
public class BigQuerySyncTool extends AbstractSyncTool {
public class BigQuerySyncTool extends HoodieSyncTool {
private static final Logger LOG = LogManager.getLogger(BigQuerySyncTool.class);
public final BigQuerySyncConfig cfg;
public final BigQuerySyncConfig config;
public final String tableName;
public final String manifestTableName;
public final String versionsTableName;
public final String snapshotViewName;
public BigQuerySyncTool(TypedProperties properties, Configuration conf, FileSystem fs) {
super(properties, conf, fs);
cfg = BigQuerySyncConfig.fromProps(properties);
manifestTableName = cfg.tableName + "_manifest";
versionsTableName = cfg.tableName + "_versions";
snapshotViewName = cfg.tableName;
public BigQuerySyncTool(Properties props) {
super(props);
this.config = new BigQuerySyncConfig(props);
this.tableName = config.getString(BIGQUERY_SYNC_TABLE_NAME);
this.manifestTableName = tableName + "_manifest";
this.versionsTableName = tableName + "_versions";
this.snapshotViewName = tableName;
}
@Override
public void syncHoodieTable() {
try (HoodieBigQuerySyncClient bqSyncClient = new HoodieBigQuerySyncClient(BigQuerySyncConfig.fromProps(props), fs)) {
try (HoodieBigQuerySyncClient bqSyncClient = new HoodieBigQuerySyncClient(config)) {
switch (bqSyncClient.getTableType()) {
case COPY_ON_WRITE:
syncCoWTable(bqSyncClient);
@@ -69,7 +78,7 @@ public class BigQuerySyncTool extends AbstractSyncTool {
throw new UnsupportedOperationException(bqSyncClient.getTableType() + " table type is not supported yet.");
}
} catch (Exception e) {
throw new HoodieBigQuerySyncException("Got runtime exception when big query syncing " + cfg.tableName, e);
throw new HoodieBigQuerySyncException("Failed to sync BigQuery for table:" + tableName, e);
}
}
@@ -78,14 +87,14 @@ public class BigQuerySyncTool extends AbstractSyncTool {
LOG.info("Sync hoodie table " + snapshotViewName + " at base path " + bqSyncClient.getBasePath());
if (!bqSyncClient.datasetExists()) {
throw new HoodieBigQuerySyncException("Dataset not found: " + cfg);
throw new HoodieBigQuerySyncException("Dataset not found: " + config.getString(BIGQUERY_SYNC_DATASET_NAME));
}
ManifestFileWriter manifestFileWriter = ManifestFileWriter.builder()
.setConf(conf)
.setBasePath(cfg.basePath)
.setUseFileListingFromMetadata(cfg.useFileListingFromMetadata)
.setAssumeDatePartitioning(cfg.assumeDatePartitioning)
.setConf(config.getHadoopConf())
.setBasePath(config.getString(BIGQUERY_SYNC_SYNC_BASE_PATH))
.setUseFileListingFromMetadata(config.getBoolean(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA))
.setAssumeDatePartitioning(config.getBoolean(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING))
.build();
manifestFileWriter.writeManifestFile();
@@ -94,7 +103,11 @@ public class BigQuerySyncTool extends AbstractSyncTool {
LOG.info("Manifest table creation complete for " + manifestTableName);
}
if (!bqSyncClient.tableExists(versionsTableName)) {
bqSyncClient.createVersionsTable(versionsTableName, cfg.sourceUri, cfg.sourceUriPrefix, cfg.partitionFields);
bqSyncClient.createVersionsTable(
versionsTableName,
config.getString(BIGQUERY_SYNC_SOURCE_URI),
config.getString(BIGQUERY_SYNC_SOURCE_URI_PREFIX),
config.getSplitStrings(BIGQUERY_SYNC_PARTITION_FIELDS));
LOG.info("Versions table creation complete for " + versionsTableName);
}
if (!bqSyncClient.tableExists(snapshotViewName)) {
@@ -107,13 +120,13 @@ public class BigQuerySyncTool extends AbstractSyncTool {
}
public static void main(String[] args) {
BigQuerySyncConfig cfg = new BigQuerySyncConfig();
JCommander cmd = new JCommander(cfg, null, args);
if (cfg.help || args.length == 0) {
final BigQuerySyncConfig.BigQuerySyncConfigParams params = new BigQuerySyncConfig.BigQuerySyncConfigParams();
JCommander cmd = JCommander.newBuilder().addObject(params).build();
cmd.parse(args);
if (params.isHelp()) {
cmd.usage();
System.exit(1);
System.exit(0);
}
FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
new BigQuerySyncTool(cfg.toProps(), fs.getConf(), fs).syncHoodieTable();
new BigQuerySyncTool(params.toProps()).syncHoodieTable();
}
}

View File

@@ -19,8 +19,7 @@
package org.apache.hudi.gcp.bigquery;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
import org.apache.hudi.sync.common.HoodieSyncClient;
import com.google.cloud.bigquery.BigQuery;
import com.google.cloud.bigquery.BigQueryException;
@@ -38,25 +37,31 @@ import com.google.cloud.bigquery.Table;
import com.google.cloud.bigquery.TableId;
import com.google.cloud.bigquery.TableInfo;
import com.google.cloud.bigquery.ViewDefinition;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.schema.MessageType;
import java.util.Collections;
import java.util.List;
import java.util.Map;
public class HoodieBigQuerySyncClient extends AbstractSyncHoodieClient {
import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_LOCATION;
import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_NAME;
import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_PROJECT_ID;
public class HoodieBigQuerySyncClient extends HoodieSyncClient {
private static final Logger LOG = LogManager.getLogger(HoodieBigQuerySyncClient.class);
private final BigQuerySyncConfig syncConfig;
protected final BigQuerySyncConfig config;
private final String projectId;
private final String datasetName;
private transient BigQuery bigquery;
public HoodieBigQuerySyncClient(final BigQuerySyncConfig syncConfig, final FileSystem fs) {
super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata,
false, fs);
this.syncConfig = syncConfig;
public HoodieBigQuerySyncClient(final BigQuerySyncConfig config) {
super(config);
this.config = config;
this.projectId = config.getString(BIGQUERY_SYNC_PROJECT_ID);
this.datasetName = config.getString(BIGQUERY_SYNC_DATASET_NAME);
this.createBigQueryConnection();
}
@@ -65,7 +70,7 @@ public class HoodieBigQuerySyncClient extends AbstractSyncHoodieClient {
try {
// Initialize client that will be used to send requests. This client only needs to be created
// once, and can be reused for multiple requests.
bigquery = BigQueryOptions.newBuilder().setLocation(syncConfig.datasetLocation).build().getService();
bigquery = BigQueryOptions.newBuilder().setLocation(config.getString(BIGQUERY_SYNC_DATASET_LOCATION)).build().getService();
LOG.info("Successfully established BigQuery connection.");
} catch (BigQueryException e) {
throw new HoodieBigQuerySyncException("Cannot create bigQuery connection ", e);
@@ -73,16 +78,9 @@ public class HoodieBigQuerySyncClient extends AbstractSyncHoodieClient {
}
}
@Override
public void createTable(final String tableName, final MessageType storageSchema, final String inputFormatClass,
final String outputFormatClass, final String serdeClass,
final Map<String, String> serdeProperties, final Map<String, String> tableProperties) {
// bigQuery create table arguments are different, so do nothing.
}
public void createManifestTable(String tableName, String sourceUri) {
try {
TableId tableId = TableId.of(syncConfig.projectId, syncConfig.datasetName, tableName);
TableId tableId = TableId.of(projectId, datasetName, tableName);
CsvOptions csvOptions = CsvOptions.newBuilder()
.setFieldDelimiter(",")
.setAllowJaggedRows(false)
@@ -108,7 +106,7 @@ public class HoodieBigQuerySyncClient extends AbstractSyncHoodieClient {
public void createVersionsTable(String tableName, String sourceUri, String sourceUriPrefix, List<String> partitionFields) {
try {
ExternalTableDefinition customTable;
TableId tableId = TableId.of(syncConfig.projectId, syncConfig.datasetName, tableName);
TableId tableId = TableId.of(projectId, datasetName, tableName);
if (partitionFields.isEmpty()) {
customTable =
@@ -143,16 +141,16 @@ public class HoodieBigQuerySyncClient extends AbstractSyncHoodieClient {
public void createSnapshotView(String viewName, String versionsTableName, String manifestTableName) {
try {
TableId tableId = TableId.of(syncConfig.projectId, syncConfig.datasetName, viewName);
TableId tableId = TableId.of(projectId, datasetName, viewName);
String query =
String.format(
"SELECT * FROM `%s.%s.%s` WHERE _hoodie_file_name IN "
+ "(SELECT filename FROM `%s.%s.%s`)",
syncConfig.projectId,
syncConfig.datasetName,
projectId,
datasetName,
versionsTableName,
syncConfig.projectId,
syncConfig.datasetName,
projectId,
datasetName,
manifestTableName);
ViewDefinition viewDefinition =
@@ -166,78 +164,25 @@ public class HoodieBigQuerySyncClient extends AbstractSyncHoodieClient {
}
@Override
public Map<String, String> getTableSchema(String tableName) {
public Map<String, String> getMetastoreSchema(String tableName) {
// TODO: Implement automatic schema evolution when you add a new column.
return Collections.emptyMap();
}
@Override
public void addPartitionsToTable(final String tableName, final List<String> partitionsToAdd) {
// bigQuery discovers the new partitions automatically, so do nothing.
throw new UnsupportedOperationException("No support for addPartitionsToTable yet.");
}
public boolean datasetExists() {
Dataset dataset = bigquery.getDataset(DatasetId.of(syncConfig.projectId, syncConfig.datasetName));
Dataset dataset = bigquery.getDataset(DatasetId.of(projectId, datasetName));
return dataset != null;
}
@Override
public boolean doesTableExist(final String tableName) {
return tableExists(tableName);
}
@Override
public boolean tableExists(String tableName) {
TableId tableId = TableId.of(syncConfig.projectId, syncConfig.datasetName, tableName);
TableId tableId = TableId.of(projectId, datasetName, tableName);
Table table = bigquery.getTable(tableId, BigQuery.TableOption.fields());
return table != null && table.exists();
}
@Override
public Option<String> getLastCommitTimeSynced(final String tableName) {
// bigQuery doesn't support tblproperties, so do nothing.
throw new UnsupportedOperationException("Not support getLastCommitTimeSynced yet.");
}
@Override
public void updateLastCommitTimeSynced(final String tableName) {
// bigQuery doesn't support tblproperties, so do nothing.
throw new UnsupportedOperationException("No support for updateLastCommitTimeSynced yet.");
}
@Override
public Option<String> getLastReplicatedTime(String tableName) {
// bigQuery doesn't support tblproperties, so do nothing.
throw new UnsupportedOperationException("Not support getLastReplicatedTime yet.");
}
@Override
public void updateLastReplicatedTimeStamp(String tableName, String timeStamp) {
// bigQuery doesn't support tblproperties, so do nothing.
throw new UnsupportedOperationException("No support for updateLastReplicatedTimeStamp yet.");
}
@Override
public void deleteLastReplicatedTimeStamp(String tableName) {
// bigQuery doesn't support tblproperties, so do nothing.
throw new UnsupportedOperationException("No support for deleteLastReplicatedTimeStamp yet.");
}
@Override
public void updatePartitionsToTable(final String tableName, final List<String> changedPartitions) {
// bigQuery updates the partitions automatically, so do nothing.
throw new UnsupportedOperationException("No support for updatePartitionsToTable yet.");
}
@Override
public void dropPartitions(String tableName, List<String> partitionsToDrop) {
// bigQuery discovers the new partitions automatically, so do nothing.
throw new UnsupportedOperationException("No support for dropPartitions yet.");
}
@Override
public void close() {
// bigQuery has no connection close method, so do nothing.
bigquery = null;
}
}

View File

@@ -19,12 +19,11 @@
package org.apache.hudi.gcp.bigquery;
import org.apache.hudi.common.config.TypedProperties;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Properties;
import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING;
import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_LOCATION;
@@ -44,75 +43,32 @@ public class TestBigQuerySyncConfig {
@BeforeEach
void setUp() {
syncConfig = new BigQuerySyncConfig();
syncConfig.projectId = "fooproject";
syncConfig.datasetName = "foodataset";
syncConfig.datasetLocation = "US";
syncConfig.tableName = "footable";
syncConfig.sourceUri = "gs://test-bucket/dwh/table_name/dt=*";
syncConfig.sourceUriPrefix = "gs://test-bucket/dwh/table_name/";
syncConfig.basePath = "gs://test-bucket/dwh/table_name";
syncConfig.partitionFields = Arrays.asList("a", "b");
syncConfig.useFileListingFromMetadata = true;
syncConfig.assumeDatePartitioning = true;
syncConfig.help = true;
Properties props = new Properties();
props.setProperty(BIGQUERY_SYNC_PROJECT_ID.key(), "fooproject");
props.setProperty(BIGQUERY_SYNC_DATASET_NAME.key(), "foodataset");
props.setProperty(BIGQUERY_SYNC_DATASET_LOCATION.key(), "US");
props.setProperty(BIGQUERY_SYNC_TABLE_NAME.key(), "footable");
props.setProperty(BIGQUERY_SYNC_SOURCE_URI.key(), "gs://test-bucket/dwh/table_name/dt=*");
props.setProperty(BIGQUERY_SYNC_SOURCE_URI_PREFIX.key(), "gs://test-bucket/dwh/table_name/");
props.setProperty(BIGQUERY_SYNC_SYNC_BASE_PATH.key(), "gs://test-bucket/dwh/table_name");
props.setProperty(BIGQUERY_SYNC_PARTITION_FIELDS.key(), "a,b");
props.setProperty(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), "true");
props.setProperty(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING.key(), "true");
syncConfig = new BigQuerySyncConfig(props);
}
@Test
public void testCopy() {
BigQuerySyncConfig copied = BigQuerySyncConfig.copy(syncConfig);
assertEquals(copied.partitionFields, syncConfig.partitionFields);
assertEquals(copied.basePath, syncConfig.basePath);
assertEquals(copied.projectId, syncConfig.projectId);
assertEquals(copied.datasetName, syncConfig.datasetName);
assertEquals(copied.datasetLocation, syncConfig.datasetLocation);
assertEquals(copied.tableName, syncConfig.tableName);
assertEquals(copied.sourceUri, syncConfig.sourceUri);
assertEquals(copied.sourceUriPrefix, syncConfig.sourceUriPrefix);
assertEquals(copied.useFileListingFromMetadata, syncConfig.useFileListingFromMetadata);
assertEquals(copied.assumeDatePartitioning, syncConfig.assumeDatePartitioning);
assertEquals(copied.help, syncConfig.help);
public void testGetConfigs() {
assertEquals("fooproject", syncConfig.getString(BIGQUERY_SYNC_PROJECT_ID));
assertEquals("foodataset", syncConfig.getString(BIGQUERY_SYNC_DATASET_NAME));
assertEquals("US", syncConfig.getString(BIGQUERY_SYNC_DATASET_LOCATION));
assertEquals("footable", syncConfig.getString(BIGQUERY_SYNC_TABLE_NAME));
assertEquals("gs://test-bucket/dwh/table_name/dt=*", syncConfig.getString(BIGQUERY_SYNC_SOURCE_URI));
assertEquals("gs://test-bucket/dwh/table_name/", syncConfig.getString(BIGQUERY_SYNC_SOURCE_URI_PREFIX));
assertEquals("gs://test-bucket/dwh/table_name", syncConfig.getString(BIGQUERY_SYNC_SYNC_BASE_PATH));
assertEquals(Arrays.asList("a", "b"), syncConfig.getSplitStrings(BIGQUERY_SYNC_PARTITION_FIELDS));
assertEquals(true, syncConfig.getBoolean(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA));
assertEquals(true, syncConfig.getBoolean(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING));
}
@Test
public void testToProps() {
TypedProperties props = syncConfig.toProps();
assertEquals("fooproject", props.getString(BIGQUERY_SYNC_PROJECT_ID));
assertEquals("foodataset", props.getString(BIGQUERY_SYNC_DATASET_NAME));
assertEquals("US", props.getString(BIGQUERY_SYNC_DATASET_LOCATION));
assertEquals("footable", props.getString(BIGQUERY_SYNC_TABLE_NAME));
assertEquals("gs://test-bucket/dwh/table_name/dt=*", props.getString(BIGQUERY_SYNC_SOURCE_URI));
assertEquals("gs://test-bucket/dwh/table_name/", props.getString(BIGQUERY_SYNC_SOURCE_URI_PREFIX));
assertEquals("gs://test-bucket/dwh/table_name", props.getString(BIGQUERY_SYNC_SYNC_BASE_PATH));
assertEquals("a,b", props.getString(BIGQUERY_SYNC_PARTITION_FIELDS));
assertEquals("true", props.getString(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA));
assertEquals("true", props.getString(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING));
}
@Test
public void fromProps() {
TypedProperties props = new TypedProperties();
props.put(BIGQUERY_SYNC_PROJECT_ID, "fooproject");
props.put(BIGQUERY_SYNC_DATASET_NAME, "foodataset");
props.put(BIGQUERY_SYNC_DATASET_LOCATION, "US");
props.put(BIGQUERY_SYNC_TABLE_NAME, "footable");
props.put(BIGQUERY_SYNC_SOURCE_URI, "gs://test-bucket/dwh/table_name/dt=*");
props.put(BIGQUERY_SYNC_SOURCE_URI_PREFIX, "gs://test-bucket/dwh/table_name/");
props.put(BIGQUERY_SYNC_SYNC_BASE_PATH, "gs://test-bucket/dwh/table_name");
props.put(BIGQUERY_SYNC_PARTITION_FIELDS, "a,b");
props.put(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA, true);
props.put(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING, true);
BigQuerySyncConfig cfg = BigQuerySyncConfig.fromProps(props);
assertEquals(syncConfig.projectId, cfg.projectId);
assertEquals(syncConfig.datasetName, cfg.datasetName);
assertEquals(syncConfig.datasetLocation, cfg.datasetLocation);
assertEquals(syncConfig.tableName, cfg.tableName);
assertEquals(syncConfig.sourceUri, cfg.sourceUri);
assertEquals(syncConfig.sourceUriPrefix, cfg.sourceUriPrefix);
assertEquals(syncConfig.basePath, cfg.basePath);
assertEquals(syncConfig.partitionFields, cfg.partitionFields);
assertEquals(syncConfig.useFileListingFromMetadata, cfg.useFileListingFromMetadata);
assertEquals(syncConfig.assumeDatePartitioning, cfg.assumeDatePartitioning);
}
}