1
0

Refactor hoodie-common and create right abstractions for Hoodie Storage V2.0

The following is the gist of changes done

- All low-level operation of creating a commit code was in HoodieClient which made it hard to share code if there was a compaction commit.
- HoodieTableMetadata contained a mix of metadata and filtering files. (Also few operations required FileSystem to be passed in because those were called from TaskExecutors and others had FileSystem as a global variable). Since merge-on-read requires a lot of that code, but will have to change slightly on how it operates on the metadata and how it filters the files. The two set of operation are split into HoodieTableMetaClient and TableFileSystemView.
- Everything (active commits, archived commits, cleaner log, save point log and in future delta and compaction commits) in HoodieTableMetaClient is a HoodieTimeline. Timeline is a series of instants, which has an in-built concept of inflight and completed commit markers.
- A timeline can be queries for ranges, contains and also use to create new datapoint (create a new commit etc). Commit (and all the above metadata) creation/deletion is streamlined in a timeline
- Multiple timelines can be merged into a single timeline, giving us an audit timeline to whatever happened in a hoodie dataset. This also helps with #55.
- Move to java 8 and introduce java 8 succinct syntax in refactored code
This commit is contained in:
Prasanna Rajaperumal
2017-01-04 14:40:26 -08:00
parent 283269e57f
commit 8ee777a9bb
76 changed files with 3480 additions and 1932 deletions

View File

@@ -16,7 +16,7 @@
package com.uber.hoodie.cli;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@@ -26,8 +26,8 @@ public class HoodieCLI {
public static Configuration conf;
public static FileSystem fs;
public static CLIState state = CLIState.INIT;
public static HoodieTableMetadata tableMetadata;
public static HoodieTableMetadata syncTableMetadata;
public static HoodieTableMetaClient tableMetadata;
public static HoodieTableMetaClient syncTableMetadata;
public enum CLIState {
@@ -48,7 +48,7 @@ public class HoodieCLI {
}
}
public static void setTableMetadata(HoodieTableMetadata tableMetadata) {
public static void setTableMetadata(HoodieTableMetaClient tableMetadata) {
HoodieCLI.tableMetadata = tableMetadata;
}
}

View File

@@ -31,13 +31,13 @@ public class HoodiePrompt extends DefaultPromptProvider {
case INIT:
return "hoodie->";
case DATASET:
return "hoodie:" + HoodieCLI.tableMetadata.getTableName() + "->";
return "hoodie:" + HoodieCLI.tableMetadata.getTableConfig().getTableName() + "->";
case SYNC:
return "hoodie:" + HoodieCLI.tableMetadata.getTableName() + " <==> "
+ HoodieCLI.syncTableMetadata.getTableName() + "->";
return "hoodie:" + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " <==> "
+ HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->";
}
if (HoodieCLI.tableMetadata != null)
return "hoodie:" + HoodieCLI.tableMetadata.getTableName() + "->";
return "hoodie:" + HoodieCLI.tableMetadata.getTableConfig().getTableName() + "->";
return "hoodie->";
}

View File

