You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/03/24 22:13:02 UTC

[GitHub] [hudi] vingov opened a new pull request #5125: MVP implementation of BigQuerySyncTool

vingov opened a new pull request #5125:
URL: https://github.com/apache/hudi/pull/5125


   ## What is the purpose of the pull request
   This pull request adds BigQuery Sync feature, this is a requirement to read Hudi tables on BigQuery.
   
   More details are in RFC-34: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=188745980
   
   ## Brief change log
   
     - *Added BigQuerySyncTool to sync Hudi dataset to Google's BigQuery.*
   
   ## Verify this pull request
   
   This change added tests and can be verified as follows:
   
   *(example:)*
     - *Manually verified the change by running a job locally.*
   
   ## Committer checklist
   
    - [x] Has a corresponding JIRA in PR title & commit
    
    - [x] Commit message is descriptive of the change
    
    - [x] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [x] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1078568661


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b630adc621c074df9b8ada2e5c4435f037484bd2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084160398


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7580e998c628309f0e37a292a4cbacbc83cad438 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613) 
   * ff55dc8ddd8a3f167ab11ab3d454841f8dffc389 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618) 
   * 4999bc9229273673625abb31ed7826b05f58722d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1086709204


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759",
       "triggerID" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7763",
       "triggerID" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e013a9ac1b05495bed3c6f6d089b4cf72584f78b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7763) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] xushiyan merged pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
xushiyan merged pull request #5125:
URL: https://github.com/apache/hudi/pull/5125


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1078436329


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b630adc621c074df9b8ada2e5c4435f037484bd2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] xushiyan commented on a change in pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
xushiyan commented on a change in pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#discussion_r839606109



