1
0

[HUDI-2883] Refactor hive sync tool / config to use reflection and standardize configs (#4175)

- Refactor hive sync tool / config to use reflection and standardize configs

Co-authored-by: sivabalan <n.siva.b@gmail.com>
Co-authored-by: Rajesh Mahindra <rmahindra@Rajeshs-MacBook-Pro.local>
Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com>
This commit is contained in:
Rajesh Mahindra
2022-03-21 19:56:31 -07:00
committed by GitHub
parent 9b6e138af2
commit 5f570ea151
43 changed files with 1521 additions and 1217 deletions

View File

@@ -18,17 +18,19 @@
package org.apache.hudi.integ.testsuite.dag.nodes;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig;
import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
import org.apache.hudi.integ.testsuite.helpers.HiveServiceProvider;
import org.apache.hudi.sync.common.HoodieSyncConfig;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
/**
* A hive query node in the DAG of operations for a workflow. used to perform a hive query with given config.
@@ -46,13 +48,14 @@ public class HiveQueryNode extends DagNode<Boolean> {
public void execute(ExecutionContext executionContext, int curItrCount) throws Exception {
log.info("Executing hive query node {}", this.getName());
this.hiveServiceProvider.startLocalHiveServiceIfNeeded(executionContext.getHoodieTestSuiteWriter().getConfiguration());
HiveSyncConfig hiveSyncConfig = DataSourceUtils
.buildHiveSyncConfig(executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
.getDeltaSyncService().getDeltaSync().getProps(),
executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
.getDeltaSyncService().getDeltaSync().getCfg().targetBasePath,
executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
.getDeltaSyncService().getDeltaSync().getCfg().baseFileFormat);
TypedProperties properties = new TypedProperties();
properties.putAll(executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
.getDeltaSyncService().getDeltaSync().getProps());
properties.put(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
.getDeltaSyncService().getDeltaSync().getCfg().targetBasePath);
properties.put(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key(), executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
.getDeltaSyncService().getDeltaSync().getCfg().baseFileFormat);
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig(properties);
this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getHoodieTestSuiteWriter());
Connection con = DriverManager.getConnection(hiveSyncConfig.jdbcUrl, hiveSyncConfig.hiveUser,
hiveSyncConfig.hivePass);

View File

@@ -18,31 +18,30 @@
package org.apache.hudi.integ.testsuite.dag.nodes;
import org.apache.hudi.integ.testsuite.helpers.HiveServiceProvider;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.hive.HiveSyncTool;
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
import org.apache.hudi.sync.common.util.SyncUtilHelpers;
import org.apache.hadoop.fs.Path;
/**
* Represents a hive sync node in the DAG of operations for a workflow. Helps to sync hoodie data to hive table.
*/
public class HiveSyncNode extends DagNode<Boolean> {
private HiveServiceProvider hiveServiceProvider;
public HiveSyncNode(Config config) {
this.config = config;
this.hiveServiceProvider = new HiveServiceProvider(config);
}
@Override
public void execute(ExecutionContext executionContext, int curItrCount) throws Exception {
log.info("Executing hive sync node");
this.hiveServiceProvider.startLocalHiveServiceIfNeeded(executionContext.getHoodieTestSuiteWriter().getConfiguration());
this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getHoodieTestSuiteWriter());
this.hiveServiceProvider.stopLocalHiveServiceIfNeeded();
}
public HiveServiceProvider getHiveServiceProvider() {
return hiveServiceProvider;
SyncUtilHelpers.runHoodieMetaSync(HiveSyncTool.class.getName(), new TypedProperties(executionContext.getHoodieTestSuiteWriter().getProps()),
executionContext.getHoodieTestSuiteWriter().getConfiguration(),
new Path(executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath).getFileSystem(executionContext.getHoodieTestSuiteWriter().getConfiguration()),
executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath, executionContext.getHoodieTestSuiteWriter().getCfg().baseFileFormat);
}
}

View File

@@ -21,6 +21,9 @@ package org.apache.hudi.integ.testsuite.helpers;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hive.service.server.HiveServer2;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.hive.HiveSyncTool;
import org.apache.hudi.hive.testutils.HiveTestService;
import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter;
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
@@ -46,12 +49,17 @@ public class HiveServiceProvider {
}
public void syncToLocalHiveIfNeeded(HoodieTestSuiteWriter writer) {
HiveSyncTool hiveSyncTool;
if (this.config.isHiveLocal()) {
writer.getDeltaStreamerWrapper().getDeltaSyncService().getDeltaSync()
.syncHive(getLocalHiveServer().getHiveConf());
hiveSyncTool = new HiveSyncTool(writer.getWriteConfig().getProps(),
getLocalHiveServer().getHiveConf(),
FSUtils.getFs(writer.getWriteConfig().getBasePath(), getLocalHiveServer().getHiveConf()));
} else {
writer.getDeltaStreamerWrapper().getDeltaSyncService().getDeltaSync().syncHive();
hiveSyncTool = new HiveSyncTool(writer.getWriteConfig().getProps(),
getLocalHiveServer().getHiveConf(),
FSUtils.getFs(writer.getWriteConfig().getBasePath(), writer.getConfiguration()));
}
hiveSyncTool.syncHoodieTable();
}
public void stopLocalHiveServiceIfNeeded() throws IOException {