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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to