##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/HoodieBigQueryClient.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+
+import com.google.cloud.bigquery.BigQuery;
+import com.google.cloud.bigquery.BigQueryException;
+import com.google.cloud.bigquery.BigQueryOptions;
+import com.google.cloud.bigquery.CsvOptions;
+import com.google.cloud.bigquery.ExternalTableDefinition;
+import com.google.cloud.bigquery.Field;
+import com.google.cloud.bigquery.FormatOptions;
+import com.google.cloud.bigquery.HivePartitioningOptions;
+import com.google.cloud.bigquery.Schema;
+import com.google.cloud.bigquery.StandardSQLTypeName;
+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 HoodieBigQueryClient extends AbstractSyncHoodieClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieBigQueryClient.class);
+  private transient BigQuery bigquery;
+
+  public HoodieBigQueryClient(final BigQuerySyncConfig syncConfig, final FileSystem fs) {
+    super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata,
+        false, fs);
+    this.createBigQueryConnection();
+  }
+
+  private void createBigQueryConnection() {
+    if (bigquery == null) {
+      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.getDefaultInstance().getService();
+        LOG.info("Successfully established BigQuery connection.");
+      } catch (BigQueryException e) {
+        throw new HoodieBigQuerySyncException("Cannot create bigQuery connection ", e);
+      }
+    }
+  }
+
+  @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 createVersionsTable(
+      String projectId, String datasetName, String tableName, String sourceUri, String sourceUriPrefix, List<String> partitionFields) {
+    try {
+      ExternalTableDefinition customTable;
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+
+      if (partitionFields.isEmpty()) {
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      } else {
+        // Configuring partitioning options for partitioned table.
+        HivePartitioningOptions hivePartitioningOptions =
+            HivePartitioningOptions.newBuilder()
+                .setMode("AUTO")
+                .setRequirePartitionFilter(false)
+                .setSourceUriPrefix(sourceUriPrefix)
+                .build();
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setHivePartitioningOptions(hivePartitioningOptions)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      }
+
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("External table created using hivepartitioningoptions");
+    } catch (BigQueryException e) {
+      throw new HoodieBigQuerySyncException("External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(
+      String projectId, String datasetName, String tableName, String sourceUri) {
+    try {
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+      CsvOptions csvOptions = CsvOptions.newBuilder()
+          .setFieldDelimiter(",")
+          .setAllowJaggedRows(false)
+          .setAllowQuotedNewLines(false)
+          .setSkipLeadingRows(0)
+          .build();
+      Schema schema = Schema.of(
+          Field.of("filename", StandardSQLTypeName.STRING));
+
+      ExternalTableDefinition customTable =
+          ExternalTableDefinition.newBuilder(sourceUri, schema, csvOptions)
+              .setAutodetect(false)
+              .setIgnoreUnknownValues(false)
+              .setMaxBadRecords(0)
+              .build();
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("Manifest External table created.");
+    } catch (BigQueryException e) {
+      throw new HoodieBigQuerySyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  public void createSnapshotView(
+      String projectId, String datasetName, String viewName, String versionsTableName, String manifestTableName) {
+    try {
+      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`)",
+              projectId,
+              datasetName,
+              versionsTableName,
+              projectId,
+              datasetName,
+              manifestTableName);
+
+      ViewDefinition viewDefinition =
+          ViewDefinition.newBuilder(query).setUseLegacySql(false).build();
+
+      bigquery.create(TableInfo.of(tableId, viewDefinition));
+      LOG.info("View created successfully");
+    } catch (BigQueryException e) {
+      throw new HoodieBigQuerySyncException("View was not created ", e);
+    }
+  }
+
+  @Override
+  public Map<String, String> getTableSchema(String tableName) {
+    // TODO: Implement automatic schema evolution when you add a new column.
+    return Collections.<String, String>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.");
+  }
+
+  @Override
+  public void dropPartitionsToTable(final String tableName, final List<String> partitionsToDrop) {
+    // bigQuery discovers the new partitions automatically, so do nothing.
+    throw new UnsupportedOperationException("No support for dropPartitionsToTable yet.");
+  }
+
+  @Override
+  public boolean doesTableExist(final String tableName) {
+    // bigQuery table exists needs different set of arguments, so do nothing.
+    throw new UnsupportedOperationException("No support for doesTableExist yet.");
+  }
+
+  @Override
+  public boolean tableExists(String tableName) {
+    // bigQuery table exists needs different set of arguments, so do nothing.
+    throw new UnsupportedOperationException("No support for tableExists yet.");
+  }
+
+  public boolean doesTableExist(final String projectId, final String datasetName, final String tableName) {
+    TableId tableId = TableId.of(projectId, datasetName, tableName);
+    return bigquery.getTable(tableId, BigQuery.TableOption.fields()) != null;
+  }
+
+  public boolean doesViewExist(final String projectId, final String datasetName, final String viewName) {
+    TableId tableId = TableId.of(projectId, datasetName, viewName);
+    return bigquery.getTable(tableId) != null;
+  }
+
+  @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.");
+  }
+
+  public void close() {

Review comment:
       missing `@Override` ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] xushiyan commented on a change in pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
xushiyan commented on a change in pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#discussion_r839575261



##########
File path: hudi-sync/pom.xml
##########
@@ -35,5 +35,6 @@
     <module>hudi-dla-sync</module>
     <module>hudi-hive-sync</module>
     <module>hudi-sync-common</module>
+    <module>hudi-bigquery-sync</module>

Review comment:
       maybe we should make this a `hudi-gcp` module at the top level, just like `hudi-aws`. In `hudi-aws` we have the glue sync tool as well.

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/BigQuerySyncTool.java
##########
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.bigquery.util.Utils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.InvalidTableException;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+
+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.List;
+import java.util.Properties;
+
+/**
+ * @Experimental
+ *
+ * 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]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync big query table schema
+ */
+public class BigQuerySyncTool extends AbstractSyncTool {
+  private static final Logger LOG = LogManager.getLogger(BigQuerySyncTool.class);
+  public final BigQuerySyncConfig cfg;
+  public final HoodieBigQueryClient hoodieBigQueryClient;
+  public String projectId;
+  public String datasetName;
+  public String manifestTableName;
+  public String versionsTableName;
+  public String snapshotViewName;
+  public String sourceUri;
+  public String sourceUriPrefix;
+  public List<String> partitionFields;
+
+  private BigQuerySyncTool(Properties properties, Configuration conf, FileSystem fs) {
+    super(new TypedProperties(properties), conf, fs);
+    hoodieBigQueryClient = new HoodieBigQueryClient(Utils.propertiesToConfig(properties), fs);
+    cfg = Utils.propertiesToConfig(properties);
+    switch (hoodieBigQueryClient.getTableType()) {
+      case COPY_ON_WRITE:
+        projectId = cfg.projectId;
+        datasetName = cfg.datasetName;
+        manifestTableName = cfg.tableName + "_manifest";
+        versionsTableName = cfg.tableName + "_versions";
+        snapshotViewName = cfg.tableName;
+        sourceUri = cfg.sourceUri;
+        sourceUriPrefix = cfg.sourceUriPrefix;
+        partitionFields = cfg.partitionFields;
+        break;
+      case MERGE_ON_READ:
+        LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      default:
+        LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+    }
+  }
+
+  public static void main(String[] args) {
+    // parse the params
+    BigQuerySyncConfig cfg = new BigQuerySyncConfig();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
+    new BigQuerySyncTool(Utils.configToProperties(cfg), fs.getConf(), fs).syncHoodieTable();
+  }
+
+  @Override
+  public void syncHoodieTable() {
+    try {
+      switch (hoodieBigQueryClient.getTableType()) {
+        case COPY_ON_WRITE:
+          syncCoWTable();
+          break;
+        case MERGE_ON_READ:
+          LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+          throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+        default:
+          LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+          throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      }
+    } catch (Exception e) {
+      throw new HoodieBigQuerySyncException("Got runtime exception when big query syncing " + cfg.tableName, e);
+    }
+  }
+
+  private void syncCoWTable() {
+    LOG.info("Trying to sync hoodie table " + snapshotViewName + " with base path " + hoodieBigQueryClient.getBasePath()
+        + " of type " + hoodieBigQueryClient.getTableType());
+
+    // TODO: Invoke generate manifest routine to refresh the manifest files.

Review comment:
       is this optional in this version?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] xushiyan commented on a change in pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
xushiyan commented on a change in pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#discussion_r839589259



##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/BigQuerySyncTool.java
##########
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.bigquery.util.Utils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.InvalidTableException;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+
+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.List;
+import java.util.Properties;
+
+/**
+ * @Experimental
+ *
+ * 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]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync big query table schema
+ */
+public class BigQuerySyncTool extends AbstractSyncTool {
+  private static final Logger LOG = LogManager.getLogger(BigQuerySyncTool.class);
+  public final BigQuerySyncConfig cfg;
+  public final HoodieBigQueryClient hoodieBigQueryClient;
+  public String projectId;
+  public String datasetName;
+  public String manifestTableName;
+  public String versionsTableName;
+  public String snapshotViewName;
+  public String sourceUri;
+  public String sourceUriPrefix;
+  public List<String> partitionFields;
+
+  private BigQuerySyncTool(Properties properties, Configuration conf, FileSystem fs) {
+    super(new TypedProperties(properties), conf, fs);
+    hoodieBigQueryClient = new HoodieBigQueryClient(Utils.propertiesToConfig(properties), fs);
+    cfg = Utils.propertiesToConfig(properties);
+    switch (hoodieBigQueryClient.getTableType()) {
+      case COPY_ON_WRITE:
+        projectId = cfg.projectId;
+        datasetName = cfg.datasetName;
+        manifestTableName = cfg.tableName + "_manifest";
+        versionsTableName = cfg.tableName + "_versions";
+        snapshotViewName = cfg.tableName;
+        sourceUri = cfg.sourceUri;
+        sourceUriPrefix = cfg.sourceUriPrefix;
+        partitionFields = cfg.partitionFields;
+        break;
+      case MERGE_ON_READ:
+        LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      default:
+        LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+    }
+  }
+
+  public static void main(String[] args) {
+    // parse the params
+    BigQuerySyncConfig cfg = new BigQuerySyncConfig();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
+    new BigQuerySyncTool(Utils.configToProperties(cfg), fs.getConf(), fs).syncHoodieTable();
+  }
+
+  @Override
+  public void syncHoodieTable() {
+    try {
+      switch (hoodieBigQueryClient.getTableType()) {
+        case COPY_ON_WRITE:
+          syncCoWTable();
+          break;
+        case MERGE_ON_READ:
+          LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+          throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+        default:
+          LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+          throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      }
+    } catch (Exception e) {
+      throw new HoodieBigQuerySyncException("Got runtime exception when big query syncing " + cfg.tableName, e);
+    }
+  }
+
+  private void syncCoWTable() {
+    LOG.info("Trying to sync hoodie table " + snapshotViewName + " with base path " + hoodieBigQueryClient.getBasePath()
+        + " of type " + hoodieBigQueryClient.getTableType());
+
+    // TODO: Invoke generate manifest routine to refresh the manifest files.

Review comment:
       is this optional in this version?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] vingov commented on a change in pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
vingov commented on a change in pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#discussion_r839192064



##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/BigQuerySyncTool.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.bigquery.util.Utils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.InvalidTableException;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+
+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.List;
+import java.util.Properties;
+
+/**
+ * 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]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync big query table schema
+ */
+public class BigQuerySyncTool extends AbstractSyncTool {
+  private static final Logger LOG = LogManager.getLogger(BigQuerySyncTool.class);
+  public final BigQuerySyncConfig cfg;
+  public final HoodieBigQueryClient hoodieBigQueryClient;
+  public String projectId;
+  public String datasetName;
+  public String manifestTableName;
+  public String versionsTableName;
+  public String snapshotViewName;
+  public String sourceUri;
+  public String sourceUriPrefix;
+  public List<String> partitionFields;
+
+  private BigQuerySyncTool(Properties properties, Configuration conf, FileSystem fs) {
+    super(new TypedProperties(properties), conf, fs);
+    hoodieBigQueryClient = new HoodieBigQueryClient(Utils.propertiesToConfig(properties), fs);
+    cfg = Utils.propertiesToConfig(properties);
+    switch (hoodieBigQueryClient.getTableType()) {
+      case COPY_ON_WRITE:
+        projectId = cfg.projectId;
+        datasetName = cfg.datasetName;
+        manifestTableName = cfg.tableName + "_manifest";
+        versionsTableName = cfg.tableName + "_versions";
+        snapshotViewName = cfg.tableName;
+        sourceUri = cfg.sourceUri;
+        sourceUriPrefix = cfg.sourceUriPrefix;
+        partitionFields = cfg.partitionFields;
+        break;
+      case MERGE_ON_READ:
+        LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      default:
+        LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+    }
+  }
+
+  public static void main(String[] args) {
+    // parse the params
+    BigQuerySyncConfig cfg = new BigQuerySyncConfig();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
+    new BigQuerySyncTool(Utils.configToProperties(cfg), new Configuration(), fs).syncHoodieTable();

Review comment:
       Updated the code.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084097341


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b630adc621c074df9b8ada2e5c4435f037484bd2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313) 
   * 7580e998c628309f0e37a292a4cbacbc83cad438 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084095771


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b630adc621c074df9b8ada2e5c4435f037484bd2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313) 
   * 7580e998c628309f0e37a292a4cbacbc83cad438 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084213427


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ff55dc8ddd8a3f167ab11ab3d454841f8dffc389 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618) 
   * 4999bc9229273673625abb31ed7826b05f58722d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1085066636


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4999bc9229273673625abb31ed7826b05f58722d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622) 
   * 07da5e1b7556d04a8f40ccba9977ee40e4c24658 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1086693284


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759",
       "triggerID" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 07da5e1b7556d04a8f40ccba9977ee40e4c24658 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658) 
   * cd347e02b31fc2d0b37aaf59da2a2454ad428396 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1078568661


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b630adc621c074df9b8ada2e5c4435f037484bd2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084101901


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7580e998c628309f0e37a292a4cbacbc83cad438 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613) 
   * ff55dc8ddd8a3f167ab11ab3d454841f8dffc389 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] xushiyan commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
xushiyan commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084659739


   @vingov I landed https://github.com/apache/hudi/pull/5153 so you can incorporate that tool here?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084095771


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b630adc621c074df9b8ada2e5c4435f037484bd2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313) 
   * 7580e998c628309f0e37a292a4cbacbc83cad438 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1078434445


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b630adc621c074df9b8ada2e5c4435f037484bd2 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084101901


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7580e998c628309f0e37a292a4cbacbc83cad438 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613) 
   * ff55dc8ddd8a3f167ab11ab3d454841f8dffc389 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] vinothchandar commented on a change in pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on a change in pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#discussion_r839999705



##########
File path: hudi-sync/pom.xml
##########
@@ -35,5 +35,6 @@
     <module>hudi-dla-sync</module>
     <module>hudi-hive-sync</module>
     <module>hudi-sync-common</module>
+    <module>hudi-bigquery-sync</module>

Review comment:
       Thinking out aloud, what you are saying is vendor code (like BigQuery) is in separate module but oss projects (like DataHub) can stay under hudi-sync? That makes sense. We can also add lock providers there for Spanner etc. @vingov You okay to move / rename the module? 
   
   Orthogonally, do we need to add bundles for hudi - aws and hudi-gcloud ? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084157940


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7580e998c628309f0e37a292a4cbacbc83cad438 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613) 
   * ff55dc8ddd8a3f167ab11ab3d454841f8dffc389 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084424292


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4999bc9229273673625abb31ed7826b05f58722d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1086704127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759",
       "triggerID" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7763",
       "triggerID" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * cd347e02b31fc2d0b37aaf59da2a2454ad428396 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759) 
   * e013a9ac1b05495bed3c6f6d089b4cf72584f78b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7763) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1078434445


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b630adc621c074df9b8ada2e5c4435f037484bd2 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1086709204


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759",
       "triggerID" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7763",
       "triggerID" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e013a9ac1b05495bed3c6f6d089b4cf72584f78b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7763) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1086692511


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 07da5e1b7556d04a8f40ccba9977ee40e4c24658 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658) 
   * cd347e02b31fc2d0b37aaf59da2a2454ad428396 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084424292


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4999bc9229273673625abb31ed7826b05f58722d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] xushiyan commented on a change in pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
xushiyan commented on a change in pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#discussion_r838955074



##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/BigQuerySyncConfig.java
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+
+import com.beust.jcommander.Parameter;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Configs needed to sync data into BigQuery.
+ */
+public class BigQuerySyncConfig implements Serializable {
+
+  @Parameter(names = {"--help", "-h"}, help = true)
+  public final Boolean help = false;

Review comment:
       why final?  would this fail if user put -h ?

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/BigQuerySyncTool.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.bigquery.util.Utils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.InvalidTableException;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+
+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.List;
+import java.util.Properties;
+
+/**
+ * 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]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync big query table schema
+ */
+public class BigQuerySyncTool extends AbstractSyncTool {
+  private static final Logger LOG = LogManager.getLogger(BigQuerySyncTool.class);
+  public final BigQuerySyncConfig cfg;
+  public final HoodieBigQueryClient hoodieBigQueryClient;
+  public String projectId;
+  public String datasetName;
+  public String manifestTableName;
+  public String versionsTableName;
+  public String snapshotViewName;
+  public String sourceUri;
+  public String sourceUriPrefix;
+  public List<String> partitionFields;
+
+  private BigQuerySyncTool(Properties properties, Configuration conf, FileSystem fs) {
+    super(new TypedProperties(properties), conf, fs);
+    hoodieBigQueryClient = new HoodieBigQueryClient(Utils.propertiesToConfig(properties), fs);
+    cfg = Utils.propertiesToConfig(properties);
+    switch (hoodieBigQueryClient.getTableType()) {
+      case COPY_ON_WRITE:
+        projectId = cfg.projectId;
+        datasetName = cfg.datasetName;
+        manifestTableName = cfg.tableName + "_manifest";
+        versionsTableName = cfg.tableName + "_versions";
+        snapshotViewName = cfg.tableName;
+        sourceUri = cfg.sourceUri;
+        sourceUriPrefix = cfg.sourceUriPrefix;
+        partitionFields = cfg.partitionFields;
+        break;
+      case MERGE_ON_READ:
+        LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      default:
+        LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+    }
+  }
+
+  public static void main(String[] args) {
+    // parse the params
+    BigQuerySyncConfig cfg = new BigQuerySyncConfig();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
+    new BigQuerySyncTool(Utils.configToProperties(cfg), new Configuration(), fs).syncHoodieTable();

Review comment:
       ```suggestion
       new BigQuerySyncTool(Utils.configToProperties(cfg), fs.getConf(), fs).syncHoodieTable();
   ```

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/BigQuerySyncTool.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.bigquery.util.Utils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.InvalidTableException;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+
+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.List;
+import java.util.Properties;
+
+/**
+ * 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]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync big query table schema
+ */
+public class BigQuerySyncTool extends AbstractSyncTool {
+  private static final Logger LOG = LogManager.getLogger(BigQuerySyncTool.class);
+  public final BigQuerySyncConfig cfg;
+  public final HoodieBigQueryClient hoodieBigQueryClient;
+  public String projectId;
+  public String datasetName;
+  public String manifestTableName;
+  public String versionsTableName;
+  public String snapshotViewName;
+  public String sourceUri;
+  public String sourceUriPrefix;
+  public List<String> partitionFields;
+
+  private BigQuerySyncTool(Properties properties, Configuration conf, FileSystem fs) {
+    super(new TypedProperties(properties), conf, fs);
+    hoodieBigQueryClient = new HoodieBigQueryClient(Utils.propertiesToConfig(properties), fs);
+    cfg = Utils.propertiesToConfig(properties);
+    switch (hoodieBigQueryClient.getTableType()) {
+      case COPY_ON_WRITE:
+        projectId = cfg.projectId;
+        datasetName = cfg.datasetName;
+        manifestTableName = cfg.tableName + "_manifest";
+        versionsTableName = cfg.tableName + "_versions";
+        snapshotViewName = cfg.tableName;
+        sourceUri = cfg.sourceUri;
+        sourceUriPrefix = cfg.sourceUriPrefix;
+        partitionFields = cfg.partitionFields;
+        break;
+      case MERGE_ON_READ:
+        LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      default:
+        LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+    }
+  }
+
+  public static void main(String[] args) {
+    // parse the params
+    BigQuerySyncConfig cfg = new BigQuerySyncConfig();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
+    new BigQuerySyncTool(Utils.configToProperties(cfg), new Configuration(), fs).syncHoodieTable();
+  }
+
+  @Override
+  public void syncHoodieTable() {
+    try {
+      switch (hoodieBigQueryClient.getTableType()) {
+        case COPY_ON_WRITE:
+          syncCoWTable();
+          break;
+        case MERGE_ON_READ:
+          LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+          throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+        default:
+          LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+          throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      }
+    } catch (RuntimeException re) {

Review comment:
       since you're re-throwing, can just catch `Exception` ?

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/BigQuerySyncTool.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.bigquery.util.Utils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.InvalidTableException;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+
+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.List;
+import java.util.Properties;
+
+/**
+ * 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]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync big query table schema

Review comment:
       suggest to mention `@Experimental` in javadoc 

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/HoodieBigQueryClient.java
##########
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+
+import com.google.cloud.bigquery.BigQuery;
+import com.google.cloud.bigquery.BigQueryException;
+import com.google.cloud.bigquery.BigQueryOptions;
+import com.google.cloud.bigquery.CsvOptions;
+import com.google.cloud.bigquery.ExternalTableDefinition;
+import com.google.cloud.bigquery.Field;
+import com.google.cloud.bigquery.FormatOptions;
+import com.google.cloud.bigquery.HivePartitioningOptions;
+import com.google.cloud.bigquery.Schema;
+import com.google.cloud.bigquery.StandardSQLTypeName;
+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.List;
+import java.util.Map;
+
+public class HoodieBigQueryClient extends AbstractSyncHoodieClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieBigQueryClient.class);
+  private transient BigQuery bigquery;
+
+  public HoodieBigQueryClient(final BigQuerySyncConfig syncConfig, final FileSystem fs) {
+    super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata,
+        false, fs);
+    this.createBigQueryConnection();
+  }
+
+  private void createBigQueryConnection() {
+    if (bigquery == null) {
+      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.getDefaultInstance().getService();
+        LOG.info("Successfully established BigQuery connection.");
+      } catch (BigQueryException e) {
+        throw new HoodieException("Cannot create bigQuery connection ", e);
+      }
+    }
+  }
+
+  @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 createVersionsTable(
+      String projectId, String datasetName, String tableName, String sourceUri, String sourceUriPrefix, List<String> partitionFields) {
+    try {
+      ExternalTableDefinition customTable;
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+
+      if (partitionFields != null) {
+        // Configuring partitioning options for partitioned table.
+        HivePartitioningOptions hivePartitioningOptions =
+            HivePartitioningOptions.newBuilder()
+                .setMode("AUTO")
+                .setRequirePartitionFilter(false)
+                .setSourceUriPrefix(sourceUriPrefix)
+                .build();
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setHivePartitioningOptions(hivePartitioningOptions)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      } else {
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      }
+
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("External table created using hivepartitioningoptions");
+    } catch (BigQueryException e) {
+      throw new HoodieException("External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(
+      String projectId, String datasetName, String tableName, String sourceUri) {
+    try {
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+      CsvOptions csvOptions = CsvOptions.newBuilder()
+          .setFieldDelimiter(",")
+          .setAllowJaggedRows(false)
+          .setAllowQuotedNewLines(false)
+          .setSkipLeadingRows(0)
+          .build();
+      Schema schema = Schema.of(
+          Field.of("filename", StandardSQLTypeName.STRING));
+
+      ExternalTableDefinition customTable =
+          ExternalTableDefinition.newBuilder(sourceUri, schema, csvOptions)
+              .setAutodetect(false)
+              .setIgnoreUnknownValues(false)
+              .setMaxBadRecords(0)
+              .build();
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("Manifest External table created.");
+    } catch (BigQueryException e) {
+      throw new HoodieException("Manifest External table was not created ", e);

Review comment:
       ditto

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/HoodieBigQueryClient.java
##########
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+
+import com.google.cloud.bigquery.BigQuery;
+import com.google.cloud.bigquery.BigQueryException;
+import com.google.cloud.bigquery.BigQueryOptions;
+import com.google.cloud.bigquery.CsvOptions;
+import com.google.cloud.bigquery.ExternalTableDefinition;
+import com.google.cloud.bigquery.Field;
+import com.google.cloud.bigquery.FormatOptions;
+import com.google.cloud.bigquery.HivePartitioningOptions;
+import com.google.cloud.bigquery.Schema;
+import com.google.cloud.bigquery.StandardSQLTypeName;
+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.List;
+import java.util.Map;
+
+public class HoodieBigQueryClient extends AbstractSyncHoodieClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieBigQueryClient.class);
+  private transient BigQuery bigquery;
+
+  public HoodieBigQueryClient(final BigQuerySyncConfig syncConfig, final FileSystem fs) {
+    super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata,
+        false, fs);
+    this.createBigQueryConnection();
+  }
+
+  private void createBigQueryConnection() {
+    if (bigquery == null) {
+      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.getDefaultInstance().getService();
+        LOG.info("Successfully established BigQuery connection.");
+      } catch (BigQueryException e) {
+        throw new HoodieException("Cannot create bigQuery connection ", e);
+      }
+    }
+  }
+
+  @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 createVersionsTable(
+      String projectId, String datasetName, String tableName, String sourceUri, String sourceUriPrefix, List<String> partitionFields) {
+    try {
+      ExternalTableDefinition customTable;
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+
+      if (partitionFields != null) {
+        // Configuring partitioning options for partitioned table.
+        HivePartitioningOptions hivePartitioningOptions =
+            HivePartitioningOptions.newBuilder()
+                .setMode("AUTO")
+                .setRequirePartitionFilter(false)
+                .setSourceUriPrefix(sourceUriPrefix)
+                .build();
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setHivePartitioningOptions(hivePartitioningOptions)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      } else {
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      }
+
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("External table created using hivepartitioningoptions");
+    } catch (BigQueryException e) {
+      throw new HoodieException("External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(
+      String projectId, String datasetName, String tableName, String sourceUri) {
+    try {
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+      CsvOptions csvOptions = CsvOptions.newBuilder()
+          .setFieldDelimiter(",")
+          .setAllowJaggedRows(false)
+          .setAllowQuotedNewLines(false)
+          .setSkipLeadingRows(0)
+          .build();
+      Schema schema = Schema.of(
+          Field.of("filename", StandardSQLTypeName.STRING));
+
+      ExternalTableDefinition customTable =
+          ExternalTableDefinition.newBuilder(sourceUri, schema, csvOptions)
+              .setAutodetect(false)
+              .setIgnoreUnknownValues(false)
+              .setMaxBadRecords(0)
+              .build();
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("Manifest External table created.");
+    } catch (BigQueryException e) {
+      throw new HoodieException("Manifest External table was not created ", e);
+    }
+  }
+
+  public void createSnapshotView(
+      String projectId, String datasetName, String viewName, String versionsTableName, String manifestTableName) {
+    try {
+      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`)",
+              projectId,
+              datasetName,
+              versionsTableName,
+              projectId,
+              datasetName,
+              manifestTableName);
+
+      ViewDefinition viewDefinition =
+          ViewDefinition.newBuilder(query).setUseLegacySql(false).build();
+
+      bigquery.create(TableInfo.of(tableId, viewDefinition));
+      LOG.info("View created successfully");
+    } catch (BigQueryException e) {
+      throw new HoodieException("View was not created ", e);
+    }
+  }
+
+  @Override
+  public Map<String, String> getTableSchema(String tableName) {
+    // TODO: Implement automatic schema evolution when you add a new column.
+    return null;
+  }
+
+  @Override
+  public void addPartitionsToTable(final String tableName, final List<String> partitionsToAdd) {
+    // bigQuery discovers the new partitions automatically, so do nothing.
+  }
+
+  @Override
+  public void dropPartitionsToTable(final String tableName, final List<String> partitionsToDrop) {
+    // bigQuery discovers the new partitions automatically, so do nothing.
+  }
+
+  @Override
+  public boolean doesTableExist(final String tableName) {
+    // bigQuery table exists needs different set of arguments, so do nothing.
+    throw new UnsupportedOperationException("Not support doesTableExist yet.");
+  }
+
+  public boolean doesTableExist(final String projectId, final String datasetName, final String tableName) {
+    TableId tableId = TableId.of(projectId, datasetName, tableName);
+    return bigquery.getTable(tableId, BigQuery.TableOption.fields()) != null;
+  }
+
+  public boolean doesViewExist(final String projectId, final String datasetName, final String viewName) {
+    TableId tableId = TableId.of(projectId, datasetName, viewName);
+    return bigquery.getTable(tableId) != null;
+  }
+
+  @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.

Review comment:
       can we standardize on all unsupported APIs to throw `UnsupportedOperationException` ?

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/HoodieBigQueryClient.java
##########
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+
+import com.google.cloud.bigquery.BigQuery;
+import com.google.cloud.bigquery.BigQueryException;
+import com.google.cloud.bigquery.BigQueryOptions;
+import com.google.cloud.bigquery.CsvOptions;
+import com.google.cloud.bigquery.ExternalTableDefinition;
+import com.google.cloud.bigquery.Field;
+import com.google.cloud.bigquery.FormatOptions;
+import com.google.cloud.bigquery.HivePartitioningOptions;
+import com.google.cloud.bigquery.Schema;
+import com.google.cloud.bigquery.StandardSQLTypeName;
+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.List;
+import java.util.Map;
+
+public class HoodieBigQueryClient extends AbstractSyncHoodieClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieBigQueryClient.class);
+  private transient BigQuery bigquery;
+
+  public HoodieBigQueryClient(final BigQuerySyncConfig syncConfig, final FileSystem fs) {
+    super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata,
+        false, fs);
+    this.createBigQueryConnection();
+  }
+
+  private void createBigQueryConnection() {
+    if (bigquery == null) {
+      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.getDefaultInstance().getService();
+        LOG.info("Successfully established BigQuery connection.");
+      } catch (BigQueryException e) {
+        throw new HoodieException("Cannot create bigQuery connection ", e);
+      }
+    }
+  }
+
+  @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 createVersionsTable(
+      String projectId, String datasetName, String tableName, String sourceUri, String sourceUriPrefix, List<String> partitionFields) {
+    try {
+      ExternalTableDefinition customTable;
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+
+      if (partitionFields != null) {
+        // Configuring partitioning options for partitioned table.
+        HivePartitioningOptions hivePartitioningOptions =
+            HivePartitioningOptions.newBuilder()
+                .setMode("AUTO")
+                .setRequirePartitionFilter(false)
+                .setSourceUriPrefix(sourceUriPrefix)
+                .build();
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setHivePartitioningOptions(hivePartitioningOptions)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      } else {
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      }
+
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("External table created using hivepartitioningoptions");
+    } catch (BigQueryException e) {
+      throw new HoodieException("External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(
+      String projectId, String datasetName, String tableName, String sourceUri) {
+    try {
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+      CsvOptions csvOptions = CsvOptions.newBuilder()
+          .setFieldDelimiter(",")
+          .setAllowJaggedRows(false)
+          .setAllowQuotedNewLines(false)
+          .setSkipLeadingRows(0)
+          .build();
+      Schema schema = Schema.of(
+          Field.of("filename", StandardSQLTypeName.STRING));
+
+      ExternalTableDefinition customTable =
+          ExternalTableDefinition.newBuilder(sourceUri, schema, csvOptions)
+              .setAutodetect(false)
+              .setIgnoreUnknownValues(false)
+              .setMaxBadRecords(0)
+              .build();
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("Manifest External table created.");
+    } catch (BigQueryException e) {
+      throw new HoodieException("Manifest External table was not created ", e);
+    }
+  }
+
+  public void createSnapshotView(
+      String projectId, String datasetName, String viewName, String versionsTableName, String manifestTableName) {
+    try {
+      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`)",
+              projectId,
+              datasetName,
+              versionsTableName,
+              projectId,
+              datasetName,
+              manifestTableName);
+
+      ViewDefinition viewDefinition =
+          ViewDefinition.newBuilder(query).setUseLegacySql(false).build();
+
+      bigquery.create(TableInfo.of(tableId, viewDefinition));
+      LOG.info("View created successfully");
+    } catch (BigQueryException e) {
+      throw new HoodieException("View was not created ", e);
+    }
+  }
+
+  @Override
+  public Map<String, String> getTableSchema(String tableName) {
+    // TODO: Implement automatic schema evolution when you add a new column.
+    return null;

Review comment:
       return emptyMap instead of null ?

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/BigQuerySyncTool.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.bigquery.util.Utils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.InvalidTableException;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+
+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.List;
+import java.util.Properties;
+
+/**
+ * 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]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync big query table schema
+ */
+public class BigQuerySyncTool extends AbstractSyncTool {
+  private static final Logger LOG = LogManager.getLogger(BigQuerySyncTool.class);
+  public final BigQuerySyncConfig cfg;
+  public final HoodieBigQueryClient hoodieBigQueryClient;
+  public String projectId;
+  public String datasetName;
+  public String manifestTableName;
+  public String versionsTableName;
+  public String snapshotViewName;
+  public String sourceUri;
+  public String sourceUriPrefix;
+  public List<String> partitionFields;
+
+  private BigQuerySyncTool(Properties properties, Configuration conf, FileSystem fs) {
+    super(new TypedProperties(properties), conf, fs);
+    hoodieBigQueryClient = new HoodieBigQueryClient(Utils.propertiesToConfig(properties), fs);
+    cfg = Utils.propertiesToConfig(properties);
+    switch (hoodieBigQueryClient.getTableType()) {
+      case COPY_ON_WRITE:
+        projectId = cfg.projectId;
+        datasetName = cfg.datasetName;
+        manifestTableName = cfg.tableName + "_manifest";
+        versionsTableName = cfg.tableName + "_versions";
+        snapshotViewName = cfg.tableName;
+        sourceUri = cfg.sourceUri;
+        sourceUriPrefix = cfg.sourceUriPrefix;
+        partitionFields = cfg.partitionFields;
+        break;
+      case MERGE_ON_READ:
+        LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      default:
+        LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+    }
+  }
+
+  public static void main(String[] args) {
+    // parse the params
+    BigQuerySyncConfig cfg = new BigQuerySyncConfig();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
+    new BigQuerySyncTool(Utils.configToProperties(cfg), new Configuration(), fs).syncHoodieTable();
+  }
+
+  @Override
+  public void syncHoodieTable() {
+    try {
+      switch (hoodieBigQueryClient.getTableType()) {
+        case COPY_ON_WRITE:
+          syncCoWTable();
+          break;
+        case MERGE_ON_READ:
+          LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+          throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+        default:
+          LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+          throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      }
+    } catch (RuntimeException re) {
+      throw new HoodieException("Got runtime exception when big query syncing " + cfg.tableName, re);
+    } finally {
+      hoodieBigQueryClient.close();
+    }
+  }
+
+  private void syncCoWTable() {
+    LOG.info("Trying to sync hoodie table " + snapshotViewName + " with base path " + hoodieBigQueryClient.getBasePath()
+        + " of type " + hoodieBigQueryClient.getTableType());

Review comment:
       can have a ValidationUtils.checkState here to make sure it's COW

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/HoodieBigQueryClient.java
##########
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+
+import com.google.cloud.bigquery.BigQuery;
+import com.google.cloud.bigquery.BigQueryException;
+import com.google.cloud.bigquery.BigQueryOptions;
+import com.google.cloud.bigquery.CsvOptions;
+import com.google.cloud.bigquery.ExternalTableDefinition;
+import com.google.cloud.bigquery.Field;
+import com.google.cloud.bigquery.FormatOptions;
+import com.google.cloud.bigquery.HivePartitioningOptions;
+import com.google.cloud.bigquery.Schema;
+import com.google.cloud.bigquery.StandardSQLTypeName;
+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.List;
+import java.util.Map;
+
+public class HoodieBigQueryClient extends AbstractSyncHoodieClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieBigQueryClient.class);
+  private transient BigQuery bigquery;
+
+  public HoodieBigQueryClient(final BigQuerySyncConfig syncConfig, final FileSystem fs) {
+    super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata,
+        false, fs);
+    this.createBigQueryConnection();
+  }
+
+  private void createBigQueryConnection() {
+    if (bigquery == null) {
+      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.getDefaultInstance().getService();
+        LOG.info("Successfully established BigQuery connection.");
+      } catch (BigQueryException e) {
+        throw new HoodieException("Cannot create bigQuery connection ", e);
+      }
+    }
+  }
+
+  @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 createVersionsTable(
+      String projectId, String datasetName, String tableName, String sourceUri, String sourceUriPrefix, List<String> partitionFields) {
+    try {
+      ExternalTableDefinition customTable;
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+
+      if (partitionFields != null) {

Review comment:
       better use the newly added `org.apache.hudi.common.util.CollectionUtils#nonEmpty`

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/HoodieBigQueryClient.java
##########
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+
+import com.google.cloud.bigquery.BigQuery;
+import com.google.cloud.bigquery.BigQueryException;
+import com.google.cloud.bigquery.BigQueryOptions;
+import com.google.cloud.bigquery.CsvOptions;
+import com.google.cloud.bigquery.ExternalTableDefinition;
+import com.google.cloud.bigquery.Field;
+import com.google.cloud.bigquery.FormatOptions;
+import com.google.cloud.bigquery.HivePartitioningOptions;
+import com.google.cloud.bigquery.Schema;
+import com.google.cloud.bigquery.StandardSQLTypeName;
+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.List;
+import java.util.Map;
+
+public class HoodieBigQueryClient extends AbstractSyncHoodieClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieBigQueryClient.class);
+  private transient BigQuery bigquery;
+
+  public HoodieBigQueryClient(final BigQuerySyncConfig syncConfig, final FileSystem fs) {
+    super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata,
+        false, fs);
+    this.createBigQueryConnection();
+  }
+
+  private void createBigQueryConnection() {
+    if (bigquery == null) {
+      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.getDefaultInstance().getService();
+        LOG.info("Successfully established BigQuery connection.");
+      } catch (BigQueryException e) {
+        throw new HoodieException("Cannot create bigQuery connection ", e);
+      }
+    }
+  }
+
+  @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 createVersionsTable(
+      String projectId, String datasetName, String tableName, String sourceUri, String sourceUriPrefix, List<String> partitionFields) {
+    try {
+      ExternalTableDefinition customTable;
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+
+      if (partitionFields != null) {
+        // Configuring partitioning options for partitioned table.
+        HivePartitioningOptions hivePartitioningOptions =
+            HivePartitioningOptions.newBuilder()
+                .setMode("AUTO")
+                .setRequirePartitionFilter(false)
+                .setSourceUriPrefix(sourceUriPrefix)
+                .build();
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setHivePartitioningOptions(hivePartitioningOptions)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      } else {
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      }
+
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("External table created using hivepartitioningoptions");
+    } catch (BigQueryException e) {
+      throw new HoodieException("External table was not created ", e);

Review comment:
       suggest a dedicated exception for this, say `HoodieBigQuerySyncException`

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/BigQuerySyncTool.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.bigquery.util.Utils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.InvalidTableException;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+
+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.List;
+import java.util.Properties;
+
+/**
+ * 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]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync big query table schema
+ */
+public class BigQuerySyncTool extends AbstractSyncTool {
+  private static final Logger LOG = LogManager.getLogger(BigQuerySyncTool.class);
+  public final BigQuerySyncConfig cfg;
+  public final HoodieBigQueryClient hoodieBigQueryClient;
+  public String projectId;
+  public String datasetName;
+  public String manifestTableName;
+  public String versionsTableName;
+  public String snapshotViewName;
+  public String sourceUri;
+  public String sourceUriPrefix;
+  public List<String> partitionFields;
+
+  private BigQuerySyncTool(Properties properties, Configuration conf, FileSystem fs) {
+    super(new TypedProperties(properties), conf, fs);
+    hoodieBigQueryClient = new HoodieBigQueryClient(Utils.propertiesToConfig(properties), fs);
+    cfg = Utils.propertiesToConfig(properties);
+    switch (hoodieBigQueryClient.getTableType()) {
+      case COPY_ON_WRITE:
+        projectId = cfg.projectId;
+        datasetName = cfg.datasetName;
+        manifestTableName = cfg.tableName + "_manifest";
+        versionsTableName = cfg.tableName + "_versions";
+        snapshotViewName = cfg.tableName;
+        sourceUri = cfg.sourceUri;
+        sourceUriPrefix = cfg.sourceUriPrefix;
+        partitionFields = cfg.partitionFields;
+        break;
+      case MERGE_ON_READ:
+        LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      default:
+        LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+    }
+  }
+
+  public static void main(String[] args) {
+    // parse the params
+    BigQuerySyncConfig cfg = new BigQuerySyncConfig();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
+    new BigQuerySyncTool(Utils.configToProperties(cfg), new Configuration(), fs).syncHoodieTable();
+  }
+
+  @Override
+  public void syncHoodieTable() {
+    try {
+      switch (hoodieBigQueryClient.getTableType()) {
+        case COPY_ON_WRITE:
+          syncCoWTable();
+          break;
+        case MERGE_ON_READ:
+          LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+          throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+        default:
+          LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+          throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      }
+    } catch (RuntimeException re) {
+      throw new HoodieException("Got runtime exception when big query syncing " + cfg.tableName, re);
+    } finally {
+      hoodieBigQueryClient.close();

Review comment:
       use try-with-resources?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084213427


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ff55dc8ddd8a3f167ab11ab3d454841f8dffc389 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618) 
   * 4999bc9229273673625abb31ed7826b05f58722d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084162447


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ff55dc8ddd8a3f167ab11ab3d454841f8dffc389 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618) 
   * 4999bc9229273673625abb31ed7826b05f58722d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084160398


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7580e998c628309f0e37a292a4cbacbc83cad438 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613) 
   * ff55dc8ddd8a3f167ab11ab3d454841f8dffc389 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618) 
   * 4999bc9229273673625abb31ed7826b05f58722d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084157940


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7580e998c628309f0e37a292a4cbacbc83cad438 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613) 
   * ff55dc8ddd8a3f167ab11ab3d454841f8dffc389 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1085063744


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4999bc9229273673625abb31ed7826b05f58722d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622) 
   * 07da5e1b7556d04a8f40ccba9977ee40e4c24658 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] vinothchandar commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1085501295


   @vingov on the bundling/package name, I think @xushiyan 's analogy is say DynamoDBLockProvider or cloud watch reporter which are all in the hudi-aws package. 
   
   We started moving towards a model where we group cloud provider services related implementations into a single module, so its easy. For now, the module can be called hudi-gcp and later we can add new indexes and lock providers even there. 
   
   We can call out bigquery clearly in our docs for ease of discovery. Wdyt?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1086692511


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 07da5e1b7556d04a8f40ccba9977ee40e4c24658 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658) 
   * cd347e02b31fc2d0b37aaf59da2a2454ad428396 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084162447


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * ff55dc8ddd8a3f167ab11ab3d454841f8dffc389 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618) 
   * 4999bc9229273673625abb31ed7826b05f58722d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] vingov commented on a change in pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
