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/05/23 05:52:00 UTC

[GitHub] [hudi] vingov opened a new pull request, #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   ## What is the purpose of the pull request
   This pull request adds the Snowflake Sync feature, this is a requirement to read Hudi tables on the Snowflake warehouse.
   
   
   ## Brief change log
   
     - *Added SnowflakeSyncTool to sync Hudi dataset to Snowflake cloud data warehouse.*
   
   ## Verify this pull request
   
   This change added tests and can be verified as follows:
   
   *(example:)*
     - *Manually verified the change by running a job locally.*
   
   1. Use the docker demo steps to set up the stock_ticks_cow table on google cloud storage.
   2. Create showflake_profile.properties file with the following configs:
   `# snowflake account details
   URL = https://el36393.us-central1.gcp.snowflakecomputing.com:443
   USER = hudidemo
   PRIVATE_KEY_FILE = /Users/vinothg/.ssh/rsa_key.p8
   ROLE = ACCOUNTADMIN
   WAREHOUSE = COMPUTE_WH
   DB = hudi
   SCHEMA = dwh` 
   3. Run the following command to sync the table to snowflake
   `java -cp guava-25.1-jre.jar:hadoop-auth-2.4.1.jar:commons-configuration-1.9.jar:commons-collections-3.2.1.jar:guava-r05.jar:woodstox-core-5.3.0.jar:stax2-api-4.2.jar:hadoop-common-2.7.3.jar:hudi-spark-bundle_2.12-0.12.0-SNAPSHOT.jar:hudi-snowflake-bundle-0.12.0-SNAPSHOT-jar-with-dependencies.jar:gcs-connector-hadoop2-latest.jar org.apache.hudi.snowflake.sync.SnowflakeSyncTool --properties-file ~/snowflake_profile.properties --base-path gs://hudi-demo/stock_ticks_cow --table-name stock_ticks_cow --storage-integration hudi_demo_int --partitioned-by "date" --partition-extract-expr "\"date\" date as to_date(substr(metadata\$filename, 22, 10), 'YYYY-MM-DD')"`
   
   ## 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] vingov commented on a diff in pull request #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

Posted by GitBox <gi...@apache.org>.
vingov commented on code in PR #5659:
URL: https://github.com/apache/hudi/pull/5659#discussion_r898723120


