-
Notifications
You must be signed in to change notification settings - Fork 2.5k
feat(utilities): add external HudiHiveSyncJob for on-demand Hive sync #18204
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
65329f7
1a66b68
8b2d2e1
4a4a709
f250ae0
d8cbd8d
dc49e60
57e2be4
6f73fc8
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
|
nsivabalan marked this conversation as resolved.
nsivabalan marked this conversation as resolved.
|
| Original file line number | Diff line number | Diff line change | ||||
|---|---|---|---|---|---|---|
| @@ -0,0 +1,145 @@ | ||||||
| /* | ||||||
| * 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.utilities; | ||||||
|
|
||||||
| import org.apache.hudi.common.config.TypedProperties; | ||||||
| import org.apache.hudi.common.util.HoodieTimer; | ||||||
| import org.apache.hudi.common.util.StringUtils; | ||||||
| import org.apache.hudi.exception.HoodieException; | ||||||
| import org.apache.hudi.hive.HiveSyncTool; | ||||||
|
|
||||||
| import com.beust.jcommander.JCommander; | ||||||
| import com.beust.jcommander.Parameter; | ||||||
| import org.apache.hadoop.conf.Configuration; | ||||||
| import org.apache.hadoop.hive.conf.HiveConf; | ||||||
| import org.apache.spark.api.java.JavaSparkContext; | ||||||
| import org.slf4j.Logger; | ||||||
| import org.slf4j.LoggerFactory; | ||||||
|
|
||||||
| import java.io.IOException; | ||||||
| import java.io.Serializable; | ||||||
| import java.util.ArrayList; | ||||||
| import java.util.List; | ||||||
|
|
||||||
| import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT; | ||||||
| import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; | ||||||
|
|
||||||
|
nsivabalan marked this conversation as resolved.
|
||||||
| /** | ||||||
| * Utility job for running Hive sync on-demand for Hudi tables. | ||||||
| * <p> | ||||||
| * This tool allows you to synchronize Hudi table metadata with Hive metastore | ||||||
| * independently from ingestion workflows, useful for backfills, manual data | ||||||
| * corrections, or quick metadata reconciliation. | ||||||
| * <p> | ||||||
| * Example usage: | ||||||
| * <pre> | ||||||
| * spark-submit \ | ||||||
| * --class org.apache.hudi.utilities.HudiHiveSyncJob \ | ||||||
| * hudi-utilities.jar \ | ||||||
| * --base-path /path/to/hudi/table \ | ||||||
| * --base-file-format PARQUET \ | ||||||
| * --props-file-path /path/to/hive-sync.properties \ | ||||||
| * --hoodie-conf hoodie.datasource.hive_sync.database=my_db \ | ||||||
| * --hoodie-conf hoodie.datasource.hive_sync.table=my_table | ||||||
| * </pre> | ||||||
| */ | ||||||
| public class HudiHiveSyncJob { | ||||||
|
|
||||||
| private static final Logger LOG = LoggerFactory.getLogger(HudiHiveSyncJob.class); | ||||||
|
|
||||||
| private final Config cfg; | ||||||
| private final Configuration hadoopConf; | ||||||
| private final TypedProperties props; | ||||||
|
|
||||||
| public HudiHiveSyncJob(JavaSparkContext jsc, Config cfg) { | ||||||
| this.cfg = cfg; | ||||||
| this.hadoopConf = jsc.hadoopConfiguration(); | ||||||
| this.props = UtilHelpers.buildProperties(hadoopConf, cfg.propsFilePath, cfg.configs); | ||||||
| } | ||||||
|
|
||||||
| public static void main(String[] args) throws IOException { | ||||||
| final Config cfg = new Config(); | ||||||
| new JCommander(cfg, null, args); | ||||||
| LOG.info("Cfg received: {}", cfg); | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sensitive values are being logged. Line [79] and Line [101] can expose credentials because 🔐 Proposed fix (redact or avoid secret-bearing logs)- LOG.info("Cfg received: {}", cfg);
+ LOG.info("Cfg received: basePath={}, baseFileFormat={}, propsFilePath={}, configsCount={}",
+ cfg.basePath, cfg.baseFileFormat, cfg.propsFilePath, cfg.configs.size());
@@
- LOG.info("HiveSyncConfig props used to sync data {}", props);
+ LOG.info("HiveSyncConfig prepared for sync ({} entries)", props.size());Also applies to: 101-101, 136-142 — CodeRabbit (original) (source:comment#3075836235)
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Avoid logging full config/properties with credentials. Both log lines can expose secrets passed via 🔐 Suggested redaction-focused change- LOG.info("Cfg received: {}", cfg);
+ LOG.info("Cfg received: basePath={}, baseFileFormat={}, propsFilePath={}, hoodieConfCount={}",
+ cfg.basePath, cfg.baseFileFormat, cfg.propsFilePath, cfg.configs.size());
...
- LOG.info("HiveSyncConfig props used to sync data {}", props);
+ LOG.info("HiveSyncConfig prepared: basePath={}, baseFileFormat={}, propCount={}",
+ cfg.basePath, cfg.baseFileFormat, props.size());Also applies to: 101-101 🤖 Prompt for AI Agents— CodeRabbit (original) (source:comment#3111728505) |
||||||
| JavaSparkContext jsc; | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 🤖 The condition here is inverted. if (StringUtils.isNullOrEmpty(cfg.sparkMaster)) {
jsc = UtilHelpers.buildSparkContext("HudiHiveSyncJob", "local[2]", true);
} else {
jsc = UtilHelpers.buildSparkContext("HudiHiveSyncJob", cfg.sparkMaster, true);
}- Generated by an AI agent and may contain mistakes. Please verify any suggestions before applying. |
||||||
| if (StringUtils.isNullOrEmpty(cfg.sparkMaster)) { | ||||||
| jsc = UtilHelpers.buildSparkContext("HudiHiveSyncJob", "local[2]", true); | ||||||
| } else { | ||||||
| jsc = UtilHelpers.buildSparkContext("HudiHiveSyncJob", cfg.sparkMaster, true); | ||||||
| } | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 🧩 Analysis chain🏁 Script executed: #!/bin/bash
# Verify actual Spark context builder semantics and conventions in sibling utilities jobs.
set -euo pipefail
echo "== Locate UtilHelpers =="
fd -i "UtilHelpers.java"
echo
echo "== Inspect buildSparkContext declarations/implementations =="
rg -n -C3 --type=java '\bbuildSparkContext\s*\('
echo
echo "== Compare spark-master fallback patterns in utilities jobs =="
rg -n -C3 --type=java 'sparkMaster|local\[2\]|buildSparkContext\("Hudi'Repository: yihua/hudi Length of output: 50367 🏁 Script executed: cat -n hudi-utilities/src/main/java/org/apache/hudi/utilities/HudiHiveSyncJob.java | head -150Repository: yihua/hudi Length of output: 6583 Fix Lines 81–85 hard-code 🤖 Prompt for AI Agents— CodeRabbit (original) (source:comment#3075836239)
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 🧩 Analysis chain🏁 Script executed: #!/bin/bash
set -euo pipefail
# 1) Locate UtilHelpers implementation(s)
fd -i UtilHelpers.java
# 2) Inspect buildSparkContext overloads and master resolution logic
rg -n --type=java -C6 '\bbuildSparkContext\s*\(' $(fd -i UtilHelpers.java)
rg -n --type=java -C8 'spark\.master|setMaster|getMaster|defaultMaster|master' $(fd -i UtilHelpers.java)
# 3) Confirm current usage in HudiHiveSyncJob
rg -n --type=java -C4 'sparkMaster|local\[2\]|buildSparkContext' hudi-utilities/src/main/java/org/apache/hudi/utilities/HudiHiveSyncJob.javaRepository: hudi-agent/hudi Length of output: 11688 Parameter description needs clarification about spark.master precedence. The current parameter description ("inherits from your environment") could mislead users about the fallback behavior. The code correctly uses Same concern applies to lines 125–128. 🤖 Prompt for AI Agents— CodeRabbit (original) (source:comment#3111728516) |
||||||
| try { | ||||||
| new HudiHiveSyncJob(jsc, cfg).run(); | ||||||
| } finally { | ||||||
| jsc.stop(); | ||||||
| } | ||||||
| } | ||||||
|
|
||||||
| public void run() throws IOException { | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
— Greptile (original) (source:comment#3075833850) |
||||||
| LOG.info("Starting hive sync for {}", cfg.basePath); | ||||||
| HoodieTimer timer = HoodieTimer.start(); | ||||||
| HiveSyncTool syncTool = null; | ||||||
| try { | ||||||
| props.put(META_SYNC_BASE_PATH.key(), cfg.basePath); | ||||||
| props.put(META_SYNC_BASE_FILE_FORMAT.key(), cfg.baseFileFormat); | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
// In the constructor, after building props:
this.props.put(META_SYNC_BASE_PATH.key(), cfg.basePath);
this.props.put(META_SYNC_BASE_FILE_FORMAT.key(), cfg.baseFileFormat);— Greptile (original) (source:comment#3075834235) |
||||||
|
|
||||||
| LOG.info("HiveSyncConfig props used to sync data {}", props); | ||||||
| syncTool = new HiveSyncTool(props, new HiveConf(hadoopConf, HiveConf.class)); | ||||||
| syncTool.syncHoodieTable(); | ||||||
| } catch (Exception e) { | ||||||
| LOG.error("Exception in running hive-sync", e); | ||||||
| throw new HoodieException("Hive sync failed", e); | ||||||
| } finally { | ||||||
| if (syncTool != null) { | ||||||
| syncTool.close(); | ||||||
| } | ||||||
| LOG.info("Hive-sync duration in ms {}", timer.endTimer()); | ||||||
|
nsivabalan marked this conversation as resolved.
|
||||||
| } | ||||||
| } | ||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
If Since public void run() {
LOG.info("Starting hive sync for {}", cfg.basePath);
HoodieTimer timer = HoodieTimer.start();
props.put(META_SYNC_BASE_PATH.key(), cfg.basePath);
props.put(META_SYNC_BASE_FILE_FORMAT.key(), cfg.baseFileFormat);
LOG.info("HiveSyncConfig props used to sync data {}", props);
try (HiveSyncTool syncTool = new HiveSyncTool(props, new HiveConf(hadoopConf, HiveConf.class))) {
syncTool.syncHoodieTable();
} catch (Exception e) {
LOG.error("Exception in running hive-sync", e);
throw new HoodieException("Hive sync failed", e);
} finally {
LOG.info("Hive-sync duration in ms {}", timer.endTimer());
}
}— Greptile (original) (source:comment#3075833789) |
||||||
|
|
||||||
| public static class Config implements Serializable { | ||||||
| @Parameter(names = {"--base-path", "-sp"}, description = "Base path for the table", required = true) | ||||||
| public String basePath = null; | ||||||
|
|
||||||
| @Parameter(names = {"--base-file-format", "-bff"}, description = "Base file format of the dataset") | ||||||
| public String baseFileFormat = "PARQUET"; | ||||||
|
|
||||||
| @Parameter(names = {"--props-file-path"}, description = "Path to properties file on localfs or dfs.") | ||||||
| public String propsFilePath = null; | ||||||
|
|
||||||
| @Parameter(names = {"--spark-master"}, | ||||||
| description = "spark master to use, if not defined inherits from your environment taking into " | ||||||
| + "account Spark Configuration priority rules (e.g. not using spark-submit command).") | ||||||
| public String sparkMaster = ""; | ||||||
|
|
||||||
| @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file " | ||||||
| + "(using the CLI parameter \"--props\") can also be passed command line using this parameter. This can be repeated", | ||||||
| splitter = IdentitySplitter.class) | ||||||
| public List<String> configs = new ArrayList<>(); | ||||||
|
|
||||||
| @Override | ||||||
| public String toString() { | ||||||
| return "Config{" | ||||||
| + "basePath='" + basePath + '\'' | ||||||
| + ", baseFileFormat='" + baseFileFormat + '\'' | ||||||
| + ", propsFilePath='" + propsFilePath + '\'' | ||||||
| + ", configs=" + configs | ||||||
| + '}'; | ||||||
| } | ||||||
| } | ||||||
| } | ||||||
| Original file line number | Diff line number | Diff line change | ||||||||
|---|---|---|---|---|---|---|---|---|---|---|
| @@ -0,0 +1,165 @@ | ||||||||||
| /* | ||||||||||
| * 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.utilities; | ||||||||||
|
|
||||||||||
| import org.apache.hudi.common.util.Option; | ||||||||||
| import org.apache.hudi.common.config.TypedProperties; | ||||||||||
| import org.apache.hudi.common.table.HoodieTableMetaClient; | ||||||||||
| import org.apache.hudi.hive.HiveSyncConfig; | ||||||||||
| import org.apache.hudi.hive.HoodieHiveSyncClient; | ||||||||||
| import org.apache.hudi.hive.testutils.HiveTestUtil; | ||||||||||
|
|
||||||||||
| import org.apache.spark.sql.Dataset; | ||||||||||
| import org.apache.spark.sql.Row; | ||||||||||
| import org.apache.spark.sql.RowFactory; | ||||||||||
| import org.apache.spark.sql.SparkSession; | ||||||||||
| import org.apache.spark.sql.types.DataTypes; | ||||||||||
| import org.apache.spark.sql.types.StructType; | ||||||||||
| import org.apache.spark.api.java.JavaSparkContext; | ||||||||||
| import org.junit.jupiter.api.AfterEach; | ||||||||||
| import org.junit.jupiter.api.BeforeEach; | ||||||||||
| import org.junit.jupiter.api.Disabled; | ||||||||||
| import org.junit.jupiter.api.Test; | ||||||||||
| import org.junit.jupiter.api.io.TempDir; | ||||||||||
| import org.junit.jupiter.api.AfterAll; | ||||||||||
|
|
||||||||||
| import java.nio.file.Path; | ||||||||||
| import java.nio.file.Files; | ||||||||||
| import java.util.Arrays; | ||||||||||
| import java.util.Locale; | ||||||||||
| import java.util.UUID; | ||||||||||
|
|
||||||||||
| import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS; | ||||||||||
| import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_MODE; | ||||||||||
| import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; | ||||||||||
| import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER; | ||||||||||
| import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; | ||||||||||
| import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; | ||||||||||
| import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; | ||||||||||
| import static org.junit.jupiter.api.Assertions.assertFalse; | ||||||||||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||||||||||
| import static org.mockito.Mockito.mock; | ||||||||||
|
|
||||||||||
| /** | ||||||||||
| * Test cases for {@link HudiHiveSyncJob}. | ||||||||||
| */ | ||||||||||
| @Disabled("Hive set up in CI is failing") | ||||||||||
| public class TestHudiHiveSyncJob { | ||||||||||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Feature test is fully disabled, so new behavior is unguarded in CI. Please avoid shipping the new sync path with its only validation class-level disabled. Prefer running it in an integration/flaky profile (or env-gated assumption) rather than disabling the class entirely. ✅ Minimal direction to keep coverage without blocking default CI-@Disabled("Hive set up in CI is failing")
+// Prefer tagging/routing this test to an integration profile instead of disabling it.
+@org.junit.jupiter.api.Tag("integration")
public class TestHudiHiveSyncJob {📝 Committable suggestion
Suggested change
🤖 Prompt for AI Agents— CodeRabbit (original) (source:comment#3111728523) |
||||||||||
|
|
||||||||||
| @TempDir | ||||||||||
| Path tempDir; | ||||||||||
|
|
||||||||||
| @BeforeEach | ||||||||||
| void setUp() throws Exception { | ||||||||||
| HiveTestUtil.setUp(Option.empty(), true); | ||||||||||
| } | ||||||||||
|
|
||||||||||
| @AfterEach | ||||||||||
| void cleanUp() { | ||||||||||
| try { | ||||||||||
| HiveTestUtil.clear(); | ||||||||||
| } catch (Throwable t) { | ||||||||||
| // no-op for cleanup failures in tests | ||||||||||
| } | ||||||||||
| } | ||||||||||
|
|
||||||||||
| @AfterAll | ||||||||||
| static void cleanUpClass() { | ||||||||||
| HiveTestUtil.shutdown(); | ||||||||||
| } | ||||||||||
|
|
||||||||||
| @Test | ||||||||||
| void testRunRegistersUnregisteredHudiDatasetInMetastore() throws Exception { | ||||||||||
| String tableName = "hive_sync_job_" + UUID.randomUUID().toString().replace("-", ""); | ||||||||||
| String basePath = Files.createDirectory(tempDir.resolve("hudi-table")).toUri().toString(); | ||||||||||
| String databaseName = "default"; | ||||||||||
| HudiHiveSyncJob.Config cfg = new HudiHiveSyncJob.Config(); | ||||||||||
| cfg.basePath = basePath; | ||||||||||
| cfg.baseFileFormat = "PARQUET"; | ||||||||||
| cfg.configs.add("hoodie.datasource.hive_sync.database=" + databaseName); | ||||||||||
| cfg.configs.add("hoodie.datasource.hive_sync.table=" + tableName); | ||||||||||
| cfg.configs.add("hoodie.datasource.meta.sync.database=" + databaseName); | ||||||||||
| cfg.configs.add("hoodie.datasource.meta.sync.table=" + tableName); | ||||||||||
| cfg.configs.add(HIVE_SYNC_MODE.key() + "=jdbc"); | ||||||||||
| cfg.configs.add(HIVE_URL.key() + "=" + HiveTestUtil.hiveSyncProps.getString(HIVE_URL.key())); | ||||||||||
| cfg.configs.add(HIVE_USER.key() + "=" + HiveTestUtil.hiveSyncProps.getString(HIVE_USER.key())); | ||||||||||
| cfg.configs.add(HIVE_PASS.key() + "=" + HiveTestUtil.hiveSyncProps.getString(HIVE_PASS.key())); | ||||||||||
|
|
||||||||||
| JavaSparkContext jsc = null; | ||||||||||
| SparkSession spark = null; | ||||||||||
| try { | ||||||||||
| jsc = UtilHelpers.buildSparkContext("test-hudi-hive-sync-job", "local[2]", false); | ||||||||||
| spark = SparkSession.builder().sparkContext(jsc.sc()).getOrCreate(); | ||||||||||
|
|
||||||||||
| StructType schema = new StructType() | ||||||||||
| .add("id", DataTypes.StringType, false) | ||||||||||
| .add("name", DataTypes.StringType, true) | ||||||||||
| .add("ts", DataTypes.LongType, false); | ||||||||||
| Dataset<Row> source = spark.createDataFrame( | ||||||||||
| Arrays.asList( | ||||||||||
| RowFactory.create("1", "a1", 1000L), | ||||||||||
| RowFactory.create("2", "a2", 1001L)), | ||||||||||
| schema); | ||||||||||
|
|
||||||||||
| // Write Hudi dataset by path only: this should create commits but not register a metastore table. | ||||||||||
| source.write().format("hudi") | ||||||||||
| .option("hoodie.table.name", tableName) | ||||||||||
| .option("hoodie.datasource.write.recordkey.field", "id") | ||||||||||
| .option("hoodie.datasource.write.precombine.field", "ts") | ||||||||||
| .option("hoodie.datasource.write.keygenerator.class", "org.apache.hudi.keygen.NonpartitionedKeyGenerator") | ||||||||||
| .option("hoodie.datasource.write.partitionpath.field", "") | ||||||||||
| .mode("overwrite") | ||||||||||
| .save(basePath); | ||||||||||
|
|
||||||||||
| assertFalse(tableExists(databaseName, tableName, basePath, spark)); | ||||||||||
|
|
||||||||||
| new HudiHiveSyncJob(jsc, cfg).run(); | ||||||||||
|
|
||||||||||
| assertTrue(tableExists(databaseName, tableName, basePath, spark)); | ||||||||||
| } finally { | ||||||||||
| if (spark != null) { | ||||||||||
| spark.close(); | ||||||||||
| } | ||||||||||
| if (jsc != null) { | ||||||||||
| jsc.stop(); | ||||||||||
| } | ||||||||||
| } | ||||||||||
| } | ||||||||||
|
|
||||||||||
| private boolean tableExists(String dbName, String tableName, String basePath, SparkSession spark) { | ||||||||||
| String catalogImpl = spark.conf().get("spark.sql.catalogImplementation", "in-memory") | ||||||||||
| .toLowerCase(Locale.ROOT); | ||||||||||
| if ("hive".equals(catalogImpl)) { | ||||||||||
| return spark.catalog().tableExists(dbName, tableName); | ||||||||||
| } | ||||||||||
| return tableExistsInMetastore(dbName, tableName, basePath); | ||||||||||
| } | ||||||||||
|
|
||||||||||
| private boolean tableExistsInMetastore(String dbName, String tableName, String basePath) { | ||||||||||
| TypedProperties props = TypedProperties.copy(HiveTestUtil.hiveSyncProps); | ||||||||||
| props.setProperty(META_SYNC_DATABASE_NAME.key(), dbName); | ||||||||||
| props.setProperty(META_SYNC_TABLE_NAME.key(), tableName); | ||||||||||
| props.setProperty(META_SYNC_BASE_PATH.key(), basePath); | ||||||||||
| try (HoodieHiveSyncClient client = new HoodieHiveSyncClient( | ||||||||||
| new HiveSyncConfig(props, HiveTestUtil.getHiveConf()), | ||||||||||
| mock(HoodieTableMetaClient.class))) { | ||||||||||
| return client.tableExists(tableName); | ||||||||||
| } | ||||||||||
| } | ||||||||||
| } | ||||||||||
Uh oh!
There was an error while loading. Please reload this page.