vingov commented on a change in pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#discussion_r839193909



##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/HoodieBigQueryClient.java
##########
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+
+import com.google.cloud.bigquery.BigQuery;
+import com.google.cloud.bigquery.BigQueryException;
+import com.google.cloud.bigquery.BigQueryOptions;
+import com.google.cloud.bigquery.CsvOptions;
+import com.google.cloud.bigquery.ExternalTableDefinition;
+import com.google.cloud.bigquery.Field;
+import com.google.cloud.bigquery.FormatOptions;
+import com.google.cloud.bigquery.HivePartitioningOptions;
+import com.google.cloud.bigquery.Schema;
+import com.google.cloud.bigquery.StandardSQLTypeName;
+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.List;
+import java.util.Map;
+
+public class HoodieBigQueryClient extends AbstractSyncHoodieClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieBigQueryClient.class);
+  private transient BigQuery bigquery;
+
+  public HoodieBigQueryClient(final BigQuerySyncConfig syncConfig, final FileSystem fs) {
+    super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata,
+        false, fs);
+    this.createBigQueryConnection();
+  }
+
+  private void createBigQueryConnection() {
+    if (bigquery == null) {
+      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.getDefaultInstance().getService();
+        LOG.info("Successfully established BigQuery connection.");
+      } catch (BigQueryException e) {
+        throw new HoodieException("Cannot create bigQuery connection ", e);
+      }
+    }
+  }
+
+  @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 createVersionsTable(
+      String projectId, String datasetName, String tableName, String sourceUri, String sourceUriPrefix, List<String> partitionFields) {
+    try {
+      ExternalTableDefinition customTable;
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+
+      if (partitionFields != null) {
+        // Configuring partitioning options for partitioned table.
+        HivePartitioningOptions hivePartitioningOptions =
+            HivePartitioningOptions.newBuilder()
+                .setMode("AUTO")
+                .setRequirePartitionFilter(false)
+                .setSourceUriPrefix(sourceUriPrefix)
+                .build();
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setHivePartitioningOptions(hivePartitioningOptions)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      } else {
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      }
+
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("External table created using hivepartitioningoptions");
+    } catch (BigQueryException e) {
+      throw new HoodieException("External table was not created ", e);

Review comment:
       Good point, Resolved.

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/HoodieBigQueryClient.java
##########
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+
+import com.google.cloud.bigquery.BigQuery;
+import com.google.cloud.bigquery.BigQueryException;
+import com.google.cloud.bigquery.BigQueryOptions;
+import com.google.cloud.bigquery.CsvOptions;
+import com.google.cloud.bigquery.ExternalTableDefinition;
+import com.google.cloud.bigquery.Field;
+import com.google.cloud.bigquery.FormatOptions;
+import com.google.cloud.bigquery.HivePartitioningOptions;
+import com.google.cloud.bigquery.Schema;
+import com.google.cloud.bigquery.StandardSQLTypeName;
+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.List;
+import java.util.Map;
+
+public class HoodieBigQueryClient extends AbstractSyncHoodieClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieBigQueryClient.class);
+  private transient BigQuery bigquery;
+
+  public HoodieBigQueryClient(final BigQuerySyncConfig syncConfig, final FileSystem fs) {
+    super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata,
+        false, fs);
+    this.createBigQueryConnection();
+  }
+
+  private void createBigQueryConnection() {
+    if (bigquery == null) {
+      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.getDefaultInstance().getService();
+        LOG.info("Successfully established BigQuery connection.");
+      } catch (BigQueryException e) {
+        throw new HoodieException("Cannot create bigQuery connection ", e);
+      }
+    }
+  }
+
+  @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 createVersionsTable(
+      String projectId, String datasetName, String tableName, String sourceUri, String sourceUriPrefix, List<String> partitionFields) {
+    try {
+      ExternalTableDefinition customTable;
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+
+      if (partitionFields != null) {
+        // Configuring partitioning options for partitioned table.
+        HivePartitioningOptions hivePartitioningOptions =
+            HivePartitioningOptions.newBuilder()
+                .setMode("AUTO")
+                .setRequirePartitionFilter(false)
+                .setSourceUriPrefix(sourceUriPrefix)
+                .build();
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setHivePartitioningOptions(hivePartitioningOptions)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      } else {
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      }
+
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("External table created using hivepartitioningoptions");
+    } catch (BigQueryException e) {
+      throw new HoodieException("External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(
+      String projectId, String datasetName, String tableName, String sourceUri) {
+    try {
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+      CsvOptions csvOptions = CsvOptions.newBuilder()
+          .setFieldDelimiter(",")
+          .setAllowJaggedRows(false)
+          .setAllowQuotedNewLines(false)
+          .setSkipLeadingRows(0)
+          .build();
+      Schema schema = Schema.of(
+          Field.of("filename", StandardSQLTypeName.STRING));
+
+      ExternalTableDefinition customTable =
+          ExternalTableDefinition.newBuilder(sourceUri, schema, csvOptions)
+              .setAutodetect(false)
+              .setIgnoreUnknownValues(false)
+              .setMaxBadRecords(0)
+              .build();
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("Manifest External table created.");
+    } catch (BigQueryException e) {
+      throw new HoodieException("Manifest External table was not created ", e);
+    }
+  }
+
+  public void createSnapshotView(
+      String projectId, String datasetName, String viewName, String versionsTableName, String manifestTableName) {
+    try {
+      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`)",
+              projectId,
+              datasetName,
+              versionsTableName,
+              projectId,
+              datasetName,
+              manifestTableName);
+
+      ViewDefinition viewDefinition =
+          ViewDefinition.newBuilder(query).setUseLegacySql(false).build();
+
+      bigquery.create(TableInfo.of(tableId, viewDefinition));
+      LOG.info("View created successfully");
+    } catch (BigQueryException e) {
+      throw new HoodieException("View was not created ", e);
+    }
+  }
+
+  @Override
+  public Map<String, String> getTableSchema(String tableName) {
+    // TODO: Implement automatic schema evolution when you add a new column.
+    return null;

Review comment:
       Good point, Resolved.

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/HoodieBigQueryClient.java
##########
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+
+import com.google.cloud.bigquery.BigQuery;
+import com.google.cloud.bigquery.BigQueryException;
+import com.google.cloud.bigquery.BigQueryOptions;
+import com.google.cloud.bigquery.CsvOptions;
+import com.google.cloud.bigquery.ExternalTableDefinition;
+import com.google.cloud.bigquery.Field;
+import com.google.cloud.bigquery.FormatOptions;
+import com.google.cloud.bigquery.HivePartitioningOptions;
+import com.google.cloud.bigquery.Schema;
+import com.google.cloud.bigquery.StandardSQLTypeName;
+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.List;
+import java.util.Map;
+
+public class HoodieBigQueryClient extends AbstractSyncHoodieClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieBigQueryClient.class);
+  private transient BigQuery bigquery;
+
+  public HoodieBigQueryClient(final BigQuerySyncConfig syncConfig, final FileSystem fs) {
+    super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata,
+        false, fs);
+    this.createBigQueryConnection();
+  }
+
+  private void createBigQueryConnection() {
+    if (bigquery == null) {
+      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.getDefaultInstance().getService();
+        LOG.info("Successfully established BigQuery connection.");
+      } catch (BigQueryException e) {
+        throw new HoodieException("Cannot create bigQuery connection ", e);
+      }
+    }
+  }
+
+  @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 createVersionsTable(
+      String projectId, String datasetName, String tableName, String sourceUri, String sourceUriPrefix, List<String> partitionFields) {
+    try {
+      ExternalTableDefinition customTable;
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+
+      if (partitionFields != null) {
+        // Configuring partitioning options for partitioned table.
+        HivePartitioningOptions hivePartitioningOptions =
+            HivePartitioningOptions.newBuilder()
+                .setMode("AUTO")
+                .setRequirePartitionFilter(false)
+                .setSourceUriPrefix(sourceUriPrefix)
+                .build();
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setHivePartitioningOptions(hivePartitioningOptions)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      } else {
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      }
+
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("External table created using hivepartitioningoptions");
+    } catch (BigQueryException e) {
+      throw new HoodieException("External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(
+      String projectId, String datasetName, String tableName, String sourceUri) {
+    try {
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+      CsvOptions csvOptions = CsvOptions.newBuilder()
+          .setFieldDelimiter(",")
+          .setAllowJaggedRows(false)
+          .setAllowQuotedNewLines(false)
+          .setSkipLeadingRows(0)
+          .build();
+      Schema schema = Schema.of(
+          Field.of("filename", StandardSQLTypeName.STRING));
+
+      ExternalTableDefinition customTable =
+          ExternalTableDefinition.newBuilder(sourceUri, schema, csvOptions)
+              .setAutodetect(false)
+              .setIgnoreUnknownValues(false)
+              .setMaxBadRecords(0)
+              .build();
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("Manifest External table created.");
+    } catch (BigQueryException e) {
+      throw new HoodieException("Manifest External table was not created ", e);

Review comment:
       Resolved.

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/HoodieBigQueryClient.java
##########
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+
+import com.google.cloud.bigquery.BigQuery;
+import com.google.cloud.bigquery.BigQueryException;
+import com.google.cloud.bigquery.BigQueryOptions;
+import com.google.cloud.bigquery.CsvOptions;
+import com.google.cloud.bigquery.ExternalTableDefinition;
+import com.google.cloud.bigquery.Field;
+import com.google.cloud.bigquery.FormatOptions;
+import com.google.cloud.bigquery.HivePartitioningOptions;
+import com.google.cloud.bigquery.Schema;
+import com.google.cloud.bigquery.StandardSQLTypeName;
+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.List;
+import java.util.Map;
+
+public class HoodieBigQueryClient extends AbstractSyncHoodieClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieBigQueryClient.class);
+  private transient BigQuery bigquery;
+
+  public HoodieBigQueryClient(final BigQuerySyncConfig syncConfig, final FileSystem fs) {
+    super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata,
+        false, fs);
+    this.createBigQueryConnection();
+  }
+
+  private void createBigQueryConnection() {
+    if (bigquery == null) {
+      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.getDefaultInstance().getService();
+        LOG.info("Successfully established BigQuery connection.");
+      } catch (BigQueryException e) {
+        throw new HoodieException("Cannot create bigQuery connection ", e);
+      }
+    }
+  }
+
+  @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 createVersionsTable(
+      String projectId, String datasetName, String tableName, String sourceUri, String sourceUriPrefix, List<String> partitionFields) {
+    try {
+      ExternalTableDefinition customTable;
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+
+      if (partitionFields != null) {
+        // Configuring partitioning options for partitioned table.
+        HivePartitioningOptions hivePartitioningOptions =
+            HivePartitioningOptions.newBuilder()
+                .setMode("AUTO")
+                .setRequirePartitionFilter(false)
+                .setSourceUriPrefix(sourceUriPrefix)
+                .build();
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setHivePartitioningOptions(hivePartitioningOptions)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      } else {
+        customTable =
+            ExternalTableDefinition.newBuilder(sourceUri, FormatOptions.parquet())
+                .setAutodetect(true)
+                .setIgnoreUnknownValues(true)
+                .setMaxBadRecords(0)
+                .build();
+      }
+
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("External table created using hivepartitioningoptions");
+    } catch (BigQueryException e) {
+      throw new HoodieException("External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(
+      String projectId, String datasetName, String tableName, String sourceUri) {
+    try {
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+      CsvOptions csvOptions = CsvOptions.newBuilder()
+          .setFieldDelimiter(",")
+          .setAllowJaggedRows(false)
+          .setAllowQuotedNewLines(false)
+          .setSkipLeadingRows(0)
+          .build();
+      Schema schema = Schema.of(
+          Field.of("filename", StandardSQLTypeName.STRING));
+
+      ExternalTableDefinition customTable =
+          ExternalTableDefinition.newBuilder(sourceUri, schema, csvOptions)
+              .setAutodetect(false)
+              .setIgnoreUnknownValues(false)
+              .setMaxBadRecords(0)
+              .build();
+      bigquery.create(TableInfo.of(tableId, customTable));
+      LOG.info("Manifest External table created.");
+    } catch (BigQueryException e) {
+      throw new HoodieException("Manifest External table was not created ", e);
+    }
+  }
+
+  public void createSnapshotView(
+      String projectId, String datasetName, String viewName, String versionsTableName, String manifestTableName) {
+    try {
+      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`)",
+              projectId,
+              datasetName,
+              versionsTableName,
+              projectId,
+              datasetName,
+              manifestTableName);
+
+      ViewDefinition viewDefinition =
+          ViewDefinition.newBuilder(query).setUseLegacySql(false).build();
+
+      bigquery.create(TableInfo.of(tableId, viewDefinition));
+      LOG.info("View created successfully");
+    } catch (BigQueryException e) {
+      throw new HoodieException("View was not created ", e);
+    }
+  }
+
+  @Override
+  public Map<String, String> getTableSchema(String tableName) {
+    // TODO: Implement automatic schema evolution when you add a new column.
+    return null;
+  }
+
+  @Override
+  public void addPartitionsToTable(final String tableName, final List<String> partitionsToAdd) {
+    // bigQuery discovers the new partitions automatically, so do nothing.
+  }
+
+  @Override
+  public void dropPartitionsToTable(final String tableName, final List<String> partitionsToDrop) {
+    // bigQuery discovers the new partitions automatically, so do nothing.
+  }
+
+  @Override
+  public boolean doesTableExist(final String tableName) {
+    // bigQuery table exists needs different set of arguments, so do nothing.
+    throw new UnsupportedOperationException("Not support doesTableExist yet.");
+  }
+
+  public boolean doesTableExist(final String projectId, final String datasetName, final String tableName) {
+    TableId tableId = TableId.of(projectId, datasetName, tableName);
+    return bigquery.getTable(tableId, BigQuery.TableOption.fields()) != null;
+  }
+
+  public boolean doesViewExist(final String projectId, final String datasetName, final String viewName) {
+    TableId tableId = TableId.of(projectId, datasetName, viewName);
+    return bigquery.getTable(tableId) != null;
+  }
+
+  @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.