@@ -21,9 +21,9 @@ import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.cli.utils.InputStreamConsumer;
import com.uber.hoodie.cli.utils.SparkUtil;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieCommits;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.NumericUtils;
import org.apache.spark.launcher.SparkLauncher;
@@ -38,7 +38,7 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.SortedMap;
import java.util.stream.Collectors;
@Component
public class CommitsCommand implements CommandMarker {
@@ -67,17 +67,14 @@ public class CommitsCommand implements CommandMarker {
@CliOption(key = {
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10")
final Integer limit) throws IOException {
SortedMap<String, HoodieCommitMetadata> map =
HoodieCLI.tableMetadata.getAllCommitMetadata();
int arraySize =
Math.min(limit, HoodieCLI.tableMetadata.getAllCommits().getCommitList().size());
String[][] rows = new String[arraySize][];
ArrayList<String> commitList =
new ArrayList<String>(HoodieCLI.tableMetadata.getAllCommits().getCommitList());
Collections.reverse(commitList);
for (int i = 0; i < arraySize; i++) {
String commit = commitList.get(i);
HoodieCommitMetadata commitMetadata = map.get(commit);
HoodieTimeline timeline = HoodieCLI.tableMetadata.getActiveCommitTimeline();
List<String> commits = timeline.getInstants().collect(Collectors.toList());
String[][] rows = new String[commits.size()][];
Collections.reverse(commits);
for (int i = 0; i < commits.size(); i++) {
String commit = commits.get(i);
HoodieCommitMetadata commitMetadata =
HoodieCommitMetadata.fromBytes(timeline.readInstantDetails(commit).get());
rows[i] = new String[] {commit,
NumericUtils.humanReadableByteCount(commitMetadata.fetchTotalBytesWritten()),
String.valueOf(commitMetadata.fetchTotalFilesInsert()),
@@ -95,10 +92,10 @@ public class CommitsCommand implements CommandMarker {
@CliCommand(value = "commits refresh", help = "Refresh the commits")
public String refreshCommits() throws IOException {
HoodieTableMetadata metadata =
new HoodieTableMetadata(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
HoodieTableMetaClient metadata =
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
HoodieCLI.setTableMetadata(metadata);
return "Metadata for table " + metadata.getTableName() + " refreshed.";
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
}
@CliCommand(value = "commit rollback", help = "Rollback a commit")
@@ -107,9 +104,9 @@ public class CommitsCommand implements CommandMarker {
final String commitTime,
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path")
final String sparkPropertiesPath) throws Exception {
if (!HoodieCLI.tableMetadata.getAllCommits().contains(commitTime)) {
if (!HoodieCLI.tableMetadata.getActiveCommitTimeline().containsInstant(commitTime)) {
return "Commit " + commitTime + " not found in Commits " + HoodieCLI.tableMetadata
.getAllCommits();
.getActiveCommitTimeline().getInstants().collect(Collectors.toList());
}
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(),
@@ -130,11 +127,13 @@ public class CommitsCommand implements CommandMarker {
public String showCommitPartitions(
@CliOption(key = {"commit"}, help = "Commit to show")
final String commitTime) throws Exception {
if (!HoodieCLI.tableMetadata.getAllCommits().contains(commitTime)) {
HoodieTimeline timeline = HoodieCLI.tableMetadata.getActiveCommitTimeline();
if (!timeline.containsInstant(commitTime)) {
return "Commit " + commitTime + " not found in Commits " + HoodieCLI.tableMetadata
.getAllCommits();
.getActiveCommitTimeline().getInstants().collect(Collectors.toList());
}
HoodieCommitMetadata meta = HoodieCLI.tableMetadata.getAllCommitMetadata().get(commitTime);
HoodieCommitMetadata meta =
HoodieCommitMetadata.fromBytes(timeline.readInstantDetails(commitTime).get());
List<String[]> rows = new ArrayList<String[]>();
for (Map.Entry<String, List<HoodieWriteStat>> entry : meta.getPartitionToWriteStats()
.entrySet()) {
@@ -174,11 +173,13 @@ public class CommitsCommand implements CommandMarker {
public String showCommitFiles(
@CliOption(key = {"commit"}, help = "Commit to show")
final String commitTime) throws Exception {
if (!HoodieCLI.tableMetadata.getAllCommits().contains(commitTime)) {
HoodieTimeline timeline = HoodieCLI.tableMetadata.getActiveCommitTimeline();
if (!timeline.containsInstant(commitTime)) {
return "Commit " + commitTime + " not found in Commits " + HoodieCLI.tableMetadata
.getAllCommits();
.getActiveCommitTimeline().getInstants().collect(Collectors.toList());
}
HoodieCommitMetadata meta = HoodieCLI.tableMetadata.getAllCommitMetadata().get(commitTime);
HoodieCommitMetadata meta =
HoodieCommitMetadata.fromBytes(timeline.readInstantDetails(commitTime).get());
List<String[]> rows = new ArrayList<String[]>();
for (Map.Entry<String, List<HoodieWriteStat>> entry : meta.getPartitionToWriteStats()
.entrySet()) {
@@ -206,23 +207,29 @@ public class CommitsCommand implements CommandMarker {
public String compareCommits(
@CliOption(key = {"path"}, help = "Path of the dataset to compare to")
final String path) throws Exception {
HoodieTableMetadata target = new HoodieTableMetadata(HoodieCLI.fs, path);
HoodieTableMetadata source = HoodieCLI.tableMetadata;
HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.fs, path);
HoodieTimeline targetTimeline = target.getActiveCommitTimeline();
HoodieTableMetaClient source = HoodieCLI.tableMetadata;
HoodieTimeline sourceTimeline = source.getActiveCommitTimeline();
String targetLatestCommit =
target.isCommitsEmpty() ? "0" : target.getAllCommits().lastCommit();
targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get();
String sourceLatestCommit =
source.isCommitsEmpty() ? "0" : source.getAllCommits().lastCommit();
sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get();
if (sourceLatestCommit != null && HoodieCommits
.isCommit1After(targetLatestCommit, sourceLatestCommit)) {
if (sourceLatestCommit != null && sourceTimeline
.compareInstants(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) {
// source is behind the target
List<String> commitsToCatchup = target.findCommitsSinceTs(sourceLatestCommit);
return "Source " + source.getTableName() + " is behind by " + commitsToCatchup.size()
List<String> commitsToCatchup =
targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE)
.collect(Collectors.toList());
return "Source " + source.getTableConfig().getTableName() + " is behind by " + commitsToCatchup.size()
+ " commits. Commits to catch up - " + commitsToCatchup;
} else {
List<String> commitsToCatchup = source.findCommitsSinceTs(targetLatestCommit);
return "Source " + source.getTableName() + " is ahead by " + commitsToCatchup.size()
+ " commits. Commits to catch up - " + commitsToCatchup;
List<String> commitsToCatchup =
sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE)
.collect(Collectors.toList());
return "Source " + source.getTableConfig().getTableName() + " is ahead by "
+ commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup;
}
}
@@ -235,10 +242,10 @@ public class CommitsCommand implements CommandMarker {
public String syncCommits(
@CliOption(key = {"path"}, help = "Path of the dataset to compare to")
final String path) throws Exception {
HoodieCLI.syncTableMetadata = new HoodieTableMetadata(HoodieCLI.fs, path);
HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.fs, path);
HoodieCLI.state = HoodieCLI.CLIState.SYNC;
return "Load sync state between " + HoodieCLI.tableMetadata.getTableName() + " and "
+ HoodieCLI.syncTableMetadata.getTableName();
return "Load sync state between " + HoodieCLI.tableMetadata.getTableConfig().getTableName()
+ " and " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName();
}
}

View File

@@ -17,9 +17,7 @@
package com.uber.hoodie.cli.commands;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliCommand;
import org.springframework.shell.core.annotation.CliOption;
@@ -31,12 +29,13 @@ import java.io.IOException;
public class DatasetsCommand implements CommandMarker {
@CliCommand(value = "connect", help = "Connect to a hoodie dataset")
public String connect(
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset")
final String path) throws IOException {
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset")
final String path) throws IOException {
boolean initialized = HoodieCLI.initConf();
HoodieCLI.initFS(initialized);
HoodieCLI.setTableMetadata(new HoodieTableMetadata(HoodieCLI.fs, path));
HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.fs, path));
HoodieCLI.state = HoodieCLI.CLIState.DATASET;
return "Metadata for table " + HoodieCLI.tableMetadata.getTableName() + " loaded";
return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName()
+ " loaded";
}
}

View File

@@ -19,8 +19,8 @@ package com.uber.hoodie.cli.commands;
import com.uber.hoodie.cli.utils.CommitUtil;
import com.uber.hoodie.cli.utils.HiveUtil;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.common.model.HoodieCommits;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import org.springframework.shell.core.CommandMarker;
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
import org.springframework.shell.core.annotation.CliCommand;
@@ -28,6 +28,7 @@ import org.springframework.shell.core.annotation.CliOption;
import org.springframework.stereotype.Component;
import java.util.List;
import java.util.stream.Collectors;
@Component
public class HoodieSyncCommand implements CommandMarker {
@@ -58,8 +59,10 @@ public class HoodieSyncCommand implements CommandMarker {
@CliOption(key = {
"hivePass"}, mandatory = true, unspecifiedDefaultValue = "", help = "hive password to connect to")
final String hivePass) throws Exception {
HoodieTableMetadata target = HoodieCLI.syncTableMetadata;
HoodieTableMetadata source = HoodieCLI.tableMetadata;
HoodieTableMetaClient target = HoodieCLI.syncTableMetadata;
HoodieTimeline targetTimeline = target.getActiveCommitTimeline();
HoodieTableMetaClient source = HoodieCLI.tableMetadata;
HoodieTimeline sourceTimeline = source.getActiveCommitTimeline();
long sourceCount = 0;
long targetCount = 0;
if ("complete".equals(mode)) {
@@ -71,33 +74,39 @@ public class HoodieSyncCommand implements CommandMarker {
}
String targetLatestCommit =
target.isCommitsEmpty() ? "0" : target.getAllCommits().lastCommit();
targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get();
String sourceLatestCommit =
source.isCommitsEmpty() ? "0" : source.getAllCommits().lastCommit();
sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get();
if (sourceLatestCommit != null && HoodieCommits
.isCommit1After(targetLatestCommit, sourceLatestCommit)) {
if (sourceLatestCommit != null && sourceTimeline
.compareInstants(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) {
// source is behind the target
List<String> commitsToCatchup = target.findCommitsSinceTs(sourceLatestCommit);
List<String> commitsToCatchup =
targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE)
.collect(Collectors.toList());
if (commitsToCatchup.isEmpty()) {
return "Count difference now is (count(" + target.getTableName() + ") - count("
+ source.getTableName() + ") == " + (targetCount - sourceCount);
return "Count difference now is (count(" + target.getTableConfig().getTableName()
+ ") - count(" + source.getTableConfig().getTableName() + ") == " + (targetCount
- sourceCount);
} else {
long newInserts = CommitUtil.countNewRecords(target, commitsToCatchup);
return "Count difference now is (count(" + target.getTableName() + ") - count("
+ source.getTableName() + ") == " + (targetCount - sourceCount)
+ ". Catch up count is " + newInserts;
return "Count difference now is (count(" + target.getTableConfig().getTableName()
+ ") - count(" + source.getTableConfig().getTableName() + ") == " + (targetCount
- sourceCount) + ". Catch up count is " + newInserts;
}
} else {
List<String> commitsToCatchup = source.findCommitsSinceTs(targetLatestCommit);
List<String> commitsToCatchup =
sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE)
.collect(Collectors.toList());
if (commitsToCatchup.isEmpty()) {
return "Count difference now is (count(" + source.getTableName() + ") - count("
+ target.getTableName() + ") == " + (sourceCount - targetCount);
return "Count difference now is (count(" + source.getTableConfig().getTableName()
+ ") - count(" + target.getTableConfig().getTableName() + ") == " + (sourceCount
- targetCount);
} else {
long newInserts = CommitUtil.countNewRecords(source, commitsToCatchup);
return "Count difference now is (count(" + source.getTableName() + ") - count("
+ target.getTableName() + ") == " + (sourceCount - targetCount)
+ ". Catch up count is " + newInserts;
return "Count difference now is (count(" + source.getTableConfig().getTableName()
+ ") - count(" + target.getTableConfig().getTableName() + ") == " + (sourceCount
- targetCount) + ". Catch up count is " + newInserts;
}
}

View File

@@ -23,6 +23,7 @@ import com.codahale.metrics.UniformReservoir;
import com.uber.hoodie.cli.HoodieCLI;
import com.uber.hoodie.cli.HoodiePrintHelper;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.NumericUtils;
@@ -38,7 +39,7 @@ import org.springframework.stereotype.Component;
import java.io.IOException;
import java.text.DecimalFormat;
import java.util.HashMap;
import java.util.Map;
import java.util.stream.Collectors;
@Component
public class StatsCommand implements CommandMarker {
@@ -52,22 +53,25 @@ public class StatsCommand implements CommandMarker {
long totalRecordsUpserted = 0;
long totalRecordsWritten = 0;
String[][] rows = new String[HoodieCLI.tableMetadata.getAllCommitMetadata().size() + 1][];
HoodieTimeline timeline = HoodieCLI.tableMetadata.getActiveCommitTimeline();
String[][] rows = new String[new Long(timeline.getTotalInstants()).intValue() + 1][];
int i = 0;
DecimalFormat df = new DecimalFormat("#.00");
for (Map.Entry<String, HoodieCommitMetadata> commit : HoodieCLI.tableMetadata
.getAllCommitMetadata().entrySet()) {
for (String commitTime : timeline.getInstants().collect(
Collectors.toList())) {
String waf = "0";
if (commit.getValue().fetchTotalUpdateRecordsWritten() > 0) {
HoodieCommitMetadata commit = HoodieCommitMetadata.fromBytes(timeline.readInstantDetails(commitTime).get());
if (commit.fetchTotalUpdateRecordsWritten() > 0) {
waf = df.format(
(float) commit.getValue().fetchTotalRecordsWritten() / commit.getValue()
(float) commit.fetchTotalRecordsWritten() / commit
.fetchTotalUpdateRecordsWritten());
}
rows[i++] = new String[] {commit.getKey(),
String.valueOf(commit.getValue().fetchTotalUpdateRecordsWritten()),
String.valueOf(commit.getValue().fetchTotalRecordsWritten()), waf};
totalRecordsUpserted += commit.getValue().fetchTotalUpdateRecordsWritten();
totalRecordsWritten += commit.getValue().fetchTotalRecordsWritten();
rows[i++] = new String[] {commitTime,
String.valueOf(commit.fetchTotalUpdateRecordsWritten()),
String.valueOf(commit.fetchTotalRecordsWritten()), waf};
totalRecordsUpserted += commit.fetchTotalUpdateRecordsWritten();
totalRecordsWritten += commit.fetchTotalRecordsWritten();
}
String waf = "0";
if (totalRecordsUpserted > 0) {

View File

@@ -17,20 +17,20 @@
package com.uber.hoodie.cli.utils;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import java.io.IOException;
import java.util.List;
import java.util.SortedMap;
import java.util.TreeMap;
public class CommitUtil {
public static long countNewRecords(HoodieTableMetadata target, List<String> commitsToCatchup)
public static long countNewRecords(HoodieTableMetaClient target, List<String> commitsToCatchup)
throws IOException {
long totalNew = 0;
SortedMap<String, HoodieCommitMetadata> meta = target.getAllCommitMetadata();
HoodieTimeline timeline = target.getActiveCommitTimeline();
timeline = timeline.reload();
for(String commit:commitsToCatchup) {
HoodieCommitMetadata c = meta.get(commit);
HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes(timeline.readInstantDetails(commit).get());
totalNew += c.fetchTotalRecordsWritten() - c.fetchTotalUpdateRecordsWritten();
}
return totalNew;

View File

@@ -16,7 +16,7 @@
package com.uber.hoodie.cli.utils;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import org.apache.commons.dbcp.BasicDataSource;
import org.joda.time.DateTime;
@@ -53,7 +53,7 @@ public class HiveUtil {
return ds;
}
public static long countRecords(String jdbcUrl, HoodieTableMetadata source, String dbName, String user, String pass) throws SQLException {
public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String dbName, String user, String pass) throws SQLException {
Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass);
ResultSet rs = null;
Statement stmt = conn.createStatement();
@@ -62,13 +62,13 @@ public class HiveUtil {
stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat" );
stmt.execute("set hive.stats.autogather=false" );
rs = stmt.executeQuery(
"select count(`_hoodie_commit_time`) as cnt from " + dbName + "." + source
"select count(`_hoodie_commit_time`) as cnt from " + dbName + "." + source.getTableConfig()
.getTableName());
long count = -1;
if(rs.next()) {
count = rs.getLong("cnt");
}
System.out.println("Total records in " + source.getTableName() + " is " + count);
System.out.println("Total records in " + source.getTableConfig().getTableName() + " is " + count);
return count;
} finally {
if (rs != null) {
@@ -80,7 +80,7 @@ public class HiveUtil {
}
}
public static long countRecords(String jdbcUrl, HoodieTableMetadata source, String srcDb,
public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb,
int partitions, String user, String pass) throws SQLException {
DateTime dateTime = DateTime.now();
String endDateStr =
@@ -94,7 +94,7 @@ public class HiveUtil {
return countRecords(jdbcUrl, source, srcDb, startDateStr, endDateStr, user, pass);
}
private static long countRecords(String jdbcUrl, HoodieTableMetadata source, String srcDb, String startDateStr,
private static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb, String startDateStr,
String endDateStr, String user, String pass) throws SQLException {
Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass);
ResultSet rs = null;
@@ -104,7 +104,7 @@ public class HiveUtil {
stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat");
stmt.execute("set hive.stats.autogather=false");
rs = stmt.executeQuery(
"select count(`_hoodie_commit_time`) as cnt from " + srcDb + "." + source
"select count(`_hoodie_commit_time`) as cnt from " + srcDb + "." + source.getTableConfig()
.getTableName() + " where datestr>'" + startDateStr + "' and datestr<='"
+ endDateStr + "'");
if(rs.next()) {