##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncTool.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+import org.apache.hudi.sync.common.util.ManifestFileWriter;
+
+import com.beust.jcommander.JCommander;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * Tool to sync a hoodie table with a snowflake table. Either use it as an api
+ * SnowflakeSyncTool.syncHoodieTable(SnowflakeSyncConfig) or as a command line java -cp hoodie-hive.jar SnowflakeSyncTool [args]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync snowflake table schema.
+ *
+ * @Experimental
+ */
+public class SnowflakeSyncTool extends AbstractSyncTool {
+
+  private static final Logger LOG = LogManager.getLogger(SnowflakeSyncTool.class);
+
+  public final SnowflakeSyncConfig cfg;
+  public final String stageName;
+  public final String manifestTableName;
+  public final String versionsTableName;
+  public final String snapshotViewName;
+
+  public SnowflakeSyncTool(TypedProperties properties, Configuration conf, FileSystem fs) {
+    super(properties, conf, fs);
+    cfg = SnowflakeSyncConfig.fromProps(properties);
+    stageName = cfg.tableName + "_stage";

Review Comment:
   Good point, done.



-- 
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 diff in pull request #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

Posted by GitBox <gi...@apache.org>.
vingov commented on code in PR #5659:
URL: https://github.com/apache/hudi/pull/5659#discussion_r898726815


##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncTool.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+import org.apache.hudi.sync.common.util.ManifestFileWriter;
+
+import com.beust.jcommander.JCommander;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * Tool to sync a hoodie table with a snowflake table. Either use it as an api
+ * SnowflakeSyncTool.syncHoodieTable(SnowflakeSyncConfig) or as a command line java -cp hoodie-hive.jar SnowflakeSyncTool [args]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync snowflake table schema.
+ *
+ * @Experimental
+ */
+public class SnowflakeSyncTool extends AbstractSyncTool {
+
+  private static final Logger LOG = LogManager.getLogger(SnowflakeSyncTool.class);
+
+  public final SnowflakeSyncConfig cfg;
+  public final String stageName;
+  public final String manifestTableName;
+  public final String versionsTableName;
+  public final String snapshotViewName;
+
+  public SnowflakeSyncTool(TypedProperties properties, Configuration conf, FileSystem fs) {
+    super(properties, conf, fs);
+    cfg = SnowflakeSyncConfig.fromProps(properties);
+    stageName = cfg.tableName + "_stage";
+    manifestTableName = cfg.tableName + "_manifest";
+    versionsTableName = cfg.tableName + "_versions";
+    snapshotViewName = cfg.tableName;
+  }
+
+  @Override
+  public void syncHoodieTable() {
+    try (HoodieSnowflakeSyncClient snowSyncClient = new HoodieSnowflakeSyncClient(SnowflakeSyncConfig.fromProps(props), fs)) {
+      switch (snowSyncClient.getTableType()) {
+        case COPY_ON_WRITE:
+          syncCoWTable(snowSyncClient);
+          break;
+        case MERGE_ON_READ:
+        default:
+          throw new UnsupportedOperationException(snowSyncClient.getTableType() + " table type is not supported yet.");
+      }
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Got runtime exception when snowflake syncing " + cfg.tableName, e);
+    }
+  }
+
+  private void syncCoWTable(HoodieSnowflakeSyncClient snowSyncClient) {
+    ValidationUtils.checkState(snowSyncClient.getTableType() == HoodieTableType.COPY_ON_WRITE);
+    LOG.info("Sync hoodie table " + snapshotViewName + " at base path " + snowSyncClient.getBasePath());
+
+    ManifestFileWriter manifestFileWriter = ManifestFileWriter.builder()
+        .setConf(conf)
+        .setBasePath(cfg.basePath)
+        .setUseFileListingFromMetadata(cfg.useFileListingFromMetadata)
+        .setAssumeDatePartitioning(cfg.assumeDatePartitioning)
+        .build();
+    manifestFileWriter.writeManifestFile();
+
+    snowSyncClient.createStage(stageName, cfg.basePath, cfg.storageIntegration);
+    LOG.info("External temporary stage creation complete for " + stageName);
+    snowSyncClient.createManifestTable(stageName, manifestTableName);

Review Comment:
   Please look at the createSnapshotView method, which is using the manifest file to filter the latest files from the versions table.



-- 
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] prasannarajaperumal commented on a diff in pull request #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

Posted by GitBox <gi...@apache.org>.
prasannarajaperumal commented on code in PR #5659:
URL: https://github.com/apache/hudi/pull/5659#discussion_r931177234


##########
hudi-snowflake/pom.xml:
##########
@@ -0,0 +1,149 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>hudi</artifactId>

Review Comment:
   Why is this a top-level module? Shouldn't this be under hudi-sync?



##########
hudi-snowflake/src/assembly/src.xml:
##########
@@ -0,0 +1,46 @@
+<!--
+ 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.
+  -->
+
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3 http://maven.apache.org/xsd/assembly-1.1.3.xsd">
+  <id>jar-with-dependencies</id>
+  <formats>
+    <format>jar</format>
+  </formats>
+
+  <includeBaseDirectory>false</includeBaseDirectory>
+  <dependencySets>
+
+    <dependencySet>
+      <outputDirectory>/</outputDirectory>
+      <unpack>true</unpack>
+      <scope>runtime</scope>
+      <excludes>
+        <exclude>junit:junit</exclude>
+        <exclude>com.google.code.findbugs:*</exclude>
+        <exclude>org.apache.hbase:*</exclude>

Review Comment:
   why exclude these packages?



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncConfig.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.sync.common.HoodieSyncConfig;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParametersDelegate;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Configs needed to sync data into Snowflake.
+ */
+public class SnowflakeSyncConfig extends HoodieSyncConfig implements Serializable {
+  public static final ConfigProperty<String> SNOWFLAKE_SYNC_PROPERTIES_FILE = ConfigProperty
+      .key("hoodie.snowflake.sync.properties_file")
+      .noDefaultValue()
+      .withDocumentation("Name of the snowflake properties file");
+
+  public static final ConfigProperty<String> SNOWFLAKE_SYNC_STORAGE_INTEGRATION = ConfigProperty
+      .key("hoodie.snowflake.sync.storage_integration")
+      .noDefaultValue()
+      .withDocumentation("Name of the snowflake storage integration");
+
+  public static final ConfigProperty<String> SNOWFLAKE_SYNC_TABLE_NAME = ConfigProperty
+      .key("hoodie.snowflake.sync.table_name")
+      .noDefaultValue()
+      .withDocumentation("Name of the target table in Snowflake");
+
+  public static final ConfigProperty<String> SNOWFLAKE_SYNC_SYNC_BASE_PATH = ConfigProperty

Review Comment:
   Did you intend to have SYNC_SYNC in the name? Can it just be SNOWFLAKE_SYNC_BASE_PATH?



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/HoodieSnowflakeSyncClient.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.sync.common.HoodieSyncClient;
+
+import com.snowflake.snowpark_java.Row;
+import com.snowflake.snowpark_java.Session;
+import com.snowflake.snowpark_java.types.StructType;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_PROPERTIES_FILE;
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT;
+
+/*
+ * Snowflake Hudi client to perform all the table operations on Snowflake Cloud Platform.
+ */
+public class HoodieSnowflakeSyncClient extends HoodieSyncClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieSnowflakeSyncClient.class);
+  private final SnowflakeSyncConfig config;
+  private transient Session snowflakeSession;
+
+  public HoodieSnowflakeSyncClient(final SnowflakeSyncConfig config) {
+    super(config);
+    this.config = config;
+    this.createSnowflakeConnection();
+  }
+
+  private void createSnowflakeConnection() {
+    if (snowflakeSession == 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.
+        snowflakeSession = Session.builder().configFile(config.getString(SNOWFLAKE_SYNC_PROPERTIES_FILE)).create();
+        LOG.info("Successfully established Snowflake connection.");
+      } catch (Exception e) {
+        throw new HoodieSnowflakeSyncException("Cannot create snowflake connection ", e);
+      }
+    }
+  }
+
+  public void createStage(String stageName, String basePath, String storageIntegration) {
+    try {
+      String query = "CREATE OR REPLACE STAGE " + stageName
+          + " url='" + basePath.replace("gs://", "gcs://")

Review Comment:
   I think we should make this generic enough to work with all the supported cloud stores?



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/HoodieSnowflakeSyncClient.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.sync.common.HoodieSyncClient;
+
+import com.snowflake.snowpark_java.Row;
+import com.snowflake.snowpark_java.Session;
+import com.snowflake.snowpark_java.types.StructType;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_PROPERTIES_FILE;
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT;
+
+/*
+ * Snowflake Hudi client to perform all the table operations on Snowflake Cloud Platform.
+ */
+public class HoodieSnowflakeSyncClient extends HoodieSyncClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieSnowflakeSyncClient.class);
+  private final SnowflakeSyncConfig config;
+  private transient Session snowflakeSession;
+
+  public HoodieSnowflakeSyncClient(final SnowflakeSyncConfig config) {
+    super(config);
+    this.config = config;
+    this.createSnowflakeConnection();
+  }
+
+  private void createSnowflakeConnection() {
+    if (snowflakeSession == 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.
+        snowflakeSession = Session.builder().configFile(config.getString(SNOWFLAKE_SYNC_PROPERTIES_FILE)).create();
+        LOG.info("Successfully established Snowflake connection.");
+      } catch (Exception e) {
+        throw new HoodieSnowflakeSyncException("Cannot create snowflake connection ", e);
+      }
+    }
+  }
+
+  public void createStage(String stageName, String basePath, String storageIntegration) {
+    try {
+      String query = "CREATE OR REPLACE STAGE " + stageName
+          + " url='" + basePath.replace("gs://", "gcs://")
+          + "' STORAGE_INTEGRATION = " + storageIntegration;
+      snowflakeSession.sql(query).show();
+      LOG.info("Manifest External table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(String stageName, String tableName) {
+    try {
+      String query = "CREATE OR REPLACE EXTERNAL TABLE " + tableName + " ("
+          + "    filename VARCHAR AS split_part(VALUE:c1, '/', -1)"
+          + "  )"
+          + "WITH LOCATION = @" + stageName + "/.hoodie/manifest/"
+          + "  FILE_FORMAT = (TYPE = CSV)"
+          + "  AUTO_REFRESH = False";
+      snowflakeSession.sql(query).show();
+      LOG.info("Manifest External table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  private void createCustomFileFormat(String fileFormatName) {
+    try {
+      String query = "CREATE OR REPLACE FILE FORMAT " + fileFormatName + " TYPE = '" + config.getString(SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT) + "';";
+      snowflakeSession.sql(query).show();
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Custom file format was not created. ", e);
+    }
+  }
+
+  private List<String> generateSchemaWithoutPartitionColumns(String stageName, String fileFormatName) {
+    try {
+      String query = "SELECT"
+          + "  generate_column_description(array_agg(object_construct(*)), 'external_table') as columns"
+          + " FROM"
+          + "  table("
+          + "    infer_schema("
+          + "      location => '@" + stageName + "',"
+          + "      file_format => '" + fileFormatName + "'"
+          + "    )"
+          + ")";
+      Optional<Row> row = snowflakeSession.sql(query).first();
+      String columns = row.get().get(0).toString();
+      if (columns.isEmpty()) {
+        throw new HoodieSnowflakeSyncException("Unable to infer the schema with the given data files.");
+      }
+      return Arrays.asList(columns.split(",", -1));
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Unable to infer the schema with the given data files. ", e);
+    }
+  }
+
+  public void createVersionsTable(String stageName, String tableName, String partitionFields, String partitionExtractExpr) {

Review Comment:
   I think we are imposing the naming convention of Gcs here on to snowflake? I dont understand why this external table is called versions table? 



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/HoodieSnowflakeSyncClient.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.sync.common.HoodieSyncClient;
+
+import com.snowflake.snowpark_java.Row;
+import com.snowflake.snowpark_java.Session;
+import com.snowflake.snowpark_java.types.StructType;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_PROPERTIES_FILE;
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT;
+
+/*
+ * Snowflake Hudi client to perform all the table operations on Snowflake Cloud Platform.
+ */
+public class HoodieSnowflakeSyncClient extends HoodieSyncClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieSnowflakeSyncClient.class);
+  private final SnowflakeSyncConfig config;
+  private transient Session snowflakeSession;
+
+  public HoodieSnowflakeSyncClient(final SnowflakeSyncConfig config) {
+    super(config);
+    this.config = config;
+    this.createSnowflakeConnection();
+  }
+
+  private void createSnowflakeConnection() {
+    if (snowflakeSession == 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.
+        snowflakeSession = Session.builder().configFile(config.getString(SNOWFLAKE_SYNC_PROPERTIES_FILE)).create();
+        LOG.info("Successfully established Snowflake connection.");
+      } catch (Exception e) {
+        throw new HoodieSnowflakeSyncException("Cannot create snowflake connection ", e);
+      }
+    }
+  }
+
+  public void createStage(String stageName, String basePath, String storageIntegration) {
+    try {
+      String query = "CREATE OR REPLACE STAGE " + stageName
+          + " url='" + basePath.replace("gs://", "gcs://")
+          + "' STORAGE_INTEGRATION = " + storageIntegration;
+      snowflakeSession.sql(query).show();
+      LOG.info("Manifest External table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(String stageName, String tableName) {
+    try {
+      String query = "CREATE OR REPLACE EXTERNAL TABLE " + tableName + " ("
+          + "    filename VARCHAR AS split_part(VALUE:c1, '/', -1)"
+          + "  )"
+          + "WITH LOCATION = @" + stageName + "/.hoodie/manifest/"
+          + "  FILE_FORMAT = (TYPE = CSV)"
+          + "  AUTO_REFRESH = False";
+      snowflakeSession.sql(query).show();
+      LOG.info("Manifest External table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  private void createCustomFileFormat(String fileFormatName) {
+    try {
+      String query = "CREATE OR REPLACE FILE FORMAT " + fileFormatName + " TYPE = '" + config.getString(SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT) + "';";
+      snowflakeSession.sql(query).show();
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Custom file format was not created. ", e);
+    }
+  }
+
+  private List<String> generateSchemaWithoutPartitionColumns(String stageName, String fileFormatName) {
+    try {
+      String query = "SELECT"
+          + "  generate_column_description(array_agg(object_construct(*)), 'external_table') as columns"
+          + " FROM"
+          + "  table("
+          + "    infer_schema("
+          + "      location => '@" + stageName + "',"
+          + "      file_format => '" + fileFormatName + "'"
+          + "    )"
+          + ")";
+      Optional<Row> row = snowflakeSession.sql(query).first();

Review Comment:
   Use JDBC to get the value. Including snowpark for this seems excessive. 



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/HoodieSnowflakeSyncClient.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.sync.common.HoodieSyncClient;
+
+import com.snowflake.snowpark_java.Row;
+import com.snowflake.snowpark_java.Session;
+import com.snowflake.snowpark_java.types.StructType;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_PROPERTIES_FILE;
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT;
+
+/*
+ * Snowflake Hudi client to perform all the table operations on Snowflake Cloud Platform.
+ */
+public class HoodieSnowflakeSyncClient extends HoodieSyncClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieSnowflakeSyncClient.class);
+  private final SnowflakeSyncConfig config;
+  private transient Session snowflakeSession;
+
+  public HoodieSnowflakeSyncClient(final SnowflakeSyncConfig config) {
+    super(config);
+    this.config = config;
+    this.createSnowflakeConnection();
+  }
+
+  private void createSnowflakeConnection() {
+    if (snowflakeSession == 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.
+        snowflakeSession = Session.builder().configFile(config.getString(SNOWFLAKE_SYNC_PROPERTIES_FILE)).create();
+        LOG.info("Successfully established Snowflake connection.");
+      } catch (Exception e) {
+        throw new HoodieSnowflakeSyncException("Cannot create snowflake connection ", e);
+      }
+    }
+  }
+
+  public void createStage(String stageName, String basePath, String storageIntegration) {
+    try {
+      String query = "CREATE OR REPLACE STAGE " + stageName
+          + " url='" + basePath.replace("gs://", "gcs://")
+          + "' STORAGE_INTEGRATION = " + storageIntegration;
+      snowflakeSession.sql(query).show();
+      LOG.info("Manifest External table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(String stageName, String tableName) {
+    try {
+      String query = "CREATE OR REPLACE EXTERNAL TABLE " + tableName + " ("

Review Comment:
   It is better for the list of files to be an internal snowflake table (not an external table). This will unlock compiler optimizations in snowflake for the sub-query on the snapshot view. Plus, This should be a fairly simple to do this using copy into statement in snowflake. 



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/HoodieSnowflakeSyncClient.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.sync.common.HoodieSyncClient;
+
+import com.snowflake.snowpark_java.Row;
+import com.snowflake.snowpark_java.Session;
+import com.snowflake.snowpark_java.types.StructType;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_PROPERTIES_FILE;
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT;
+
+/*
+ * Snowflake Hudi client to perform all the table operations on Snowflake Cloud Platform.
+ */
+public class HoodieSnowflakeSyncClient extends HoodieSyncClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieSnowflakeSyncClient.class);
+  private final SnowflakeSyncConfig config;
+  private transient Session snowflakeSession;
+
+  public HoodieSnowflakeSyncClient(final SnowflakeSyncConfig config) {
+    super(config);
+    this.config = config;
+    this.createSnowflakeConnection();
+  }
+
+  private void createSnowflakeConnection() {
+    if (snowflakeSession == 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.
+        snowflakeSession = Session.builder().configFile(config.getString(SNOWFLAKE_SYNC_PROPERTIES_FILE)).create();
+        LOG.info("Successfully established Snowflake connection.");
+      } catch (Exception e) {
+        throw new HoodieSnowflakeSyncException("Cannot create snowflake connection ", e);
+      }
+    }
+  }
+
+  public void createStage(String stageName, String basePath, String storageIntegration) {
+    try {
+      String query = "CREATE OR REPLACE STAGE " + stageName
+          + " url='" + basePath.replace("gs://", "gcs://")
+          + "' STORAGE_INTEGRATION = " + storageIntegration;
+      snowflakeSession.sql(query).show();
+      LOG.info("Manifest External table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(String stageName, String tableName) {
+    try {
+      String query = "CREATE OR REPLACE EXTERNAL TABLE " + tableName + " ("
+          + "    filename VARCHAR AS split_part(VALUE:c1, '/', -1)"
+          + "  )"
+          + "WITH LOCATION = @" + stageName + "/.hoodie/manifest/"
+          + "  FILE_FORMAT = (TYPE = CSV)"
+          + "  AUTO_REFRESH = False";
+      snowflakeSession.sql(query).show();
+      LOG.info("Manifest External table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  private void createCustomFileFormat(String fileFormatName) {
+    try {
+      String query = "CREATE OR REPLACE FILE FORMAT " + fileFormatName + " TYPE = '" + config.getString(SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT) + "';";
+      snowflakeSession.sql(query).show();
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Custom file format was not created. ", e);
+    }
+  }
+
+  private List<String> generateSchemaWithoutPartitionColumns(String stageName, String fileFormatName) {
+    try {
+      String query = "SELECT"
+          + "  generate_column_description(array_agg(object_construct(*)), 'external_table') as columns"
+          + " FROM"
+          + "  table("
+          + "    infer_schema("
+          + "      location => '@" + stageName + "',"
+          + "      file_format => '" + fileFormatName + "'"
+          + "    )"
+          + ")";
+      Optional<Row> row = snowflakeSession.sql(query).first();
+      String columns = row.get().get(0).toString();
+      if (columns.isEmpty()) {
+        throw new HoodieSnowflakeSyncException("Unable to infer the schema with the given data files.");
+      }
+      return Arrays.asList(columns.split(",", -1));
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Unable to infer the schema with the given data files. ", e);
+    }
+  }
+
+  public void createVersionsTable(String stageName, String tableName, String partitionFields, String partitionExtractExpr) {
+    try {
+      String fileFormatName = "my_custom_file_format";

Review Comment:
   Please do not hardcode the file format name. 



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncConfig.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.sync.common.HoodieSyncConfig;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.ParametersDelegate;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+/**
+ * Configs needed to sync data into Snowflake.
+ */
+public class SnowflakeSyncConfig extends HoodieSyncConfig implements Serializable {
+  public static final ConfigProperty<String> SNOWFLAKE_SYNC_PROPERTIES_FILE = ConfigProperty
+      .key("hoodie.snowflake.sync.properties_file")
+      .noDefaultValue()
+      .withDocumentation("Name of the snowflake properties file");
+
+  public static final ConfigProperty<String> SNOWFLAKE_SYNC_STORAGE_INTEGRATION = ConfigProperty
+      .key("hoodie.snowflake.sync.storage_integration")
+      .noDefaultValue()
+      .withDocumentation("Name of the snowflake storage integration");
+
+  public static final ConfigProperty<String> SNOWFLAKE_SYNC_TABLE_NAME = ConfigProperty
+      .key("hoodie.snowflake.sync.table_name")
+      .noDefaultValue()
+      .withDocumentation("Name of the target table in Snowflake");
+
+  public static final ConfigProperty<String> SNOWFLAKE_SYNC_SYNC_BASE_PATH = ConfigProperty
+      .key("hoodie.snowflake.sync.base_path")
+      .noDefaultValue()
+      .withDocumentation("Base path of the hoodie table to sync.");
+
+  public static final ConfigProperty<String> SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT = ConfigProperty

Review Comment:
   Same as above. 



##########
hudi-snowflake/pom.xml:
##########
@@ -0,0 +1,149 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<project xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns="http://maven.apache.org/POM/4.0.0"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>hudi</artifactId>
+    <groupId>org.apache.hudi</groupId>
+    <version>0.12.0-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>hudi-snowflake</artifactId>
+  <packaging>jar</packaging>
+
+  <dependencies>
+    <!-- Hoodie -->
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-sync-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <!-- Snowflake -->
+    <dependency>
+      <groupId>com.snowflake</groupId>
+      <artifactId>snowpark</artifactId>

Review Comment:
   I dont understand why we need snowpark here?



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/HoodieSnowflakeSyncClient.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.sync.common.HoodieSyncClient;
+
+import com.snowflake.snowpark_java.Row;
+import com.snowflake.snowpark_java.Session;
+import com.snowflake.snowpark_java.types.StructType;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_PROPERTIES_FILE;
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT;
+
+/*
+ * Snowflake Hudi client to perform all the table operations on Snowflake Cloud Platform.
+ */
+public class HoodieSnowflakeSyncClient extends HoodieSyncClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieSnowflakeSyncClient.class);
+  private final SnowflakeSyncConfig config;
+  private transient Session snowflakeSession;
+
+  public HoodieSnowflakeSyncClient(final SnowflakeSyncConfig config) {
+    super(config);
+    this.config = config;
+    this.createSnowflakeConnection();
+  }
+
+  private void createSnowflakeConnection() {
+    if (snowflakeSession == 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.
+        snowflakeSession = Session.builder().configFile(config.getString(SNOWFLAKE_SYNC_PROPERTIES_FILE)).create();
+        LOG.info("Successfully established Snowflake connection.");
+      } catch (Exception e) {
+        throw new HoodieSnowflakeSyncException("Cannot create snowflake connection ", e);
+      }
+    }
+  }
+
+  public void createStage(String stageName, String basePath, String storageIntegration) {
+    try {
+      String query = "CREATE OR REPLACE STAGE " + stageName
+          + " url='" + basePath.replace("gs://", "gcs://")
+          + "' STORAGE_INTEGRATION = " + storageIntegration;
+      snowflakeSession.sql(query).show();
+      LOG.info("Manifest External table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(String stageName, String tableName) {
+    try {
+      String query = "CREATE OR REPLACE EXTERNAL TABLE " + tableName + " ("
+          + "    filename VARCHAR AS split_part(VALUE:c1, '/', -1)"
+          + "  )"
+          + "WITH LOCATION = @" + stageName + "/.hoodie/manifest/"
+          + "  FILE_FORMAT = (TYPE = CSV)"
+          + "  AUTO_REFRESH = False";
+      snowflakeSession.sql(query).show();
+      LOG.info("Manifest External table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  private void createCustomFileFormat(String fileFormatName) {
+    try {
+      String query = "CREATE OR REPLACE FILE FORMAT " + fileFormatName + " TYPE = '" + config.getString(SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT) + "';";
+      snowflakeSession.sql(query).show();
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Custom file format was not created. ", e);
+    }
+  }
+
+  private List<String> generateSchemaWithoutPartitionColumns(String stageName, String fileFormatName) {
+    try {
+      String query = "SELECT"
+          + "  generate_column_description(array_agg(object_construct(*)), 'external_table') as columns"
+          + " FROM"
+          + "  table("
+          + "    infer_schema("
+          + "      location => '@" + stageName + "',"
+          + "      file_format => '" + fileFormatName + "'"
+          + "    )"
+          + ")";
+      Optional<Row> row = snowflakeSession.sql(query).first();
+      String columns = row.get().get(0).toString();
+      if (columns.isEmpty()) {
+        throw new HoodieSnowflakeSyncException("Unable to infer the schema with the given data files.");
+      }
+      return Arrays.asList(columns.split(",", -1));
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Unable to infer the schema with the given data files. ", e);
+    }
+  }
+
+  public void createVersionsTable(String stageName, String tableName, String partitionFields, String partitionExtractExpr) {
+    try {
+      String fileFormatName = "my_custom_file_format";
+      createCustomFileFormat(fileFormatName);

Review Comment:
   Snowflake will disable a lot of performance optimization on external tables if we end up creating a custom file format? Why create a custom file formats if the data is in one of the snowflake supported formats? (https://docs.snowflake.com/en/sql-reference/sql/create-external-table.html#format-type-options-formattypeoptions)  



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncTool.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.sync.common.HoodieSyncTool;
+import org.apache.hudi.sync.common.util.ManifestFileWriter;
+
+import com.beust.jcommander.JCommander;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.Properties;
+
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_PARTITION_EXTRACT_EXPRESSION;
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_PARTITION_FIELDS;
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_STORAGE_INTEGRATION;
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_SYNC_BASE_PATH;
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_TABLE_NAME;
+
+/**
+ * Tool to sync a hoodie table with a snowflake table. Either use it as an api
+ * SnowflakeSyncTool.syncHoodieTable(SnowflakeSyncConfig) or as a command line java -cp hoodie-hive.jar SnowflakeSyncTool [args]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync snowflake table schema.
+ * <p>
+ * Example:
+ * snoflake_profile.properties file:
+ * URL = https://el48293.us-central1.gcp.snowflakecomputing.com:443
+ * USER = hudidemo
+ * PRIVATE_KEY_FILE = /Users/username/.ssh/rsa_key.p8
+ * ROLE = ACCOUNTADMIN
+ * WAREHOUSE = COMPUTE_WH
+ * DB = hudi
+ * SCHEMA = dwh:113
+ *
+ * command:
+ * java -cp hudi-spark-bundle_2.12-0.12.0-SNAPSHOT.jar:hudi-snowflake-bundle-0.12.0-SNAPSHOT-jar-with-dependencies.jar:gcs-connector-hadoop2-latest.jar
+ *   org.apache.hudi.snowflake.sync.SnowflakeSyncTool
+ *   --properties-file snowflake_profile.properties
+ *   --base-path gs://hudi-demo/stock_ticks_cow
+ *   --table-name stock_ticks_cow
+ *   --storage-integration hudi_demo_int
+ *   --partitioned-by "date"
+ *   --partition-extract-expr "\"date\" date as to_date(substr(metadata\$filename, 22, 10), 'YYYY-MM-DD')
+ * <p>
+ * Use these command line options, to enable along with delta streamer execution:
+ *   --enable-sync
+ *   --sync-tool-classes org.apache.hudi.snowflake.sync.SnowflakeSyncTool
+ *
+ * @Experimental
+ */
+public class SnowflakeSyncTool extends HoodieSyncTool {
+
+  private static final Logger LOG = LogManager.getLogger(SnowflakeSyncTool.class);
+  public final SnowflakeSyncConfig config;
+  public final String tableName;
+  public final String stageName;
+  public final String manifestTableName;
+  public final String versionsTableName;
+  public final String snapshotViewName;
+
+  public SnowflakeSyncTool(Properties props) {
+    super(props);
+    this.config = new SnowflakeSyncConfig(props);
+    this.tableName = config.getString(SNOWFLAKE_SYNC_TABLE_NAME);
+    stageName = tableName + "_stage";
+    manifestTableName = tableName + "_manifest";
+    versionsTableName = tableName + "_versions";
+    snapshotViewName = tableName;
+  }
+
+  public static void main(String[] args) {
+    final SnowflakeSyncConfig.SnowflakeSyncConfigParams params = new SnowflakeSyncConfig.SnowflakeSyncConfigParams();
+    JCommander cmd = JCommander.newBuilder().addObject(params).build();
+    cmd.parse(args);
+    if (params.isHelp()) {
+      cmd.usage();
+      System.exit(0);
+    }
+    new SnowflakeSyncTool(params.toProps()).syncHoodieTable();
+  }
+
+  @Override
+  public void syncHoodieTable() {
+    try (HoodieSnowflakeSyncClient snowSyncClient = new HoodieSnowflakeSyncClient(config)) {
+      switch (snowSyncClient.getTableType()) {
+        case COPY_ON_WRITE:
+          syncCoWTable(snowSyncClient);
+          break;
+        case MERGE_ON_READ:
+        default:
+          throw new UnsupportedOperationException(snowSyncClient.getTableType() + " table type is not supported yet.");
+      }
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Got runtime exception when snowflake syncing " + tableName, e);
+    }
+  }
+
+  private void syncCoWTable(HoodieSnowflakeSyncClient snowSyncClient) {

Review Comment:
   Can you please add some java docs on the steps to sync a CoW table? It will be easy to follow. 



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/HoodieSnowflakeSyncClient.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.sync.common.HoodieSyncClient;
+
+import com.snowflake.snowpark_java.Row;
+import com.snowflake.snowpark_java.Session;
+import com.snowflake.snowpark_java.types.StructType;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_PROPERTIES_FILE;
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT;
+
+/*
+ * Snowflake Hudi client to perform all the table operations on Snowflake Cloud Platform.
+ */
+public class HoodieSnowflakeSyncClient extends HoodieSyncClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieSnowflakeSyncClient.class);
+  private final SnowflakeSyncConfig config;
+  private transient Session snowflakeSession;
+
+  public HoodieSnowflakeSyncClient(final SnowflakeSyncConfig config) {
+    super(config);
+    this.config = config;
+    this.createSnowflakeConnection();
+  }
+
+  private void createSnowflakeConnection() {
+    if (snowflakeSession == 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.
+        snowflakeSession = Session.builder().configFile(config.getString(SNOWFLAKE_SYNC_PROPERTIES_FILE)).create();
+        LOG.info("Successfully established Snowflake connection.");
+      } catch (Exception e) {
+        throw new HoodieSnowflakeSyncException("Cannot create snowflake connection ", e);
+      }
+    }
+  }
+
+  public void createStage(String stageName, String basePath, String storageIntegration) {
+    try {
+      String query = "CREATE OR REPLACE STAGE " + stageName
+          + " url='" + basePath.replace("gs://", "gcs://")
+          + "' STORAGE_INTEGRATION = " + storageIntegration;
+      snowflakeSession.sql(query).show();
+      LOG.info("Manifest External table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(String stageName, String tableName) {
+    try {
+      String query = "CREATE OR REPLACE EXTERNAL TABLE " + tableName + " ("
+          + "    filename VARCHAR AS split_part(VALUE:c1, '/', -1)"
+          + "  )"
+          + "WITH LOCATION = @" + stageName + "/.hoodie/manifest/"
+          + "  FILE_FORMAT = (TYPE = CSV)"
+          + "  AUTO_REFRESH = False";
+      snowflakeSession.sql(query).show();
+      LOG.info("Manifest External table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  private void createCustomFileFormat(String fileFormatName) {
+    try {
+      String query = "CREATE OR REPLACE FILE FORMAT " + fileFormatName + " TYPE = '" + config.getString(SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT) + "';";
+      snowflakeSession.sql(query).show();
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Custom file format was not created. ", e);
+    }
+  }
+
+  private List<String> generateSchemaWithoutPartitionColumns(String stageName, String fileFormatName) {
+    try {
+      String query = "SELECT"
+          + "  generate_column_description(array_agg(object_construct(*)), 'external_table') as columns"
+          + " FROM"
+          + "  table("
+          + "    infer_schema("
+          + "      location => '@" + stageName + "',"
+          + "      file_format => '" + fileFormatName + "'"
+          + "    )"
+          + ")";
+      Optional<Row> row = snowflakeSession.sql(query).first();
+      String columns = row.get().get(0).toString();
+      if (columns.isEmpty()) {
+        throw new HoodieSnowflakeSyncException("Unable to infer the schema with the given data files.");
+      }
+      return Arrays.asList(columns.split(",", -1));
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Unable to infer the schema with the given data files. ", e);
+    }
+  }
+
+  public void createVersionsTable(String stageName, String tableName, String partitionFields, String partitionExtractExpr) {
+    try {
+      String fileFormatName = "my_custom_file_format";
+      createCustomFileFormat(fileFormatName);
+      List<String> inferredColumns = new ArrayList<String>();
+      inferredColumns.addAll(generateSchemaWithoutPartitionColumns(stageName, fileFormatName));
+      String query = "";
+      if (partitionFields.isEmpty()) {
+        query = "CREATE OR REPLACE EXTERNAL TABLE " + tableName + "("
+            + String.join(", ", inferredColumns) + ") ";
+      } else {
+        // Configuring partitioning options for partitioned table.
+        inferredColumns.addAll(Arrays.asList(partitionExtractExpr.split(",")));
+        query = "CREATE OR REPLACE EXTERNAL TABLE " + tableName + "("
+            + String.join(", ", inferredColumns)
+            + ") PARTITION BY (" + partitionFields + ") ";
+      }
+      query += " WITH LOCATION = @" + stageName
+          + "  FILE_FORMAT = (TYPE = " + config.getString(SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT) + ")"
+          + "  PATTERN = '.*[.]" + config.getString(SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT).toLowerCase() + "'"
+          + "  AUTO_REFRESH = false";
+      snowflakeSession.sql(query).show();
+      LOG.info("External versions table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("External versions table was not created ", e);
+    }
+  }
+
+  public void createSnapshotView(String viewName, String versionsTableName, String manifestTableName) {
+    try {
+      String query = "CREATE OR REPLACE VIEW " + viewName + " AS"
+          + " SELECT * FROM " + versionsTableName
+          + " WHERE \"_hoodie_file_name\" IN (SELECT filename FROM " + manifestTableName + ")";
+      snowflakeSession.sql(query).show();
+      LOG.info("View created successfully");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("View was not created ", e);
+    }
+  }
+
+  @Override
+  public void addPartitionsToTable(final String tableName, final List<String> partitionsToAdd) {
+    try {
+      String query = "ALTER EXTERNAL TABLE " + tableName + " REFRESH";
+      snowflakeSession.sql(query).show();
+      LOG.info("Table metadata refreshed successfully");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Table metadata not refreshed ", e);
+    }
+  }
+
+  @Override
+  public boolean tableExists(String tableName) {
+    try {
+      StructType schema = snowflakeSession.table(tableName).schema();
+      return true;
+    } catch (Exception e) {
+      LOG.info("Table doesn't exist " + tableName);
+      return false;
+    }
+  }
+
+  @Override
+  public Option<String> getLastCommitTimeSynced(final String tableName) {
+    // snowflake doesn't support tblproperties, so do nothing.
+    throw new UnsupportedOperationException("Not support getLastCommitTimeSynced yet.");

Review Comment:
   Can we try storing this as tags on the external table we create? 
   https://docs.snowflake.com/en/sql-reference/sql/create-tag.html
   



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/HoodieSnowflakeSyncClient.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.sync.common.HoodieSyncClient;
+
+import com.snowflake.snowpark_java.Row;
+import com.snowflake.snowpark_java.Session;
+import com.snowflake.snowpark_java.types.StructType;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_PROPERTIES_FILE;
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT;
+
+/*
+ * Snowflake Hudi client to perform all the table operations on Snowflake Cloud Platform.
+ */
+public class HoodieSnowflakeSyncClient extends HoodieSyncClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieSnowflakeSyncClient.class);
+  private final SnowflakeSyncConfig config;
+  private transient Session snowflakeSession;
+
+  public HoodieSnowflakeSyncClient(final SnowflakeSyncConfig config) {
+    super(config);
+    this.config = config;
+    this.createSnowflakeConnection();
+  }
+
+  private void createSnowflakeConnection() {
+    if (snowflakeSession == 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.
+        snowflakeSession = Session.builder().configFile(config.getString(SNOWFLAKE_SYNC_PROPERTIES_FILE)).create();
+        LOG.info("Successfully established Snowflake connection.");
+      } catch (Exception e) {
+        throw new HoodieSnowflakeSyncException("Cannot create snowflake connection ", e);
+      }
+    }
+  }
+
+  public void createStage(String stageName, String basePath, String storageIntegration) {
+    try {
+      String query = "CREATE OR REPLACE STAGE " + stageName
+          + " url='" + basePath.replace("gs://", "gcs://")
+          + "' STORAGE_INTEGRATION = " + storageIntegration;
+      snowflakeSession.sql(query).show();
+      LOG.info("Manifest External table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(String stageName, String tableName) {
+    try {
+      String query = "CREATE OR REPLACE EXTERNAL TABLE " + tableName + " ("
+          + "    filename VARCHAR AS split_part(VALUE:c1, '/', -1)"
+          + "  )"
+          + "WITH LOCATION = @" + stageName + "/.hoodie/manifest/"

Review Comment:
   I dont think we should reserve the top level name manifest under .hoodie for snowflake integration?
   /.hoodie/snowflake/manifest ?



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/HoodieSnowflakeSyncClient.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.sync.common.HoodieSyncClient;
+
+import com.snowflake.snowpark_java.Row;
+import com.snowflake.snowpark_java.Session;
+import com.snowflake.snowpark_java.types.StructType;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_PROPERTIES_FILE;
+import static org.apache.hudi.snowflake.sync.SnowflakeSyncConfig.SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT;
+
+/*
+ * Snowflake Hudi client to perform all the table operations on Snowflake Cloud Platform.
+ */
+public class HoodieSnowflakeSyncClient extends HoodieSyncClient {
+  private static final Logger LOG = LogManager.getLogger(HoodieSnowflakeSyncClient.class);
+  private final SnowflakeSyncConfig config;
+  private transient Session snowflakeSession;
+
+  public HoodieSnowflakeSyncClient(final SnowflakeSyncConfig config) {
+    super(config);
+    this.config = config;
+    this.createSnowflakeConnection();
+  }
+
+  private void createSnowflakeConnection() {
+    if (snowflakeSession == 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.
+        snowflakeSession = Session.builder().configFile(config.getString(SNOWFLAKE_SYNC_PROPERTIES_FILE)).create();
+        LOG.info("Successfully established Snowflake connection.");
+      } catch (Exception e) {
+        throw new HoodieSnowflakeSyncException("Cannot create snowflake connection ", e);
+      }
+    }
+  }
+
+  public void createStage(String stageName, String basePath, String storageIntegration) {
+    try {
+      String query = "CREATE OR REPLACE STAGE " + stageName
+          + " url='" + basePath.replace("gs://", "gcs://")
+          + "' STORAGE_INTEGRATION = " + storageIntegration;
+      snowflakeSession.sql(query).show();
+      LOG.info("Manifest External table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  public void createManifestTable(String stageName, String tableName) {
+    try {
+      String query = "CREATE OR REPLACE EXTERNAL TABLE " + tableName + " ("
+          + "    filename VARCHAR AS split_part(VALUE:c1, '/', -1)"
+          + "  )"
+          + "WITH LOCATION = @" + stageName + "/.hoodie/manifest/"
+          + "  FILE_FORMAT = (TYPE = CSV)"
+          + "  AUTO_REFRESH = False";
+      snowflakeSession.sql(query).show();
+      LOG.info("Manifest External table created.");
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Manifest External table was not created ", e);
+    }
+  }
+
+  private void createCustomFileFormat(String fileFormatName) {
+    try {
+      String query = "CREATE OR REPLACE FILE FORMAT " + fileFormatName + " TYPE = '" + config.getString(SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT) + "';";
+      snowflakeSession.sql(query).show();
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Custom file format was not created. ", e);
+    }
+  }
+
+  private List<String> generateSchemaWithoutPartitionColumns(String stageName, String fileFormatName) {
+    try {
+      String query = "SELECT"
+          + "  generate_column_description(array_agg(object_construct(*)), 'external_table') as columns"
+          + " FROM"
+          + "  table("
+          + "    infer_schema("
+          + "      location => '@" + stageName + "',"
+          + "      file_format => '" + fileFormatName + "'"
+          + "    )"
+          + ")";
+      Optional<Row> row = snowflakeSession.sql(query).first();
+      String columns = row.get().get(0).toString();
+      if (columns.isEmpty()) {
+        throw new HoodieSnowflakeSyncException("Unable to infer the schema with the given data files.");
+      }
+      return Arrays.asList(columns.split(",", -1));
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Unable to infer the schema with the given data files. ", e);
+    }
+  }
+
+  public void createVersionsTable(String stageName, String tableName, String partitionFields, String partitionExtractExpr) {
+    try {
+      String fileFormatName = "my_custom_file_format";
+      createCustomFileFormat(fileFormatName);
+      List<String> inferredColumns = new ArrayList<String>();
+      inferredColumns.addAll(generateSchemaWithoutPartitionColumns(stageName, fileFormatName));
+      String query = "";
+      if (partitionFields.isEmpty()) {
+        query = "CREATE OR REPLACE EXTERNAL TABLE " + tableName + "("
+            + String.join(", ", inferredColumns) + ") ";
+      } else {
+        // Configuring partitioning options for partitioned table.
+        inferredColumns.addAll(Arrays.asList(partitionExtractExpr.split(",")));
+        query = "CREATE OR REPLACE EXTERNAL TABLE " + tableName + "("
+            + String.join(", ", inferredColumns)
+            + ") PARTITION BY (" + partitionFields + ") ";
+      }
+      query += " WITH LOCATION = @" + stageName
+          + "  FILE_FORMAT = (TYPE = " + config.getString(SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT) + ")"
+          + "  PATTERN = '.*[.]" + config.getString(SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT).toLowerCase() + "'"
+          + "  AUTO_REFRESH = false";

Review Comment:
   Make auto-refresh option configurable?



-- 
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] nsivabalan commented on pull request #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on PR #5659:
URL: https://github.com/apache/hudi/pull/5659#issuecomment-1151724408

   @prasannarajaperumal : Can you review this when you can. 


-- 
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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   @nsivabalan - all the requested changes have been done, please review it again, thanks!


-- 
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 diff in pull request #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

Posted by GitBox <gi...@apache.org>.
vingov commented on code in PR #5659:
URL: https://github.com/apache/hudi/pull/5659#discussion_r898724244


##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncTool.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+import org.apache.hudi.sync.common.util.ManifestFileWriter;
+
+import com.beust.jcommander.JCommander;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * Tool to sync a hoodie table with a snowflake table. Either use it as an api
+ * SnowflakeSyncTool.syncHoodieTable(SnowflakeSyncConfig) or as a command line java -cp hoodie-hive.jar SnowflakeSyncTool [args]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync snowflake table schema.
+ *
+ * @Experimental
+ */
+public class SnowflakeSyncTool extends AbstractSyncTool {
+
+  private static final Logger LOG = LogManager.getLogger(SnowflakeSyncTool.class);
+
+  public final SnowflakeSyncConfig cfg;
+  public final String stageName;
+  public final String manifestTableName;
+  public final String versionsTableName;
+  public final String snapshotViewName;
+
+  public SnowflakeSyncTool(TypedProperties properties, Configuration conf, FileSystem fs) {
+    super(properties, conf, fs);
+    cfg = SnowflakeSyncConfig.fromProps(properties);
+    stageName = cfg.tableName + "_stage";
+    manifestTableName = cfg.tableName + "_manifest";
+    versionsTableName = cfg.tableName + "_versions";
+    snapshotViewName = cfg.tableName;
+  }
+
+  @Override
+  public void syncHoodieTable() {
+    try (HoodieSnowflakeSyncClient snowSyncClient = new HoodieSnowflakeSyncClient(SnowflakeSyncConfig.fromProps(props), fs)) {
+      switch (snowSyncClient.getTableType()) {
+        case COPY_ON_WRITE:
+          syncCoWTable(snowSyncClient);
+          break;
+        case MERGE_ON_READ:
+        default:
+          throw new UnsupportedOperationException(snowSyncClient.getTableType() + " table type is not supported yet.");
+      }
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Got runtime exception when snowflake syncing " + cfg.tableName, e);
+    }
+  }
+
+  private void syncCoWTable(HoodieSnowflakeSyncClient snowSyncClient) {
+    ValidationUtils.checkState(snowSyncClient.getTableType() == HoodieTableType.COPY_ON_WRITE);
+    LOG.info("Sync hoodie table " + snapshotViewName + " at base path " + snowSyncClient.getBasePath());
+
+    ManifestFileWriter manifestFileWriter = ManifestFileWriter.builder()
+        .setConf(conf)
+        .setBasePath(cfg.basePath)
+        .setUseFileListingFromMetadata(cfg.useFileListingFromMetadata)
+        .setAssumeDatePartitioning(cfg.assumeDatePartitioning)
+        .build();
+    manifestFileWriter.writeManifestFile();
+
+    snowSyncClient.createStage(stageName, cfg.basePath, cfg.storageIntegration);
+    LOG.info("External temporary stage creation complete for " + stageName);
+    snowSyncClient.createManifestTable(stageName, manifestTableName);
+    LOG.info("Manifest table creation complete for " + manifestTableName);
+    snowSyncClient.createVersionsTable(stageName, versionsTableName, cfg.partitionFields, cfg.partitionExtractExpr);
+    LOG.info("Versions table creation complete for " + versionsTableName);
+    snowSyncClient.createSnapshotView(snapshotViewName, versionsTableName, manifestTableName);

Review Comment:
   yep, created another JIRA to refactor the code: https://issues.apache.org/jira/browse/HUDI-4254



-- 
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] nsivabalan commented on pull request #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on PR #5659:
URL: https://github.com/apache/hudi/pull/5659#issuecomment-1234483477

   @vingov : may I know when are you planning to address the comments. 
   


-- 
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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845",
       "triggerID" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 223068cbc3deeac3f39b5592f3812d5b4643ad2d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845) 
   
   <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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845",
       "triggerID" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8937",
       "triggerID" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "699d684f56a84104a617d53582465ba305f8d4ab",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9330",
       "triggerID" : "699d684f56a84104a617d53582465ba305f8d4ab",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 518decf92b2b9b27f0fbe342f1dcb99159650ea4 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8937) 
   * 699d684f56a84104a617d53582465ba305f8d4ab Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9330) 
   
   <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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845",
       "triggerID" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8937",
       "triggerID" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "699d684f56a84104a617d53582465ba305f8d4ab",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9330",
       "triggerID" : "699d684f56a84104a617d53582465ba305f8d4ab",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 699d684f56a84104a617d53582465ba305f8d4ab Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9330) 
   
   <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] nikoshet commented on pull request #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

Posted by "nikoshet (via GitHub)" <gi...@apache.org>.
nikoshet commented on PR #5659:
URL: https://github.com/apache/hudi/pull/5659#issuecomment-1501704718

   hello, any news on this PR;


-- 
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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845",
       "triggerID" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8937",
       "triggerID" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "699d684f56a84104a617d53582465ba305f8d4ab",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9330",
       "triggerID" : "699d684f56a84104a617d53582465ba305f8d4ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b8c6c515dfd37f7fbce8a98468b17e6998947b7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10009",
       "triggerID" : "5b8c6c515dfd37f7fbce8a98468b17e6998947b7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5b8c6c515dfd37f7fbce8a98468b17e6998947b7 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10009) 
   
   <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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845",
       "triggerID" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8937",
       "triggerID" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "699d684f56a84104a617d53582465ba305f8d4ab",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9330",
       "triggerID" : "699d684f56a84104a617d53582465ba305f8d4ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b8c6c515dfd37f7fbce8a98468b17e6998947b7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10009",
       "triggerID" : "5b8c6c515dfd37f7fbce8a98468b17e6998947b7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 699d684f56a84104a617d53582465ba305f8d4ab Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9330) 
   * 5b8c6c515dfd37f7fbce8a98468b17e6998947b7 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10009) 
   
   <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 diff in pull request #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

Posted by GitBox <gi...@apache.org>.
vingov commented on code in PR #5659:
URL: https://github.com/apache/hudi/pull/5659#discussion_r898723534


##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncTool.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+import org.apache.hudi.sync.common.util.ManifestFileWriter;
+
+import com.beust.jcommander.JCommander;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * Tool to sync a hoodie table with a snowflake table. Either use it as an api
+ * SnowflakeSyncTool.syncHoodieTable(SnowflakeSyncConfig) or as a command line java -cp hoodie-hive.jar SnowflakeSyncTool [args]
+ * <p>

Review Comment:
   updated.



-- 
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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845",
       "triggerID" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 223068cbc3deeac3f39b5592f3812d5b4643ad2d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845) 
   * 518decf92b2b9b27f0fbe342f1dcb99159650ea4 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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b8c6c515dfd37f7fbce8a98468b17e6998947b7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10009",
       "triggerID" : "5b8c6c515dfd37f7fbce8a98468b17e6998947b7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5b8c6c515dfd37f7fbce8a98468b17e6998947b7 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10009) 
   
   <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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b8c6c515dfd37f7fbce8a98468b17e6998947b7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b8c6c515dfd37f7fbce8a98468b17e6998947b7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5b8c6c515dfd37f7fbce8a98468b17e6998947b7 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] nsivabalan commented on a diff in pull request #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on code in PR #5659:
URL: https://github.com/apache/hudi/pull/5659#discussion_r894018141


##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncConfig.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.TypedProperties;
+
+import com.beust.jcommander.Parameter;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Configs needed to sync data into Snowflake.
+ */
+public class SnowflakeSyncConfig implements Serializable {
+  public static String SNOWFLAKE_SYNC_PROPERTIES_FILE = "hoodie.snowflake.sync.properties_file";
+  public static String SNOWFLAKE_SYNC_STORAGE_INTEGRATION = "hoodie.snowflake.sync.storage_integration";
+  public static String SNOWFLAKE_SYNC_TABLE_NAME = "hoodie.snowflake.sync.table_name";
+  public static String SNOWFLAKE_SYNC_SYNC_BASE_PATH = "hoodie.snowflake.sync.base_path";
+  public static String SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT = "hoodie.snowflake.sync.base_file_format";
+  public static String SNOWFLAKE_SYNC_PARTITION_FIELDS = "hoodie.snowflake.sync.partition_fields";
+  public static String SNOWFLAKE_SYNC_PARTITION_EXTRACT_EXPRESSION = "hoodie.snowflake.sync.partition_extract_expression";
+  public static String SNOWFLAKE_SYNC_USE_FILE_LISTING_FROM_METADATA = "hoodie.snowflake.sync.use_file_listing_from_metadata";
+  public static String SNOWFLAKE_SYNC_ASSUME_DATE_PARTITIONING = "hoodie.snowflake.sync.assume_date_partitioning";
+
+  @Parameter(names = {"--properties-file"}, description = "name of the snowflake profile properties file.", required = true)
+  public String propertiesFile;
+  @Parameter(names = {"--storage-integration"}, description = "name of the storage integration in snowflake", required = true)
+  public String storageIntegration;
+  @Parameter(names = {"--table-name"}, description = "name of the target table in snowflake", required = true)
+  public String tableName;
+  @Parameter(names = {"--base-path"}, description = "Base path of the hoodie table to sync", required = true)
+  public String basePath;
+  @Parameter(names = {"--base-file-format"}, description = "Base path of the hoodie table to sync")
+  public String baseFileFormat = "PARQUET";
+  @Parameter(names = {"--partitioned-by"}, description = "Comma-delimited partition fields. Default to non-partitioned.")
+  public List<String> partitionFields = new ArrayList<>();
+  @Parameter(names = {"--partition-extract-expr"}, description = "Comma-delimited partition extract expression. Default to non-partitioned.")
+  public List<String> partitionExtractExpr = new ArrayList<>();
+  @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata")
+  public Boolean useFileListingFromMetadata = false;
+  @Parameter(names = {"--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this"

Review Comment:
   we can get rid of this param. don't think its used anymore. 



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncConfig.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.TypedProperties;
+
+import com.beust.jcommander.Parameter;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Configs needed to sync data into Snowflake.
+ */
+public class SnowflakeSyncConfig implements Serializable {
+  public static String SNOWFLAKE_SYNC_PROPERTIES_FILE = "hoodie.snowflake.sync.properties_file";
+  public static String SNOWFLAKE_SYNC_STORAGE_INTEGRATION = "hoodie.snowflake.sync.storage_integration";
+  public static String SNOWFLAKE_SYNC_TABLE_NAME = "hoodie.snowflake.sync.table_name";
+  public static String SNOWFLAKE_SYNC_SYNC_BASE_PATH = "hoodie.snowflake.sync.base_path";
+  public static String SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT = "hoodie.snowflake.sync.base_file_format";
+  public static String SNOWFLAKE_SYNC_PARTITION_FIELDS = "hoodie.snowflake.sync.partition_fields";
+  public static String SNOWFLAKE_SYNC_PARTITION_EXTRACT_EXPRESSION = "hoodie.snowflake.sync.partition_extract_expression";
+  public static String SNOWFLAKE_SYNC_USE_FILE_LISTING_FROM_METADATA = "hoodie.snowflake.sync.use_file_listing_from_metadata";

Review Comment:
   if this does not make sense for Snowflakesync, we can remove it here. when calling super constructor whereever necessary, we can use the default value. 



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncTool.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+import org.apache.hudi.sync.common.util.ManifestFileWriter;
+
+import com.beust.jcommander.JCommander;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * Tool to sync a hoodie table with a snowflake table. Either use it as an api
+ * SnowflakeSyncTool.syncHoodieTable(SnowflakeSyncConfig) or as a command line java -cp hoodie-hive.jar SnowflakeSyncTool [args]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync snowflake table schema.
+ *
+ * @Experimental
+ */
+public class SnowflakeSyncTool extends AbstractSyncTool {
+
+  private static final Logger LOG = LogManager.getLogger(SnowflakeSyncTool.class);
+
+  public final SnowflakeSyncConfig cfg;
+  public final String stageName;
+  public final String manifestTableName;
+  public final String versionsTableName;
+  public final String snapshotViewName;
+
+  public SnowflakeSyncTool(TypedProperties properties, Configuration conf, FileSystem fs) {
+    super(properties, conf, fs);
+    cfg = SnowflakeSyncConfig.fromProps(properties);
+    stageName = cfg.tableName + "_stage";

Review Comment:
   can we please create constants for "_state", "_manifest" etc. 



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncTool.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+import org.apache.hudi.sync.common.util.ManifestFileWriter;
+
+import com.beust.jcommander.JCommander;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * Tool to sync a hoodie table with a snowflake table. Either use it as an api
+ * SnowflakeSyncTool.syncHoodieTable(SnowflakeSyncConfig) or as a command line java -cp hoodie-hive.jar SnowflakeSyncTool [args]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync snowflake table schema.
+ *
+ * @Experimental
+ */
+public class SnowflakeSyncTool extends AbstractSyncTool {
+
+  private static final Logger LOG = LogManager.getLogger(SnowflakeSyncTool.class);
+
+  public final SnowflakeSyncConfig cfg;
+  public final String stageName;
+  public final String manifestTableName;
+  public final String versionsTableName;
+  public final String snapshotViewName;
+
+  public SnowflakeSyncTool(TypedProperties properties, Configuration conf, FileSystem fs) {
+    super(properties, conf, fs);
+    cfg = SnowflakeSyncConfig.fromProps(properties);
+    stageName = cfg.tableName + "_stage";
+    manifestTableName = cfg.tableName + "_manifest";
+    versionsTableName = cfg.tableName + "_versions";
+    snapshotViewName = cfg.tableName;
+  }
+
+  @Override
+  public void syncHoodieTable() {
+    try (HoodieSnowflakeSyncClient snowSyncClient = new HoodieSnowflakeSyncClient(SnowflakeSyncConfig.fromProps(props), fs)) {
+      switch (snowSyncClient.getTableType()) {
+        case COPY_ON_WRITE:
+          syncCoWTable(snowSyncClient);
+          break;
+        case MERGE_ON_READ:
+        default:
+          throw new UnsupportedOperationException(snowSyncClient.getTableType() + " table type is not supported yet.");
+      }
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Got runtime exception when snowflake syncing " + cfg.tableName, e);
+    }
+  }
+
+  private void syncCoWTable(HoodieSnowflakeSyncClient snowSyncClient) {
+    ValidationUtils.checkState(snowSyncClient.getTableType() == HoodieTableType.COPY_ON_WRITE);
+    LOG.info("Sync hoodie table " + snapshotViewName + " at base path " + snowSyncClient.getBasePath());
+
+    ManifestFileWriter manifestFileWriter = ManifestFileWriter.builder()
+        .setConf(conf)
+        .setBasePath(cfg.basePath)
+        .setUseFileListingFromMetadata(cfg.useFileListingFromMetadata)
+        .setAssumeDatePartitioning(cfg.assumeDatePartitioning)
+        .build();
+    manifestFileWriter.writeManifestFile();
+
+    snowSyncClient.createStage(stageName, cfg.basePath, cfg.storageIntegration);
+    LOG.info("External temporary stage creation complete for " + stageName);
+    snowSyncClient.createManifestTable(stageName, manifestTableName);
+    LOG.info("Manifest table creation complete for " + manifestTableName);
+    snowSyncClient.createVersionsTable(stageName, versionsTableName, cfg.partitionFields, cfg.partitionExtractExpr);
+    LOG.info("Versions table creation complete for " + versionsTableName);
+    snowSyncClient.createSnapshotView(snapshotViewName, versionsTableName, manifestTableName);

Review Comment:
   I see quite a few similarities between BQSyncTool and htis one. may be in a follow up PR, we can try to do abstract it out and re-use code if possible. 



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncTool.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+import org.apache.hudi.sync.common.util.ManifestFileWriter;
+
+import com.beust.jcommander.JCommander;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * Tool to sync a hoodie table with a snowflake table. Either use it as an api
+ * SnowflakeSyncTool.syncHoodieTable(SnowflakeSyncConfig) or as a command line java -cp hoodie-hive.jar SnowflakeSyncTool [args]
+ * <p>

Review Comment:
   Also add a line how one can enable sync w/ spark-data soruce writes or deltastreamer. 
   



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncTool.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+import org.apache.hudi.sync.common.util.ManifestFileWriter;
+
+import com.beust.jcommander.JCommander;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * Tool to sync a hoodie table with a snowflake table. Either use it as an api
+ * SnowflakeSyncTool.syncHoodieTable(SnowflakeSyncConfig) or as a command line java -cp hoodie-hive.jar SnowflakeSyncTool [args]
+ * <p>

Review Comment:
   can we add an example here please 



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncTool.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.sync.common.AbstractSyncTool;
+import org.apache.hudi.sync.common.util.ManifestFileWriter;
+
+import com.beust.jcommander.JCommander;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * Tool to sync a hoodie table with a snowflake table. Either use it as an api
+ * SnowflakeSyncTool.syncHoodieTable(SnowflakeSyncConfig) or as a command line java -cp hoodie-hive.jar SnowflakeSyncTool [args]
+ * <p>
+ * This utility will get the schema from the latest commit and will sync snowflake table schema.
+ *
+ * @Experimental
+ */
+public class SnowflakeSyncTool extends AbstractSyncTool {
+
+  private static final Logger LOG = LogManager.getLogger(SnowflakeSyncTool.class);
+
+  public final SnowflakeSyncConfig cfg;
+  public final String stageName;
+  public final String manifestTableName;
+  public final String versionsTableName;
+  public final String snapshotViewName;
+
+  public SnowflakeSyncTool(TypedProperties properties, Configuration conf, FileSystem fs) {
+    super(properties, conf, fs);
+    cfg = SnowflakeSyncConfig.fromProps(properties);
+    stageName = cfg.tableName + "_stage";
+    manifestTableName = cfg.tableName + "_manifest";
+    versionsTableName = cfg.tableName + "_versions";
+    snapshotViewName = cfg.tableName;
+  }
+
+  @Override
+  public void syncHoodieTable() {
+    try (HoodieSnowflakeSyncClient snowSyncClient = new HoodieSnowflakeSyncClient(SnowflakeSyncConfig.fromProps(props), fs)) {
+      switch (snowSyncClient.getTableType()) {
+        case COPY_ON_WRITE:
+          syncCoWTable(snowSyncClient);
+          break;
+        case MERGE_ON_READ:
+        default:
+          throw new UnsupportedOperationException(snowSyncClient.getTableType() + " table type is not supported yet.");
+      }
+    } catch (Exception e) {
+      throw new HoodieSnowflakeSyncException("Got runtime exception when snowflake syncing " + cfg.tableName, e);
+    }
+  }
+
+  private void syncCoWTable(HoodieSnowflakeSyncClient snowSyncClient) {
+    ValidationUtils.checkState(snowSyncClient.getTableType() == HoodieTableType.COPY_ON_WRITE);
+    LOG.info("Sync hoodie table " + snapshotViewName + " at base path " + snowSyncClient.getBasePath());
+
+    ManifestFileWriter manifestFileWriter = ManifestFileWriter.builder()
+        .setConf(conf)
+        .setBasePath(cfg.basePath)
+        .setUseFileListingFromMetadata(cfg.useFileListingFromMetadata)
+        .setAssumeDatePartitioning(cfg.assumeDatePartitioning)
+        .build();
+    manifestFileWriter.writeManifestFile();
+
+    snowSyncClient.createStage(stageName, cfg.basePath, cfg.storageIntegration);
+    LOG.info("External temporary stage creation complete for " + stageName);
+    snowSyncClient.createManifestTable(stageName, manifestTableName);

Review Comment:
   I am trying to understand how does snowSyncClient makes use of the manifest file that we just wrote. just the name would suffice is it? 
   in BigQuerySync, I see we are fetching sourceUri (relative path) and passing it in to the client. here I don't see a good connection. can you help me understand.



-- 
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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845",
       "triggerID" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 223068cbc3deeac3f39b5592f3812d5b4643ad2d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845) 
   
   <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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845",
       "triggerID" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8937",
       "triggerID" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "699d684f56a84104a617d53582465ba305f8d4ab",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9330",
       "triggerID" : "699d684f56a84104a617d53582465ba305f8d4ab",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b8c6c515dfd37f7fbce8a98468b17e6998947b7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b8c6c515dfd37f7fbce8a98468b17e6998947b7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 699d684f56a84104a617d53582465ba305f8d4ab Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9330) 
   * 5b8c6c515dfd37f7fbce8a98468b17e6998947b7 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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845",
       "triggerID" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8937",
       "triggerID" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "699d684f56a84104a617d53582465ba305f8d4ab",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "699d684f56a84104a617d53582465ba305f8d4ab",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 518decf92b2b9b27f0fbe342f1dcb99159650ea4 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8937) 
   * 699d684f56a84104a617d53582465ba305f8d4ab 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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845",
       "triggerID" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8937",
       "triggerID" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 223068cbc3deeac3f39b5592f3812d5b4643ad2d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845) 
   * 518decf92b2b9b27f0fbe342f1dcb99159650ea4 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8937) 
   
   <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] zhangyue19921010 commented on pull request #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

Posted by GitBox <gi...@apache.org>.
zhangyue19921010 commented on PR #5659:
URL: https://github.com/apache/hudi/pull/5659#issuecomment-1313136378

   Hey everyone. How is this amazing work going?
   
   We are planing to migrate our all historical data into Lakehouse using Hudi. 
   At the same time, we are also investigating whether our client can access these hudi data through snowflake.
   
   Looking forward for your replay! Thanks!


-- 
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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b8c6c515dfd37f7fbce8a98468b17e6998947b7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10009",
       "triggerID" : "5b8c6c515dfd37f7fbce8a98468b17e6998947b7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5b8c6c515dfd37f7fbce8a98468b17e6998947b7 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10009) 
   
   <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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8845",
       "triggerID" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8937",
       "triggerID" : "518decf92b2b9b27f0fbe342f1dcb99159650ea4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 518decf92b2b9b27f0fbe342f1dcb99159650ea4 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8937) 
   
   <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 diff in pull request #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

Posted by GitBox <gi...@apache.org>.
vingov commented on code in PR #5659:
URL: https://github.com/apache/hudi/pull/5659#discussion_r898727113


##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncConfig.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.TypedProperties;
+
+import com.beust.jcommander.Parameter;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Configs needed to sync data into Snowflake.
+ */
+public class SnowflakeSyncConfig implements Serializable {
+  public static String SNOWFLAKE_SYNC_PROPERTIES_FILE = "hoodie.snowflake.sync.properties_file";
+  public static String SNOWFLAKE_SYNC_STORAGE_INTEGRATION = "hoodie.snowflake.sync.storage_integration";
+  public static String SNOWFLAKE_SYNC_TABLE_NAME = "hoodie.snowflake.sync.table_name";
+  public static String SNOWFLAKE_SYNC_SYNC_BASE_PATH = "hoodie.snowflake.sync.base_path";
+  public static String SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT = "hoodie.snowflake.sync.base_file_format";
+  public static String SNOWFLAKE_SYNC_PARTITION_FIELDS = "hoodie.snowflake.sync.partition_fields";
+  public static String SNOWFLAKE_SYNC_PARTITION_EXTRACT_EXPRESSION = "hoodie.snowflake.sync.partition_extract_expression";
+  public static String SNOWFLAKE_SYNC_USE_FILE_LISTING_FROM_METADATA = "hoodie.snowflake.sync.use_file_listing_from_metadata";

Review Comment:
   updated.



##########
hudi-snowflake/src/main/java/org/apache/hudi/snowflake/sync/SnowflakeSyncConfig.java:
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.snowflake.sync;
+
+import org.apache.hudi.common.config.TypedProperties;
+
+import com.beust.jcommander.Parameter;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Configs needed to sync data into Snowflake.
+ */
+public class SnowflakeSyncConfig implements Serializable {
+  public static String SNOWFLAKE_SYNC_PROPERTIES_FILE = "hoodie.snowflake.sync.properties_file";
+  public static String SNOWFLAKE_SYNC_STORAGE_INTEGRATION = "hoodie.snowflake.sync.storage_integration";
+  public static String SNOWFLAKE_SYNC_TABLE_NAME = "hoodie.snowflake.sync.table_name";
+  public static String SNOWFLAKE_SYNC_SYNC_BASE_PATH = "hoodie.snowflake.sync.base_path";
+  public static String SNOWFLAKE_SYNC_SYNC_BASE_FILE_FORMAT = "hoodie.snowflake.sync.base_file_format";
+  public static String SNOWFLAKE_SYNC_PARTITION_FIELDS = "hoodie.snowflake.sync.partition_fields";
+  public static String SNOWFLAKE_SYNC_PARTITION_EXTRACT_EXPRESSION = "hoodie.snowflake.sync.partition_extract_expression";
+  public static String SNOWFLAKE_SYNC_USE_FILE_LISTING_FROM_METADATA = "hoodie.snowflake.sync.use_file_listing_from_metadata";
+  public static String SNOWFLAKE_SYNC_ASSUME_DATE_PARTITIONING = "hoodie.snowflake.sync.assume_date_partitioning";
+
+  @Parameter(names = {"--properties-file"}, description = "name of the snowflake profile properties file.", required = true)
+  public String propertiesFile;
+  @Parameter(names = {"--storage-integration"}, description = "name of the storage integration in snowflake", required = true)
+  public String storageIntegration;
+  @Parameter(names = {"--table-name"}, description = "name of the target table in snowflake", required = true)
+  public String tableName;
+  @Parameter(names = {"--base-path"}, description = "Base path of the hoodie table to sync", required = true)
+  public String basePath;
+  @Parameter(names = {"--base-file-format"}, description = "Base path of the hoodie table to sync")
+  public String baseFileFormat = "PARQUET";
+  @Parameter(names = {"--partitioned-by"}, description = "Comma-delimited partition fields. Default to non-partitioned.")
+  public List<String> partitionFields = new ArrayList<>();
+  @Parameter(names = {"--partition-extract-expr"}, description = "Comma-delimited partition extract expression. Default to non-partitioned.")
+  public List<String> partitionExtractExpr = new ArrayList<>();
+  @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata")
+  public Boolean useFileListingFromMetadata = false;
+  @Parameter(names = {"--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this"

Review Comment:
   removed.



-- 
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] nsivabalan commented on pull request #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on PR #5659:
URL: https://github.com/apache/hudi/pull/5659#issuecomment-1151723727

   @vinothchandar : You may want to review this sometime. 


-- 
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 #5659: [HUDI-4137] SnowflakeSyncTool MVP implementation to integrate with Snowflake

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "223068cbc3deeac3f39b5592f3812d5b4643ad2d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 223068cbc3deeac3f39b5592f3812d5b4643ad2d 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