Review comment:
       Good point, Resolved.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1085063744


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4999bc9229273673625abb31ed7826b05f58722d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622) 
   * 07da5e1b7556d04a8f40ccba9977ee40e4c24658 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1085183021


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 07da5e1b7556d04a8f40ccba9977ee40e4c24658 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1086703546


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759",
       "triggerID" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 07da5e1b7556d04a8f40ccba9977ee40e4c24658 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658) 
   * cd347e02b31fc2d0b37aaf59da2a2454ad428396 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759) 
   * e013a9ac1b05495bed3c6f6d089b4cf72584f78b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1086693284


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759",
       "triggerID" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 07da5e1b7556d04a8f40ccba9977ee40e4c24658 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658) 
   * cd347e02b31fc2d0b37aaf59da2a2454ad428396 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] vingov commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
vingov commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1085487405


   > @vingov I landed #5153 so you can incorporate that tool here?
   
   Thanks for merging the other PR, I've integrated that tool in the Sync Tool class.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] vingov commented on a change in pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
vingov commented on a change in pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#discussion_r840280326



##########
File path: hudi-sync/pom.xml
##########
@@ -35,5 +35,6 @@
     <module>hudi-dla-sync</module>
     <module>hudi-hive-sync</module>
     <module>hudi-sync-common</module>
