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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