+    <module>hudi-bigquery-sync</module>

Review comment:
       I initially thought about it, then decided to go with bigquery name because, even without this bigquery sync, users can use hudi with gcp file system, I don't want to cause any confusion for the users, hence went ahead with bigquery and not gcp.
   
   Based on @vinothchandar's suggestion, if we plan to add more gcp eco-system tools in this module, then would it make sense to keep it under hudi-sync module or does it demand a higher level module say hudi-gcp?
   
   I'm okay to rename it after we decide on the correct strategy for future-proofing this module. 
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] vingov commented on a change in pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
vingov commented on a change in pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#discussion_r839193430



##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/BigQuerySyncTool.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.bigquery.util.Utils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.InvalidTableException;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+
+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.List;
+import java.util.Properties;
+
+/**
+ * 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]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync big query table schema
+ */
+public class BigQuerySyncTool extends AbstractSyncTool {
+  private static final Logger LOG = LogManager.getLogger(BigQuerySyncTool.class);
+  public final BigQuerySyncConfig cfg;
+  public final HoodieBigQueryClient hoodieBigQueryClient;
+  public String projectId;
+  public String datasetName;
+  public String manifestTableName;
+  public String versionsTableName;
+  public String snapshotViewName;
+  public String sourceUri;
+  public String sourceUriPrefix;
+  public List<String> partitionFields;
+
+  private BigQuerySyncTool(Properties properties, Configuration conf, FileSystem fs) {
+    super(new TypedProperties(properties), conf, fs);
+    hoodieBigQueryClient = new HoodieBigQueryClient(Utils.propertiesToConfig(properties), fs);
+    cfg = Utils.propertiesToConfig(properties);
+    switch (hoodieBigQueryClient.getTableType()) {
+      case COPY_ON_WRITE:
+        projectId = cfg.projectId;
+        datasetName = cfg.datasetName;
+        manifestTableName = cfg.tableName + "_manifest";
+        versionsTableName = cfg.tableName + "_versions";
+        snapshotViewName = cfg.tableName;
+        sourceUri = cfg.sourceUri;
+        sourceUriPrefix = cfg.sourceUriPrefix;
+        partitionFields = cfg.partitionFields;
+        break;
+      case MERGE_ON_READ:
+        LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      default:
+        LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+    }
+  }
+
+  public static void main(String[] args) {
+    // parse the params
+    BigQuerySyncConfig cfg = new BigQuerySyncConfig();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
+    new BigQuerySyncTool(Utils.configToProperties(cfg), new Configuration(), fs).syncHoodieTable();
+  }
+
+  @Override
+  public void syncHoodieTable() {
+    try {
+      switch (hoodieBigQueryClient.getTableType()) {
+        case COPY_ON_WRITE:
+          syncCoWTable();
+          break;
+        case MERGE_ON_READ:
+          LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+          throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+        default:
+          LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+          throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      }
+    } catch (RuntimeException re) {
+      throw new HoodieException("Got runtime exception when big query syncing " + cfg.tableName, re);
+    } finally {
+      hoodieBigQueryClient.close();

Review comment:
       Since the client is created on a different method, it's hard to use the try-with-resource block in this context.

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/BigQuerySyncTool.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.bigquery.util.Utils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.InvalidTableException;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+
+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.List;
+import java.util.Properties;
+
+/**
+ * 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]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync big query table schema

Review comment:
       Resolved.

##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/HoodieBigQueryClient.java
##########
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.sync.common.AbstractSyncHoodieClient;
+
+import com.google.cloud.bigquery.BigQuery;
+import com.google.cloud.bigquery.BigQueryException;
+import com.google.cloud.bigquery.BigQueryOptions;
+import com.google.cloud.bigquery.CsvOptions;
+import com.google.cloud.bigquery.ExternalTableDefinition;
+import com.google.cloud.bigquery.Field;
+import com.google.cloud.bigquery.FormatOptions;
+import com.google.cloud.bigquery.HivePartitioningOptions;
+import com.google.cloud.bigquery.Schema;
+import com.google.cloud.bigquery.StandardSQLTypeName;
+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.List;
+import java.util.Map;
+
+public class HoodieBigQueryClient extends AbstractSyncHoodieClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieBigQueryClient.class);
+  private transient BigQuery bigquery;
+
+  public HoodieBigQueryClient(final BigQuerySyncConfig syncConfig, final FileSystem fs) {
+    super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata,
+        false, fs);
+    this.createBigQueryConnection();
+  }
+
+  private void createBigQueryConnection() {
+    if (bigquery == null) {
+      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.getDefaultInstance().getService();
+        LOG.info("Successfully established BigQuery connection.");
+      } catch (BigQueryException e) {
+        throw new HoodieException("Cannot create bigQuery connection ", e);
+      }
+    }
+  }
+
+  @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 createVersionsTable(
+      String projectId, String datasetName, String tableName, String sourceUri, String sourceUriPrefix, List<String> partitionFields) {
+    try {
+      ExternalTableDefinition customTable;
+      TableId tableId = TableId.of(projectId, datasetName, tableName);
+
+      if (partitionFields != null) {

Review comment:
       Resolved.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] vingov commented on a change in pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
vingov commented on a change in pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#discussion_r839192680



##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/BigQuerySyncTool.java
##########
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.bigquery.util.Utils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.InvalidTableException;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+
+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.List;
+import java.util.Properties;
+
+/**
+ * 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]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync big query table schema
+ */
+public class BigQuerySyncTool extends AbstractSyncTool {
+  private static final Logger LOG = LogManager.getLogger(BigQuerySyncTool.class);
+  public final BigQuerySyncConfig cfg;
+  public final HoodieBigQueryClient hoodieBigQueryClient;
+  public String projectId;
+  public String datasetName;
+  public String manifestTableName;
+  public String versionsTableName;
+  public String snapshotViewName;
+  public String sourceUri;
+  public String sourceUriPrefix;
+  public List<String> partitionFields;
+
+  private BigQuerySyncTool(Properties properties, Configuration conf, FileSystem fs) {
+    super(new TypedProperties(properties), conf, fs);
+    hoodieBigQueryClient = new HoodieBigQueryClient(Utils.propertiesToConfig(properties), fs);
+    cfg = Utils.propertiesToConfig(properties);
+    switch (hoodieBigQueryClient.getTableType()) {
+      case COPY_ON_WRITE:
+        projectId = cfg.projectId;
+        datasetName = cfg.datasetName;
+        manifestTableName = cfg.tableName + "_manifest";
+        versionsTableName = cfg.tableName + "_versions";
+        snapshotViewName = cfg.tableName;
+        sourceUri = cfg.sourceUri;
+        sourceUriPrefix = cfg.sourceUriPrefix;
+        partitionFields = cfg.partitionFields;
+        break;
+      case MERGE_ON_READ:
+        LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      default:
+        LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+        throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+    }
+  }
+
+  public static void main(String[] args) {
+    // parse the params
+    BigQuerySyncConfig cfg = new BigQuerySyncConfig();
+    JCommander cmd = new JCommander(cfg, null, args);
+    if (cfg.help || args.length == 0) {
+      cmd.usage();
+      System.exit(1);
+    }
+    FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
+    new BigQuerySyncTool(Utils.configToProperties(cfg), new Configuration(), fs).syncHoodieTable();
+  }
+
+  @Override
+  public void syncHoodieTable() {
+    try {
+      switch (hoodieBigQueryClient.getTableType()) {
+        case COPY_ON_WRITE:
+          syncCoWTable();
+          break;
+        case MERGE_ON_READ:
+          LOG.error("Not supported table type " + hoodieBigQueryClient.getTableType());
+          throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+        default:
+          LOG.error("Unknown table type " + hoodieBigQueryClient.getTableType());
+          throw new InvalidTableException(hoodieBigQueryClient.getBasePath());
+      }
+    } catch (RuntimeException re) {

Review comment:
       Resolved.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084100355


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b630adc621c074df9b8ada2e5c4435f037484bd2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313) 
   * 7580e998c628309f0e37a292a4cbacbc83cad438 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613) 
   * ff55dc8ddd8a3f167ab11ab3d454841f8dffc389 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1086703546


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759",
       "triggerID" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 07da5e1b7556d04a8f40ccba9977ee40e4c24658 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658) 
   * cd347e02b31fc2d0b37aaf59da2a2454ad428396 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759) 
   * e013a9ac1b05495bed3c6f6d089b4cf72584f78b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1085183021


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 07da5e1b7556d04a8f40ccba9977ee40e4c24658 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] xushiyan commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
xushiyan commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1086715332


   ## Test setup
   
   - Launch Dataproc 2.0.34-ubuntu18
   - From Dataproc instance launch spark-shell
   ```shell
   spark-shell \
     --jars gs://xxx/hudi-spark3.1-bundle_2.12-0.11.0-SNAPSHOT.jar \
     --packages org.apache.spark:spark-avro_2.12:3.1.2 \
     --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer' \
     --conf 'spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension' \
   ```
   - prepare a partitioned table on GS
   ```scala
   import org.apache.hudi.QuickstartUtils._
   import scala.collection.JavaConversions._
   import org.apache.spark.sql.SaveMode._
   import org.apache.hudi.DataSourceReadOptions._
   import org.apache.hudi.DataSourceWriteOptions._
   import org.apache.hudi.config.HoodieWriteConfig._
   
   val tableName = "hudi_cow_pt_tbl"
   
   spark.sql(
     s"""
       |create table $tableName (
       |  id bigint,
       |  name string,
       |  ts bigint,
       |  dt string
       |) using hudi
       |tblproperties (
       |  type = 'cow',
       |  primaryKey = 'id',
       |  preCombineField = 'ts',
       |  hoodie.datasource.write.hive_style_partitioning = 'true',
       |  hoodie.datasource.write.drop.partition.columns = 'true',
       |  hoodie.metadata.enable = 'false'
       | )
       |partitioned by (dt)
       |location 'gs://foo/bar';
       |""")
   spark.sql(
     s"""
       |insert into $tableName partition (dt)
       |select 1 as id, 'a1' as name, 1000 as ts, '2021-12-09' as dt;
       |""")
   ```
   - Build bundle jars and assembly jar from `hudi-gcp-bundle` module
   ```shell
   mvn -T 2.5C clean install -DskipTests -Dcheckstyle.skip=true -Drat.skip=true -Djacoco.skip=true -Dmaven.javadoc.skip=true -Dscala12 -Dspark3.1
   mvn assembly:single package -pl packaging/hudi-gcp-bundle
   ```
   - Put bundle jars and gcp bundle fat jar in GS bucket
   ```
   gs://xxx/hudi-gcp-bundle-0.11.0-SNAPSHOT-jar-with-dependencies.jar 
   ```
   - Go to BigQuery and create a Dataset `mydataset` (set its location to the same as GS bucket's)
   - From Dataproc server submit the sync tool job
   ```shell
   spark-submit --master yarn \
   --packages org.apache.spark:spark-avro_2.12:3.1.2 \
   --class org.apache.hudi.gcp.bigquery.BigQuerySyncTool \
   gs://xxx/hudi-gcp-bundle-0.11.0-SNAPSHOT-jar-with-dependencies.jar \
   --project-id myproject \
   --dataset-name mydataset \
   --dataset-location <location> \
   --table-name foobar \
   --source-uri gs://foo/bar/dt=* \
   --source-uri-prefix gs://foo/bar/ \
   --base-path gs://foo/bar \
   --partitioned-by dt \
   ```
   - See the job complete from logs
   ```
   22/04/02 19:42:59 INFO org.apache.hudi.gcp.bigquery.HoodieBigQuerySyncClient: Manifest External table created.
   22/04/02 19:42:59 INFO org.apache.hudi.gcp.bigquery.BigQuerySyncTool: Manifest table creation complete for 20220402t081216_manifest
   22/04/02 19:42:59 INFO org.apache.hudi.gcp.bigquery.HoodieBigQuerySyncClient: External table created using hivepartitioningoptions
   22/04/02 19:42:59 INFO org.apache.hudi.gcp.bigquery.BigQuerySyncTool: Versions table creation complete for 20220402t081216_versions
   22/04/02 19:43:01 INFO org.apache.hudi.gcp.bigquery.HoodieBigQuerySyncClient: View created successfully
   22/04/02 19:43:01 INFO org.apache.hudi.gcp.bigquery.BigQuerySyncTool: Snapshot view creation complete for 20220402t081216
   22/04/02 19:43:01 INFO org.apache.hudi.gcp.bigquery.BigQuerySyncTool: Sync table complete for 20220402t081216
   ```
   - See the tables created from bigquery. there should be 2 tables (with suffix `manifest` and ` versions`) and 1 view created. Query the view for the hudi table. Before https://issues.apache.org/jira/browse/HUDI-3290 is landed, manually delete the `.hoodie_partition_metadata` to see the results as a workaround.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1086710619


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759",
       "triggerID" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7763",
       "triggerID" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "754eb69c0a3f164dcfb5ad3099efba0907939052",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "754eb69c0a3f164dcfb5ad3099efba0907939052",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e013a9ac1b05495bed3c6f6d089b4cf72584f78b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7763) 
   * 754eb69c0a3f164dcfb5ad3099efba0907939052 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1078436329


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b630adc621c074df9b8ada2e5c4435f037484bd2 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] vingov commented on a change in pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
vingov commented on a change in pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#discussion_r839191258



##########
File path: hudi-sync/hudi-bigquery-sync/src/main/java/org/apache/hudi/bigquery/BigQuerySyncConfig.java
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.bigquery;
+
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+
+import com.beust.jcommander.Parameter;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Configs needed to sync data into BigQuery.
+ */
+public class BigQuerySyncConfig implements Serializable {
+
+  @Parameter(names = {"--help", "-h"}, help = true)
+  public final Boolean help = false;

Review comment:
       Agreed, updated the code, thanks for flagging.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084100355


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b630adc621c074df9b8ada2e5c4435f037484bd2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313) 
   * 7580e998c628309f0e37a292a4cbacbc83cad438 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613) 
   * ff55dc8ddd8a3f167ab11ab3d454841f8dffc389 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot removed a comment on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot removed a comment on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1084097341


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * b630adc621c074df9b8ada2e5c4435f037484bd2 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313) 
   * 7580e998c628309f0e37a292a4cbacbc83cad438 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1085066636


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4999bc9229273673625abb31ed7826b05f58722d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622) 
   * 07da5e1b7556d04a8f40ccba9977ee40e4c24658 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] xushiyan commented on a change in pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
xushiyan commented on a change in pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#discussion_r840397038



##########
File path: hudi-sync/pom.xml
##########
@@ -35,5 +35,6 @@
     <module>hudi-dla-sync</module>
     <module>hudi-hive-sync</module>
     <module>hudi-sync-common</module>
+    <module>hudi-bigquery-sync</module>

Review comment:
       @vingov a top level module like `hudi-gcp` makes sense, then we'll provide a gcp bundle for everything on gcp. We already have the top-level `hudi-aws`, in which we have glue sync tool. The structure makes things cleaner and easy for bundling and usage.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] vingov commented on a change in pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
vingov commented on a change in pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#discussion_r840977390



##########
File path: hudi-sync/pom.xml
##########
@@ -35,5 +35,6 @@
     <module>hudi-dla-sync</module>
     <module>hudi-hive-sync</module>
     <module>hudi-sync-common</module>
+    <module>hudi-bigquery-sync</module>

Review comment:
       That makes sense to me, I'll make the necessary changes.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1086704127


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759",
       "triggerID" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7763",
       "triggerID" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * cd347e02b31fc2d0b37aaf59da2a2454ad428396 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759) 
   * e013a9ac1b05495bed3c6f6d089b4cf72584f78b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7763) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1086710619


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759",
       "triggerID" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7763",
       "triggerID" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "754eb69c0a3f164dcfb5ad3099efba0907939052",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "754eb69c0a3f164dcfb5ad3099efba0907939052",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e013a9ac1b05495bed3c6f6d089b4cf72584f78b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7763) 
   * 754eb69c0a3f164dcfb5ad3099efba0907939052 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [hudi] hudi-bot commented on pull request #5125: [HUDI-3357] MVP implementation of BigQuerySyncTool

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on pull request #5125:
URL: https://github.com/apache/hudi/pull/5125#issuecomment-1086714523


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7313",
       "triggerID" : "b630adc621c074df9b8ada2e5c4435f037484bd2",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7613",
       "triggerID" : "7580e998c628309f0e37a292a4cbacbc83cad438",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7618",
       "triggerID" : "ff55dc8ddd8a3f167ab11ab3d454841f8dffc389",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4999bc9229273673625abb31ed7826b05f58722d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7622",
       "triggerID" : "4999bc9229273673625abb31ed7826b05f58722d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7658",
       "triggerID" : "07da5e1b7556d04a8f40ccba9977ee40e4c24658",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7759",
       "triggerID" : "cd347e02b31fc2d0b37aaf59da2a2454ad428396",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7763",
       "triggerID" : "e013a9ac1b05495bed3c6f6d089b4cf72584f78b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "754eb69c0a3f164dcfb5ad3099efba0907939052",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7765",
       "triggerID" : "754eb69c0a3f164dcfb5ad3099efba0907939052",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e013a9ac1b05495bed3c6f6d089b4cf72584f78b Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7763) 
   * 754eb69c0a3f164dcfb5ad3099efba0907939052 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=7765) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org