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:
@@ -61,14 +61,6 @@
|
||||
</pluginManagement>
|
||||
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
<configuration>
|
||||
<source>1.5</source>
|
||||
<target>1.5</target>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-dependency-plugin</artifactId>
|
||||
@@ -203,6 +195,7 @@
|
||||
<artifactId>joda-time</artifactId>
|
||||
<version>2.9.6</version>
|
||||
</dependency>
|
||||
|
||||
</dependencies>
|
||||
|
||||
</project>
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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->";
|
||||
}
|
||||
|
||||
|
||||
@@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
@@ -35,8 +33,9 @@ public class DatasetsCommand implements CommandMarker {
|
||||
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";
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -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;
|
||||
|
||||
@@ -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()) {
|
||||
|
||||
@@ -16,7 +16,11 @@
|
||||
|
||||
package com.uber.hoodie.cli
|
||||
|
||||
import com.uber.hoodie.common.model.{HoodieRecord, HoodieTableMetadata}
|
||||
import java.util.stream.Collectors
|
||||
|
||||
import com.uber.hoodie.common.model.{HoodieDataFile, HoodieRecord}
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView
|
||||
import com.uber.hoodie.common.util.FSUtils
|
||||
import com.uber.hoodie.exception.HoodieException
|
||||
import org.apache.hadoop.fs.{FileSystem, FileUtil, Path}
|
||||
@@ -70,9 +74,12 @@ class DedupeSparkJob (basePath: String,
|
||||
val tmpTableName = s"htbl_${System.currentTimeMillis()}"
|
||||
val dedupeTblName = s"${tmpTableName}_dupeKeys"
|
||||
|
||||
val metadata = new HoodieTableMetadata(fs, basePath)
|
||||
val metadata = new HoodieTableMetaClient(fs, basePath)
|
||||
val fsView = new ReadOptimizedTableView(fs, metadata)
|
||||
|
||||
val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"${basePath}/${duplicatedPartitionPath}"))
|
||||
val filteredStatuses = metadata.getLatestVersions(allFiles).map(f => f.getPath.toString);
|
||||
val latestFiles:java.util.List[HoodieDataFile] = fsView.streamLatestVersions(allFiles).collect(Collectors.toList[HoodieDataFile]())
|
||||
val filteredStatuses = latestFiles.map(f => f.getPath)
|
||||
LOG.info(s" List of files under partition: ${} => ${filteredStatuses.mkString(" ")}")
|
||||
|
||||
val df = sqlContext.parquetFile(filteredStatuses:_*)
|
||||
@@ -118,9 +125,13 @@ class DedupeSparkJob (basePath: String,
|
||||
|
||||
|
||||
def fixDuplicates(dryRun: Boolean = true) = {
|
||||
val metadata = new HoodieTableMetadata(fs, basePath)
|
||||
val metadata = new HoodieTableMetaClient(fs, basePath)
|
||||
val fsView = new ReadOptimizedTableView(fs, metadata)
|
||||
|
||||
val allFiles = fs.listStatus(new Path(s"${basePath}/${duplicatedPartitionPath}"))
|
||||
val fileNameToPathMap = metadata.getLatestVersions(allFiles).map(f => (FSUtils.getFileId(f.getPath.getName), f.getPath)).toMap;
|
||||
val latestFiles:java.util.List[HoodieDataFile] = fsView.streamLatestVersions(allFiles).collect(Collectors.toList[HoodieDataFile]())
|
||||
|
||||
val fileNameToPathMap = latestFiles.map(f => (f.getFileId, new Path(f.getPath))).toMap
|
||||
val dupeFixPlan = planDuplicateFix()
|
||||
|
||||
// 1. Copy all latest files into the temp fix path
|
||||
|
||||
@@ -28,8 +28,8 @@
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.codehaus.mojo</groupId>
|
||||
<artifactId>cobertura-maven-plugin</artifactId>
|
||||
<groupId>org.jacoco</groupId>
|
||||
<artifactId>jacoco-maven-plugin</artifactId>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
|
||||
@@ -19,17 +19,18 @@ package com.uber.hoodie;
|
||||
import com.google.common.base.Optional;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieCommits;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
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.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.index.HoodieBloomIndex;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
@@ -53,6 +54,7 @@ import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
@@ -75,7 +77,8 @@ public class HoodieReadClient implements Serializable {
|
||||
* BloomIndex
|
||||
*/
|
||||
private transient final HoodieBloomIndex index;
|
||||
private HoodieTableMetadata metadata;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private transient Optional<SQLContext> sqlContextOpt;
|
||||
|
||||
|
||||
@@ -85,7 +88,8 @@ public class HoodieReadClient implements Serializable {
|
||||
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
|
||||
this.jsc = jsc;
|
||||
this.fs = FSUtils.getFs();
|
||||
this.metadata = new HoodieTableMetadata(fs, basePath);
|
||||
this.metaClient = new HoodieTableMetaClient(fs, basePath, true);
|
||||
this.commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
this.index = new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
|
||||
this.sqlContextOpt = Optional.absent();
|
||||
}
|
||||
@@ -127,7 +131,7 @@ public class HoodieReadClient implements Serializable {
|
||||
|
||||
assertSqlContext();
|
||||
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD =
|
||||
index.fetchRecordLocation(hoodieKeys, metadata);
|
||||
index.fetchRecordLocation(hoodieKeys, metaClient);
|
||||
List<String> paths = keyToFileRDD
|
||||
.filter(new Function<Tuple2<HoodieKey, Optional<String>>, Boolean>() {
|
||||
@Override
|
||||
@@ -177,17 +181,20 @@ public class HoodieReadClient implements Serializable {
|
||||
public Dataset<Row> read(String... paths) {
|
||||
assertSqlContext();
|
||||
List<String> filteredPaths = new ArrayList<>();
|
||||
TableFileSystemView fileSystemView = new ReadOptimizedTableView(fs, metaClient);
|
||||
|
||||
try {
|
||||
for (String path : paths) {
|
||||
if (!path.contains(metadata.getBasePath())) {
|
||||
if (!path.contains(metaClient.getBasePath())) {
|
||||
throw new HoodieException("Path " + path
|
||||
+ " does not seem to be a part of a Hoodie dataset at base path "
|
||||
+ metadata.getBasePath());
|
||||
+ metaClient.getBasePath());
|
||||
}
|
||||
|
||||
FileStatus[] latestFiles = metadata.getLatestVersions(fs.globStatus(new Path(path)));
|
||||
for (FileStatus file : latestFiles) {
|
||||
filteredPaths.add(file.getPath().toString());
|
||||
List<HoodieDataFile> latestFiles = fileSystemView.streamLatestVersions(fs.globStatus(new Path(path))).collect(
|
||||
Collectors.toList());
|
||||
for (HoodieDataFile file : latestFiles) {
|
||||
filteredPaths.add(file.getPath());
|
||||
}
|
||||
}
|
||||
return sqlContextOpt.get().read()
|
||||
@@ -205,15 +212,19 @@ public class HoodieReadClient implements Serializable {
|
||||
*/
|
||||
public Dataset<Row> readSince(String lastCommitTimestamp) {
|
||||
|
||||
List<String> commitsToReturn = metadata.findCommitsAfter(lastCommitTimestamp, Integer.MAX_VALUE);
|
||||
List<String> commitsToReturn =
|
||||
commitTimeline.findInstantsAfter(lastCommitTimestamp, Integer.MAX_VALUE)
|
||||
.collect(Collectors.toList());
|
||||
//TODO: we can potentially trim this down to only affected partitions, using CommitMetadata
|
||||
try {
|
||||
|
||||
// Go over the commit metadata, and obtain the new files that need to be read.
|
||||
HashMap<String, String> fileIdToFullPath = new HashMap<>();
|
||||
for (String commit: commitsToReturn) {
|
||||
HoodieCommitMetadata metadata =
|
||||
HoodieCommitMetadata.fromBytes(commitTimeline.readInstantDetails(commit).get());
|
||||
// get files from each commit, and replace any previous versions
|
||||
fileIdToFullPath.putAll(metadata.getCommitMetadata(commit).getFileIdAndFullPaths());
|
||||
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths());
|
||||
}
|
||||
|
||||
return sqlContextOpt.get().read()
|
||||
@@ -229,13 +240,13 @@ public class HoodieReadClient implements Serializable {
|
||||
*/
|
||||
public Dataset<Row> readCommit(String commitTime) {
|
||||
assertSqlContext();
|
||||
HoodieCommits commits = metadata.getAllCommits();
|
||||
if (!commits.contains(commitTime)) {
|
||||
if (!commitTimeline.containsInstant(commitTime)) {
|
||||
new HoodieException("No commit exists at " + commitTime);
|
||||
}
|
||||
|
||||
try {
|
||||
HoodieCommitMetadata commitMetdata = metadata.getCommitMetadata(commitTime);
|
||||
HoodieCommitMetadata commitMetdata =
|
||||
HoodieCommitMetadata.fromBytes(commitTimeline.readInstantDetails(commitTime).get());
|
||||
Collection<String> paths = commitMetdata.getFileIdAndFullPaths().values();
|
||||
return sqlContextOpt.get().read()
|
||||
.parquet(paths.toArray(new String[paths.size()]))
|
||||
@@ -253,7 +264,7 @@ public class HoodieReadClient implements Serializable {
|
||||
*/
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> checkExists(
|
||||
JavaRDD<HoodieKey> hoodieKeys) {
|
||||
return index.fetchRecordLocation(hoodieKeys, metadata);
|
||||
return index.fetchRecordLocation(hoodieKeys, metaClient);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -264,7 +275,7 @@ public class HoodieReadClient implements Serializable {
|
||||
* @return A subset of hoodieRecords RDD, with existing records filtered out.
|
||||
*/
|
||||
public JavaRDD<HoodieRecord> filterExists(JavaRDD<HoodieRecord> hoodieRecords) {
|
||||
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, metadata);
|
||||
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, metaClient);
|
||||
return recordsWithLocation.filter(new Function<HoodieRecord, Boolean>() {
|
||||
@Override
|
||||
public Boolean call(HoodieRecord v1) throws Exception {
|
||||
@@ -287,13 +298,13 @@ public class HoodieReadClient implements Serializable {
|
||||
* @return
|
||||
*/
|
||||
public List<String> listCommitsSince(String commitTimestamp) {
|
||||
return metadata.getAllCommits().findCommitsAfter(commitTimestamp, Integer.MAX_VALUE);
|
||||
return commitTimeline.findInstantsAfter(commitTimestamp, Integer.MAX_VALUE).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the last successful commit (a successful write operation) into a Hoodie table.
|
||||
*/
|
||||
public String latestCommit() {
|
||||
return metadata.getAllCommits().lastCommit();
|
||||
return commitTimeline.lastInstant().get();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,8 +22,9 @@ import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
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.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieCommitException;
|
||||
@@ -31,7 +32,7 @@ import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.HoodieInsertException;
|
||||
import com.uber.hoodie.exception.HoodieRollbackException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.func.BulkInsertMapFunction;
|
||||
import com.uber.hoodie.func.InsertMapFunction;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.io.HoodieCleaner;
|
||||
import com.uber.hoodie.io.HoodieCommitArchiveLog;
|
||||
@@ -39,7 +40,6 @@ import com.uber.hoodie.metrics.HoodieMetrics;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import com.uber.hoodie.table.WorkloadProfile;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -67,16 +67,19 @@ import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Option;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient
|
||||
* mutations on a HDFS dataset [upsert()]
|
||||
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then
|
||||
* perform efficient mutations on a HDFS dataset [upsert()]
|
||||
*
|
||||
* Note that, at any given time, there can only be one Spark job performing
|
||||
* these operatons on a Hoodie dataset.
|
||||
*
|
||||
* Note that, at any given time, there can only be one Spark job performing these operatons on a
|
||||
* Hoodie dataset.
|
||||
*/
|
||||
public class HoodieWriteClient<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
@@ -111,7 +114,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
this.config = clientConfig;
|
||||
this.index = HoodieIndex.createIndex(config, jsc);
|
||||
this.metrics = new HoodieMetrics(config, config.getTableName());
|
||||
this.archiveLog = new HoodieCommitArchiveLog(clientConfig);
|
||||
this.archiveLog = new HoodieCommitArchiveLog(clientConfig, fs);
|
||||
|
||||
if (rollbackInFlight) {
|
||||
rollbackInflightCommits();
|
||||
}
|
||||
@@ -125,9 +129,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* @return A subset of hoodieRecords RDD, with existing records filtered out.
|
||||
*/
|
||||
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
|
||||
final HoodieTableMetadata metadata =
|
||||
new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
|
||||
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, metadata);
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, metaClient);
|
||||
return recordsWithLocation.filter(new Function<HoodieRecord<T>, Boolean>() {
|
||||
@Override
|
||||
public Boolean call(HoodieRecord<T> v1) throws Exception {
|
||||
@@ -140,9 +144,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* Upserts a bunch of new records into the Hoodie table, at the supplied commitTime
|
||||
*/
|
||||
public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
|
||||
final HoodieTableMetadata metadata =
|
||||
new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
|
||||
writeContext = metrics.getCommitCtx();
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
|
||||
final HoodieTable table =
|
||||
HoodieTable.getHoodieTable(metaClient, commitTime, config);
|
||||
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
@@ -151,8 +158,60 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
config.getUpsertShuffleParallelism());
|
||||
|
||||
// perform index loop up to get existing location of records
|
||||
JavaRDD<HoodieRecord<T>> taggedRecords = index.tagLocation(dedupedRecords, metadata);
|
||||
return upsertRecordsInternal(taggedRecords, commitTime, metadata, true);
|
||||
JavaRDD<HoodieRecord<T>> taggedRecords = index.tagLocation(dedupedRecords, metaClient);
|
||||
|
||||
// Cache the tagged records, so we don't end up computing both
|
||||
taggedRecords.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
||||
|
||||
|
||||
WorkloadProfile profile = null;
|
||||
if (table.isWorkloadProfileNeeded()) {
|
||||
profile = new WorkloadProfile(taggedRecords);
|
||||
logger.info("Workload profile :" + profile);
|
||||
}
|
||||
|
||||
// obtain the upsert partitioner, and the run the tagger records through that & get a partitioned RDD.
|
||||
final Partitioner upsertPartitioner = table.getUpsertPartitioner(profile);
|
||||
JavaRDD<HoodieRecord<T>> partitionedRecords = taggedRecords.mapToPair(
|
||||
new PairFunction<HoodieRecord<T>, Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>() {
|
||||
@Override
|
||||
public Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>> call(
|
||||
HoodieRecord<T> record) throws Exception {
|
||||
return new Tuple2<>(new Tuple2<>(record.getKey(),
|
||||
Option.apply(record.getCurrentLocation())), record);
|
||||
}
|
||||
}).partitionBy(upsertPartitioner).map(
|
||||
new Function<Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>, HoodieRecord<T>>() {
|
||||
@Override
|
||||
public HoodieRecord<T> call(
|
||||
Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>> tuple)
|
||||
throws Exception {
|
||||
return tuple._2();
|
||||
}
|
||||
});
|
||||
|
||||
|
||||
// Perform the actual writing.
|
||||
JavaRDD<WriteStatus> upsertStatusRDD = partitionedRecords.mapPartitionsWithIndex(
|
||||
new Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>>() {
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> call(Integer partition,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
return table.handleUpsertPartition(partition, recordItr, upsertPartitioner);
|
||||
}
|
||||
}, true).flatMap(new FlatMapFunction<List<WriteStatus>, WriteStatus>() {
|
||||
@Override
|
||||
public Iterable<WriteStatus> call(List<WriteStatus> writeStatuses)
|
||||
throws Exception {
|
||||
return writeStatuses;
|
||||
}
|
||||
});
|
||||
|
||||
// Update the index back.
|
||||
JavaRDD<WriteStatus> resultRDD = index.updateLocation(upsertStatusRDD, metaClient);
|
||||
resultRDD = resultRDD.persist(config.getWriteStatusStorageLevel());
|
||||
commitOnAutoCommit(commitTime, resultRDD);
|
||||
return resultRDD;
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieUpsertException) {
|
||||
throw (HoodieUpsertException) e;
|
||||
@@ -161,36 +220,6 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Inserts the given HoodieRecords, into the table. This API is intended to be used for normal
|
||||
* writes.
|
||||
*
|
||||
* This implementation skips the index check and is able to leverage benefits such as
|
||||
* small file handling/blocking alignment, as with upsert(), by profiling the workload
|
||||
*
|
||||
* @param records HoodieRecords to insert
|
||||
* @param commitTime Commit Time handle
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
|
||||
final HoodieTableMetadata metadata =
|
||||
new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
|
||||
writeContext = metrics.getCommitCtx();
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords =
|
||||
combineOnCondition(config.shouldCombineBeforeInsert(), records,
|
||||
config.getInsertShuffleParallelism());
|
||||
|
||||
return upsertRecordsInternal(dedupedRecords, commitTime, metadata, false);
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieInsertException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieInsertException("Failed to insert for commit time " + commitTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void commitOnAutoCommit(String commitTime, JavaRDD<WriteStatus> resultRDD) {
|
||||
if(config.shouldAutoCommit()) {
|
||||
logger.info("Auto commit enabled: Committing " + commitTime);
|
||||
@@ -211,105 +240,20 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
return records;
|
||||
}
|
||||
|
||||
private JavaRDD<HoodieRecord<T>> partition(JavaRDD<HoodieRecord<T>> dedupedRecords, Partitioner partitioner) {
|
||||
return dedupedRecords.mapToPair(
|
||||
new PairFunction<HoodieRecord<T>, Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>() {
|
||||
@Override
|
||||
public Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>> call(
|
||||
HoodieRecord<T> record) throws Exception {
|
||||
return new Tuple2<>(new Tuple2<>(record.getKey(),
|
||||
Option.apply(record.getCurrentLocation())), record);
|
||||
}
|
||||
}).partitionBy(partitioner).map(
|
||||
new Function<Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>, HoodieRecord<T>>() {
|
||||
@Override
|
||||
public HoodieRecord<T> call(
|
||||
Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>> tuple)
|
||||
throws Exception {
|
||||
return tuple._2();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, WorkloadProfile profile) {
|
||||
if (isUpsert) {
|
||||
return table.getUpsertPartitioner(profile);
|
||||
} else {
|
||||
return table.getInsertPartitioner(profile);
|
||||
}
|
||||
}
|
||||
|
||||
private JavaRDD<WriteStatus> updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTableMetadata metadata,
|
||||
String commitTime) {
|
||||
// Update the index back
|
||||
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, metadata);
|
||||
// Trigger the insert and collect statuses
|
||||
statuses = statuses.persist(config.getWriteStatusStorageLevel());
|
||||
commitOnAutoCommit(commitTime, statuses);
|
||||
return statuses;
|
||||
}
|
||||
|
||||
private JavaRDD<WriteStatus> upsertRecordsInternal(JavaRDD<HoodieRecord<T>> preppedRecords,
|
||||
String commitTime,
|
||||
HoodieTableMetadata metadata,
|
||||
final boolean isUpsert) {
|
||||
|
||||
final HoodieTable table =
|
||||
HoodieTable.getHoodieTable(metadata.getTableType(), commitTime, config, metadata);
|
||||
|
||||
// Cache the tagged records, so we don't end up computing both
|
||||
preppedRecords.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
||||
|
||||
WorkloadProfile profile = null;
|
||||
if (table.isWorkloadProfileNeeded()) {
|
||||
profile = new WorkloadProfile(preppedRecords);
|
||||
logger.info("Workload profile :" + profile);
|
||||
}
|
||||
|
||||
// partition using the insert partitioner
|
||||
final Partitioner partitioner = getPartitioner(table, isUpsert, profile);
|
||||
JavaRDD<HoodieRecord<T>> partitionedRecords = partition(preppedRecords, partitioner);
|
||||
JavaRDD<WriteStatus> writeStatusRDD = partitionedRecords.mapPartitionsWithIndex(
|
||||
new Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>>() {
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> call(Integer partition,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
if (isUpsert) {
|
||||
return table.handleUpsertPartition(partition, recordItr, partitioner);
|
||||
} else {
|
||||
return table.handleInsertPartition(partition, recordItr, partitioner);
|
||||
}
|
||||
}
|
||||
}, true).flatMap(new FlatMapFunction<List<WriteStatus>, WriteStatus>() {
|
||||
@Override
|
||||
public Iterator<WriteStatus> call(List<WriteStatus> writeStatuses)
|
||||
throws Exception {
|
||||
return writeStatuses.iterator();
|
||||
}
|
||||
});
|
||||
|
||||
return updateIndexAndCommitIfNeeded(writeStatusRDD, metadata, commitTime);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk
|
||||
* loads into a Hoodie table for the very first time (e.g: converting an existing dataset to
|
||||
* Hoodie).
|
||||
*
|
||||
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and
|
||||
* attempts to control the numbers of files with less memory compared to the {@link
|
||||
* HoodieWriteClient#insert(JavaRDD, String)}
|
||||
* Loads the given HoodieRecords, as inserts into the table.
|
||||
* (This implementation uses sortBy and attempts to control the numbers of files with less memory)
|
||||
*
|
||||
* @param records HoodieRecords to insert
|
||||
* @param commitTime Commit Time handle
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*
|
||||
*/
|
||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
|
||||
final HoodieTableMetadata metadata =
|
||||
new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
|
||||
public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
|
||||
writeContext = metrics.getCommitCtx();
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords =
|
||||
@@ -329,21 +273,25 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
}, true, config.getInsertShuffleParallelism());
|
||||
JavaRDD<WriteStatus> writeStatusRDD = sortedRecords
|
||||
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(commitTime, config, metadata),
|
||||
.mapPartitionsWithIndex(new InsertMapFunction<T>(commitTime, config, metaClient),
|
||||
true).flatMap(new FlatMapFunction<List<WriteStatus>, WriteStatus>() {
|
||||
@Override
|
||||
public Iterator<WriteStatus> call(List<WriteStatus> writeStatuses)
|
||||
public Iterable<WriteStatus> call(List<WriteStatus> writeStatuses)
|
||||
throws Exception {
|
||||
return writeStatuses.iterator();
|
||||
return writeStatuses;
|
||||
}
|
||||
});
|
||||
|
||||
return updateIndexAndCommitIfNeeded(writeStatusRDD, metadata, commitTime);
|
||||
// Update the index back
|
||||
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, metaClient);
|
||||
// Trigger the insert and collect statuses
|
||||
statuses = statuses.persist(config.getWriteStatusStorageLevel());
|
||||
commitOnAutoCommit(commitTime, statuses);
|
||||
return statuses;
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieInsertException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime, e);
|
||||
throw new HoodieInsertException("Failed to insert for commit time " + commitTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -352,13 +300,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*/
|
||||
public boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses) {
|
||||
logger.info("Comitting " + commitTime);
|
||||
Path commitFile =
|
||||
new Path(config.getBasePath() + "/.hoodie/" + FSUtils.makeCommitFileName(commitTime));
|
||||
try {
|
||||
|
||||
if (fs.exists(commitFile)) {
|
||||
throw new HoodieCommitException("Duplicate commit found. " + commitTime);
|
||||
}
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
|
||||
List<Tuple2<String, HoodieWriteStat>> stats =
|
||||
writeStatuses.mapToPair(new PairFunction<WriteStatus, String, HoodieWriteStat>() {
|
||||
@@ -374,16 +318,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
metadata.addWriteStat(stat._1(), stat._2());
|
||||
}
|
||||
|
||||
// open a new file and write the commit metadata in
|
||||
Path inflightCommitFile = new Path(config.getBasePath() + "/.hoodie/" + FSUtils
|
||||
.makeInflightCommitFileName(commitTime));
|
||||
FSDataOutputStream fsout = fs.create(inflightCommitFile, true);
|
||||
fsout.writeBytes(new String(metadata.toJsonString().getBytes(StandardCharsets.UTF_8),
|
||||
StandardCharsets.UTF_8));
|
||||
fsout.close();
|
||||
|
||||
boolean success = fs.rename(inflightCommitFile, commitFile);
|
||||
if (success) {
|
||||
try {
|
||||
commitTimeline.saveInstantAsComplete(commitTime,
|
||||
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
// Save was a success
|
||||
// We cannot have unbounded commit files. Archive commits if we have to archive
|
||||
archiveLog.archiveIfRequired();
|
||||
// Call clean to cleanup if there is anything to cleanup after the commit,
|
||||
@@ -394,9 +332,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
metadata);
|
||||
writeContext = null;
|
||||
}
|
||||
}
|
||||
logger.info("Status of the commit " + commitTime + ": " + success);
|
||||
return success;
|
||||
logger.info("Status of the commit " + commitTime);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCommitException(
|
||||
"Failed to commit " + config.getBasePath() + " at time " + commitTime, e);
|
||||
@@ -405,51 +341,55 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
"Commit time is not of valid format.Failed to commit " + config.getBasePath()
|
||||
+ " at time " + commitTime, e);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Rollback the (inflight/committed) record changes with the given commit time. Three steps: (0)
|
||||
* Obtain the commit or rollback file (1) clean indexing data, (2) clean new generated parquet
|
||||
* files. (3) Finally delete .commit or .inflight file,
|
||||
* Rollback the (inflight/committed) record changes with the given commit time.
|
||||
* Three steps:
|
||||
* (1) Atomically unpublish this commit
|
||||
* (2) clean indexing data,
|
||||
* (3) clean new generated parquet files.
|
||||
* (4) Finally delete .commit or .inflight file,
|
||||
*/
|
||||
public boolean rollback(final String commitTime) throws HoodieRollbackException {
|
||||
|
||||
final Timer.Context context = metrics.getRollbackCtx();
|
||||
final HoodieTableMetadata metadata =
|
||||
new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
|
||||
final String metaPath = config.getBasePath() + "/" + HoodieTableMetadata.METAFOLDER_NAME;
|
||||
try {
|
||||
// 0. Obtain the commit/.inflight file, to work on
|
||||
FileStatus[] commitFiles =
|
||||
fs.globStatus(new Path(metaPath + "/" + commitTime + ".*"));
|
||||
if (commitFiles.length != 1) {
|
||||
throw new HoodieRollbackException("Expected exactly one .commit or .inflight file for commitTime: " + commitTime);
|
||||
}
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
|
||||
// we first need to unpublish the commit by making it .inflight again. (this will ensure no future queries see this data)
|
||||
Path filePath = commitFiles[0].getPath();
|
||||
if (filePath.getName().endsWith(HoodieTableMetadata.COMMIT_FILE_SUFFIX)) {
|
||||
if (metadata.findCommitsAfter(commitTime, Integer.MAX_VALUE).size() > 0) {
|
||||
try {
|
||||
if (commitTimeline.lastInstant().isPresent()
|
||||
&& commitTimeline.findInstantsAfter(commitTime, Integer.MAX_VALUE).count() > 0) {
|
||||
throw new HoodieRollbackException("Found commits after time :" + commitTime +
|
||||
", please rollback greater commits first");
|
||||
}
|
||||
Path newInflightPath = new Path(metaPath + "/" + commitTime + HoodieTableMetadata.INFLIGHT_FILE_SUFFIX);
|
||||
if (!fs.rename(filePath, newInflightPath)) {
|
||||
throw new HoodieRollbackException("Unable to rename .commit file to .inflight for commitTime:" + commitTime);
|
||||
}
|
||||
filePath = newInflightPath;
|
||||
|
||||
List<String> inflights =
|
||||
commitTimeline.getInflightInstants().collect(Collectors.toList());
|
||||
if (!inflights.isEmpty() && inflights.indexOf(commitTime) != inflights.size() - 1) {
|
||||
throw new HoodieRollbackException(
|
||||
"Found in-flight commits after time :" + commitTime +
|
||||
", please rollback greater commits first");
|
||||
}
|
||||
|
||||
// 1. Revert the index changes
|
||||
if (inflights.contains(commitTime) || (commitTimeline.lastInstant().isPresent()
|
||||
&& commitTimeline.lastInstant().get().equals(commitTime))) {
|
||||
// 1. Atomically unpublish this commit
|
||||
if(commitTimeline.containsInstant(commitTime)) {
|
||||
commitTimeline.revertInstantToInflight(commitTime);
|
||||
}
|
||||
// 2. Revert the index changes
|
||||
logger.info("Clean out index changes at time: " + commitTime);
|
||||
if (!index.rollbackCommit(commitTime)) {
|
||||
throw new HoodieRollbackException("Clean out index changes failed, for time :" + commitTime);
|
||||
throw new HoodieRollbackException(
|
||||
"Clean out index changes failed, for time :" + commitTime);
|
||||
}
|
||||
|
||||
// 2. Delete the new generated parquet files
|
||||
// 3. Delete the new generated parquet files
|
||||
logger.info("Clean out all parquet files generated at time: " + commitTime);
|
||||
final Accumulator<Integer> numFilesDeletedAccu = jsc.accumulator(0);
|
||||
jsc.parallelize(FSUtils.getAllPartitionPaths(fs, metadata.getBasePath()))
|
||||
jsc.parallelize(FSUtils.getAllPartitionPaths(fs, metaClient.getBasePath()))
|
||||
.foreach(new VoidFunction<String>() {
|
||||
@Override
|
||||
public void call(String partitionPath) throws Exception {
|
||||
@@ -473,20 +413,16 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
// 3. Clean out metadata (.commit or .tmp)
|
||||
// 4. Remove commit
|
||||
logger.info("Clean out metadata files at time: " + commitTime);
|
||||
if (!fs.delete(filePath, false)) {
|
||||
logger.error("Deleting file " + filePath + " failed.");
|
||||
throw new HoodieRollbackException("Delete file " + filePath + " failed.");
|
||||
}
|
||||
commitTimeline.removeInflightFromTimeline(commitTime);
|
||||
|
||||
if (context != null) {
|
||||
long durationInMs = metrics.getDurationInMs(context.stop());
|
||||
int numFilesDeleted = numFilesDeletedAccu.value();
|
||||
metrics.updateRollbackMetrics(durationInMs, numFilesDeleted);
|
||||
}
|
||||
|
||||
}
|
||||
return true;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRollbackException("Failed to rollback " +
|
||||
@@ -508,8 +444,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
try {
|
||||
logger.info("Cleaner started");
|
||||
final Timer.Context context = metrics.getCleanCtx();
|
||||
final HoodieTableMetadata metadata = new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
|
||||
List<String> partitionsToClean = FSUtils.getAllPartitionPaths(fs, metadata.getBasePath());
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
|
||||
List<String> partitionsToClean = FSUtils.getAllPartitionPaths(fs, metaClient.getBasePath());
|
||||
// shuffle to distribute cleaning work across partitions evenly
|
||||
Collections.shuffle(partitionsToClean);
|
||||
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config.getCleanerPolicy());
|
||||
@@ -524,7 +463,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
@Override
|
||||
public Integer call(String partitionPathToClean) throws Exception {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
HoodieCleaner cleaner = new HoodieCleaner(metadata, config, fs);
|
||||
HoodieCleaner cleaner = new HoodieCleaner(metaClient, config, fs);
|
||||
return cleaner.clean(partitionPathToClean);
|
||||
}
|
||||
}).reduce(new Function2<Integer, Integer, Integer>() {
|
||||
@@ -556,21 +495,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
public void startCommitWithTime(String commitTime) {
|
||||
logger.info("Generate a new commit time " + commitTime);
|
||||
// Create the in-flight commit file
|
||||
Path inflightCommitFilePath = new Path(
|
||||
config.getBasePath() + "/.hoodie/" + FSUtils.makeInflightCommitFileName(commitTime));
|
||||
try {
|
||||
if (fs.createNewFile(inflightCommitFilePath)) {
|
||||
logger.info("Create an inflight commit file " + inflightCommitFilePath);
|
||||
return;
|
||||
}
|
||||
throw new HoodieCommitException(
|
||||
"Failed to create the inflight commit file " + inflightCommitFilePath);
|
||||
} catch (IOException e) {
|
||||
// handled below
|
||||
throw new HoodieCommitException(
|
||||
"Failed to create the inflight commit file " + inflightCommitFilePath, e);
|
||||
}
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
commitTimeline.saveInstantAsInflight(commitTime);
|
||||
}
|
||||
|
||||
public static SparkConf registerClasses(SparkConf conf) {
|
||||
@@ -606,10 +534,16 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
/**
|
||||
* Cleanup all inflight commits
|
||||
* @throws IOException
|
||||
*/
|
||||
private void rollbackInflightCommits() {
|
||||
final HoodieTableMetadata metadata = new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
|
||||
for (String commit : metadata.getAllInflightCommits()) {
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
|
||||
List<String> commits = commitTimeline.getInflightInstants().collect(Collectors.toList());
|
||||
Collections.reverse(commits);
|
||||
for (String commit : commits) {
|
||||
rollback(commit);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -16,11 +16,11 @@
|
||||
|
||||
package com.uber.hoodie.func;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
|
||||
import java.util.Iterator;
|
||||
@@ -30,23 +30,23 @@ import java.util.List;
|
||||
/**
|
||||
* Map function that handles a sorted stream of HoodieRecords
|
||||
*/
|
||||
public class BulkInsertMapFunction<T extends HoodieRecordPayload>
|
||||
public class InsertMapFunction<T extends HoodieRecordPayload>
|
||||
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>> {
|
||||
|
||||
private String commitTime;
|
||||
private HoodieWriteConfig config;
|
||||
private HoodieTableMetadata metadata;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
|
||||
public BulkInsertMapFunction(String commitTime, HoodieWriteConfig config,
|
||||
HoodieTableMetadata metadata) {
|
||||
public InsertMapFunction(String commitTime, HoodieWriteConfig config,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
this.commitTime = commitTime;
|
||||
this.config = config;
|
||||
this.metadata = metadata;
|
||||
this.metaClient = metaClient;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> call(Integer partition, Iterator<HoodieRecord<T>> sortedRecordItr)
|
||||
throws Exception {
|
||||
return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, metadata);
|
||||
return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, metaClient);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -16,11 +16,11 @@
|
||||
|
||||
package com.uber.hoodie.func;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
|
||||
import com.uber.hoodie.io.HoodieIOHandle;
|
||||
import com.uber.hoodie.io.HoodieInsertHandle;
|
||||
@@ -40,17 +40,17 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends LazyItera
|
||||
|
||||
private final HoodieWriteConfig hoodieConfig;
|
||||
private final String commitTime;
|
||||
private final HoodieTableMetadata tableMetadata;
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
private Set<String> partitionsCleaned;
|
||||
private HoodieInsertHandle handle;
|
||||
|
||||
public LazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
|
||||
String commitTime, HoodieTableMetadata metadata) {
|
||||
String commitTime, HoodieTableMetaClient metaClient) {
|
||||
super(sortedRecordItr);
|
||||
this.partitionsCleaned = new HashSet<>();
|
||||
this.hoodieConfig = config;
|
||||
this.commitTime = commitTime;
|
||||
this.tableMetadata = metadata;
|
||||
this.metaClient = metaClient;
|
||||
}
|
||||
|
||||
@Override protected void start() {
|
||||
@@ -78,7 +78,7 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends LazyItera
|
||||
// lazily initialize the handle, for the first time
|
||||
if (handle == null) {
|
||||
handle =
|
||||
new HoodieInsertHandle(hoodieConfig, commitTime, tableMetadata,
|
||||
new HoodieInsertHandle(hoodieConfig, commitTime, metaClient,
|
||||
record.getPartitionPath());
|
||||
}
|
||||
|
||||
@@ -90,7 +90,7 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends LazyItera
|
||||
statuses.add(handle.close());
|
||||
// Need to handle the rejected record & open new handle
|
||||
handle =
|
||||
new HoodieInsertHandle(hoodieConfig, commitTime, tableMetadata,
|
||||
new HoodieInsertHandle(hoodieConfig, commitTime, metaClient,
|
||||
record.getPartitionPath());
|
||||
handle.write(record); // we should be able to write 1 record.
|
||||
break;
|
||||
|
||||
@@ -17,12 +17,13 @@
|
||||
package com.uber.hoodie.index;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
@@ -65,7 +66,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, HoodieTableMetadata metadata) {
|
||||
JavaRDD<HoodieKey> hoodieKeys, HoodieTableMetaClient metaClient) {
|
||||
throw new UnsupportedOperationException("HBase index does not implement check exist yet");
|
||||
}
|
||||
|
||||
@@ -91,10 +92,10 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
class LocationTagFunction
|
||||
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
|
||||
|
||||
private final HoodieTableMetadata metadata;
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
|
||||
LocationTagFunction(HoodieTableMetadata metadata) {
|
||||
this.metadata = metadata;
|
||||
LocationTagFunction(HoodieTableMetaClient metaClient) {
|
||||
this.metaClient = metaClient;
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -127,8 +128,9 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
String fileId =
|
||||
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
|
||||
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
// if the last commit ts for this row is less than the system commit ts
|
||||
if (!metadata.isCommitsEmpty() && metadata.isCommitTsSafe(commitTs)) {
|
||||
if (commitTimeline.hasInstants() && commitTimeline.containsInstant(commitTs)) {
|
||||
rec.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
|
||||
}
|
||||
}
|
||||
@@ -155,8 +157,8 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieTableMetadata metadata) {
|
||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(metadata), true);
|
||||
HoodieTableMetaClient metaClient) {
|
||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(metaClient), true);
|
||||
}
|
||||
|
||||
class UpdateLocationTask implements Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> {
|
||||
@@ -217,7 +219,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTableMetadata metadata) {
|
||||
HoodieTableMetaClient metaClient) {
|
||||
return writeStatusRDD.mapPartitionsWithIndex(new UpdateLocationTask(), true);
|
||||
}
|
||||
|
||||
|
||||
@@ -19,16 +19,18 @@ package com.uber.hoodie.index;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Optional;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
@@ -43,6 +45,7 @@ import org.apache.spark.api.java.function.PairFunction;
|
||||
|
||||
import scala.Tuple2;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in
|
||||
@@ -63,10 +66,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
}
|
||||
|
||||
@Override
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, final HoodieTableMetadata metadata) {
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, final HoodieTableMetaClient metaClient) {
|
||||
|
||||
// Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD
|
||||
@@ -79,7 +79,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
|
||||
lookupIndex(partitionRecordKeyPairRDD, metadata);
|
||||
lookupIndex(partitionRecordKeyPairRDD, metaClient);
|
||||
|
||||
// Cache the result, for subsequent stages.
|
||||
rowKeyFilenamePairRDD.cache();
|
||||
@@ -93,7 +93,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
}
|
||||
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetadata metadata) {
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetaClient metaClient) {
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
|
||||
hoodieKeys.mapToPair(new PairFunction<HoodieKey, String, String>() {
|
||||
@Override
|
||||
@@ -104,7 +104,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
|
||||
lookupIndex(partitionRecordKeyPairRDD, metadata);
|
||||
lookupIndex(partitionRecordKeyPairRDD, metaClient);
|
||||
|
||||
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD =
|
||||
hoodieKeys.mapToPair(new PairFunction<HoodieKey, String, HoodieKey>() {
|
||||
@@ -115,17 +115,17 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
});
|
||||
|
||||
return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).mapToPair(
|
||||
new PairFunction<Tuple2<String, Tuple2<HoodieKey, org.apache.spark.api.java.Optional<String>>>, HoodieKey, Optional<String>>() {
|
||||
new PairFunction<Tuple2<String, Tuple2<HoodieKey, Optional<String>>>, HoodieKey, Optional<String>>() {
|
||||
@Override
|
||||
public Tuple2<HoodieKey, Optional<String>> call(
|
||||
Tuple2<String, Tuple2<HoodieKey, org.apache.spark.api.java.Optional<String>>> keyPathTuple)
|
||||
Tuple2<String, Tuple2<HoodieKey, Optional<String>>> keyPathTuple)
|
||||
throws Exception {
|
||||
Optional<String> recordLocationPath;
|
||||
if (keyPathTuple._2._2.isPresent()) {
|
||||
String fileName = keyPathTuple._2._2.get();
|
||||
String partitionPath = keyPathTuple._2._1.getPartitionPath();
|
||||
recordLocationPath = Optional
|
||||
.of(new Path(new Path(metadata.getBasePath(), partitionPath), fileName)
|
||||
.of(new Path(new Path(metaClient.getBasePath(), partitionPath), fileName)
|
||||
.toUri().getPath());
|
||||
} else {
|
||||
recordLocationPath = Optional.absent();
|
||||
@@ -140,19 +140,19 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
* record keys already present and drop the record keys if not present
|
||||
*
|
||||
* @param partitionRecordKeyPairRDD
|
||||
* @param metadata
|
||||
* @param metaClient
|
||||
* @return
|
||||
*/
|
||||
private JavaPairRDD<String, String> lookupIndex(
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTableMetadata metadata) {
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTableMetaClient metaClient) {
|
||||
// Obtain records per partition, in the incoming records
|
||||
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
|
||||
Map<String, Object> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
|
||||
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
|
||||
|
||||
// Step 2: Load all involved files as <Partition, filename> pairs
|
||||
JavaPairRDD<String, String> partitionFilePairRDD =
|
||||
loadInvolvedFiles(affectedPartitionPathList, metadata);
|
||||
Map<String, Long> filesPerPartition = partitionFilePairRDD.countByKey();
|
||||
loadInvolvedFiles(affectedPartitionPathList, metaClient);
|
||||
Map<String, Object> filesPerPartition = partitionFilePairRDD.countByKey();
|
||||
|
||||
// Compute total subpartitions, to split partitions into.
|
||||
Map<String, Long> subpartitionCountMap =
|
||||
@@ -174,7 +174,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
* @param filesPerPartition
|
||||
* @return
|
||||
*/
|
||||
private Map<String, Long> computeSubPartitions(Map<String, Long> recordsPerPartition, Map<String, Long> filesPerPartition) {
|
||||
private Map<String, Long> computeSubPartitions(Map<String, Object> recordsPerPartition, Map<String, Object> filesPerPartition) {
|
||||
Map<String, Long> subpartitionCountMap = new HashMap<>();
|
||||
long totalRecords = 0;
|
||||
long totalFiles = 0;
|
||||
@@ -210,19 +210,26 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
* Load all involved files as <Partition, filename> pair RDD.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
JavaPairRDD<String, String> loadInvolvedFiles(List<String> partitions, final HoodieTableMetadata metadata) {
|
||||
JavaPairRDD<String, String> loadInvolvedFiles(List<String> partitions,
|
||||
final HoodieTableMetaClient metaClient) {
|
||||
return jsc.parallelize(partitions, Math.max(partitions.size(), 1))
|
||||
.flatMapToPair(new PairFlatMapFunction<String, String, String>() {
|
||||
@Override
|
||||
public Iterator<Tuple2<String, String>> call(String partitionPath) {
|
||||
public Iterable<Tuple2<String, String>> call(String partitionPath) {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
String latestCommitTime = metadata.getAllCommits().lastCommit();
|
||||
FileStatus[] filteredStatus = metadata.getLatestVersionInPartition(fs, partitionPath, latestCommitTime);
|
||||
TableFileSystemView view = new ReadOptimizedTableView(fs, metaClient);
|
||||
java.util.Optional<String> latestCommitTime =
|
||||
metaClient.getActiveCommitTimeline().lastInstant();
|
||||
List<Tuple2<String, String>> list = new ArrayList<>();
|
||||
for (FileStatus fileStatus : filteredStatus) {
|
||||
list.add(new Tuple2<>(partitionPath, fileStatus.getPath().getName()));
|
||||
if (latestCommitTime.isPresent()) {
|
||||
List<HoodieDataFile> filteredFiles =
|
||||
view.streamLatestVersionInPartition(partitionPath,
|
||||
latestCommitTime.get()).collect(Collectors.toList());
|
||||
for (HoodieDataFile file : filteredFiles) {
|
||||
list.add(new Tuple2<>(partitionPath, file.getFileName()));
|
||||
}
|
||||
return list.iterator();
|
||||
}
|
||||
return list;
|
||||
}
|
||||
});
|
||||
}
|
||||
@@ -261,8 +268,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
})
|
||||
.flatMapToPair(new PairFlatMapFunction<List<Tuple2<String, String>>, String, String>() {
|
||||
@Override
|
||||
public Iterator<Tuple2<String, String>> call(List<Tuple2<String, String>> exploded) throws Exception {
|
||||
return exploded.iterator();
|
||||
public Iterable<Tuple2<String, String>> call(List<Tuple2<String, String>> exploded) throws Exception {
|
||||
return exploded;
|
||||
}
|
||||
});
|
||||
|
||||
@@ -323,7 +330,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
/**
|
||||
* Find out <RowKey, filename> pair. All workload grouped by file-level.
|
||||
*
|
||||
* // Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) and then repartition such that
|
||||
* // Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) & then repartition such that
|
||||
// each RDD partition is a file, then for each file, we do (1) load bloom filter, (2) load rowKeys, (3) Tag rowKey
|
||||
// Make sure the parallelism is atleast the groupby parallelism for tagging location
|
||||
*/
|
||||
@@ -362,9 +369,9 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(config.getBasePath()), true)
|
||||
.flatMap(new FlatMapFunction<List<IndexLookupResult>, IndexLookupResult>() {
|
||||
@Override
|
||||
public Iterator<IndexLookupResult> call(List<IndexLookupResult> indexLookupResults)
|
||||
public Iterable<IndexLookupResult> call(List<IndexLookupResult> indexLookupResults)
|
||||
throws Exception {
|
||||
return indexLookupResults.iterator();
|
||||
return indexLookupResults;
|
||||
}
|
||||
}).filter(new Function<IndexLookupResult, Boolean>() {
|
||||
@Override
|
||||
@@ -373,13 +380,13 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
}
|
||||
}).flatMapToPair(new PairFlatMapFunction<IndexLookupResult, String, String>() {
|
||||
@Override
|
||||
public Iterator<Tuple2<String, String>> call(IndexLookupResult lookupResult)
|
||||
public Iterable<Tuple2<String, String>> call(IndexLookupResult lookupResult)
|
||||
throws Exception {
|
||||
List<Tuple2<String, String>> vals = new ArrayList<>();
|
||||
for (String recordKey : lookupResult.getMatchingRecordKeys()) {
|
||||
vals.add(new Tuple2<>(recordKey, lookupResult.getFileName()));
|
||||
}
|
||||
return vals.iterator();
|
||||
return vals;
|
||||
}
|
||||
});
|
||||
}
|
||||
@@ -399,9 +406,9 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
|
||||
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), so we do left outer join.
|
||||
return rowKeyRecordPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).values().map(
|
||||
new Function<Tuple2<HoodieRecord<T>, org.apache.spark.api.java.Optional<String>>, HoodieRecord<T>>() {
|
||||
new Function<Tuple2<HoodieRecord<T>, Optional<String>>, HoodieRecord<T>>() {
|
||||
@Override
|
||||
public HoodieRecord<T> call(Tuple2<HoodieRecord<T>, org.apache.spark.api.java.Optional<String>> v1) throws Exception {
|
||||
public HoodieRecord<T> call(Tuple2<HoodieRecord<T>, Optional<String>> v1) throws Exception {
|
||||
HoodieRecord<T> record = v1._1();
|
||||
if (v1._2().isPresent()) {
|
||||
String filename = v1._2().get();
|
||||
@@ -416,7 +423,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTableMetadata metadata) {
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTableMetaClient metaClient) {
|
||||
return writeStatusRDD;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -17,11 +17,12 @@
|
||||
package com.uber.hoodie.index;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
|
||||
import com.uber.hoodie.exception.HoodieIndexException;
|
||||
@@ -58,18 +59,18 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
|
||||
* value is present, it is the path component (without scheme) of the URI underlying file
|
||||
*
|
||||
* @param hoodieKeys
|
||||
* @param metadata
|
||||
* @param metaClient
|
||||
* @return
|
||||
*/
|
||||
public abstract JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetadata metadata);
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetaClient metaClient);
|
||||
|
||||
/**
|
||||
* Looks up the index and tags each incoming record with a location of a file that contains the
|
||||
* row (if it is actually present)
|
||||
*/
|
||||
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieTableMetadata metadata) throws
|
||||
HoodieTableMetaClient metaClient) throws
|
||||
HoodieIndexException;
|
||||
|
||||
/**
|
||||
@@ -78,8 +79,7 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
|
||||
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
|
||||
*/
|
||||
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTableMetadata metadata) throws
|
||||
HoodieIndexException;
|
||||
HoodieTableMetaClient metaClient) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Rollback the efffects of the commit made at commitTime.
|
||||
|
||||
@@ -17,13 +17,13 @@
|
||||
package com.uber.hoodie.index;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
@@ -55,7 +55,7 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetadata metadata) {
|
||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetaClient metaClient) {
|
||||
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
|
||||
}
|
||||
|
||||
@@ -81,13 +81,13 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
HoodieTableMetadata metadata) {
|
||||
HoodieTableMetaClient metaClient) {
|
||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTableMetadata metadata) {
|
||||
HoodieTableMetaClient metaClient) {
|
||||
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
|
||||
@Override
|
||||
public WriteStatus call(WriteStatus writeStatus) {
|
||||
|
||||
@@ -16,9 +16,12 @@
|
||||
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.common.model.HoodieCommits;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
@@ -31,7 +34,7 @@ import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
|
||||
@@ -45,26 +48,25 @@ import java.util.Map;
|
||||
*
|
||||
*/
|
||||
public class HoodieCleaner {
|
||||
private static Logger logger = LogManager.getLogger(HoodieCleaner.class);
|
||||
|
||||
public enum CleaningPolicy {
|
||||
KEEP_LATEST_FILE_VERSIONS,
|
||||
KEEP_LATEST_COMMITS
|
||||
}
|
||||
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieCleaner.class);
|
||||
|
||||
|
||||
private HoodieTableMetadata metadata;
|
||||
|
||||
private final TableFileSystemView fileSystemView;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private HoodieWriteConfig config;
|
||||
|
||||
private FileSystem fs;
|
||||
|
||||
public HoodieCleaner(HoodieTableMetadata metadata,
|
||||
public HoodieCleaner(HoodieTableMetaClient metaClient,
|
||||
HoodieWriteConfig config,
|
||||
FileSystem fs) {
|
||||
this.metadata = metadata;
|
||||
this.metaClient = metaClient;
|
||||
this.fileSystemView = new ReadOptimizedTableView(fs, metaClient);
|
||||
this.commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
this.config = config;
|
||||
this.fs = fs;
|
||||
}
|
||||
@@ -83,13 +85,13 @@ public class HoodieCleaner {
|
||||
*/
|
||||
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath) throws IOException {
|
||||
logger.info("Cleaning "+ partitionPath+", retaining latest "+ config.getCleanerFileVersionsRetained()+" file versions. ");
|
||||
Map<String, List<FileStatus>> fileVersions = metadata.getAllVersionsInPartition(fs, partitionPath);
|
||||
List<List<HoodieDataFile>> fileVersions = fileSystemView.streamEveryVersionInPartition(partitionPath).collect(
|
||||
Collectors.toList());
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
|
||||
for (String file : fileVersions.keySet()) {
|
||||
List<FileStatus> commitList = fileVersions.get(file);
|
||||
for (List<HoodieDataFile> versionsForFileId : fileVersions) {
|
||||
int keepVersions = config.getCleanerFileVersionsRetained();
|
||||
Iterator<FileStatus> commitItr = commitList.iterator();
|
||||
Iterator<HoodieDataFile> commitItr = versionsForFileId.iterator();
|
||||
while (commitItr.hasNext() && keepVersions > 0) {
|
||||
// Skip this most recent version
|
||||
commitItr.next();
|
||||
@@ -100,7 +102,7 @@ public class HoodieCleaner {
|
||||
deletePaths.add(String.format("%s/%s/%s",
|
||||
config.getBasePath(),
|
||||
partitionPath,
|
||||
commitItr.next().getPath().getName()));
|
||||
commitItr.next().getFileName()));
|
||||
}
|
||||
}
|
||||
return deletePaths;
|
||||
@@ -133,22 +135,20 @@ public class HoodieCleaner {
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
|
||||
// determine if we have enough commits, to start cleaning.
|
||||
HoodieCommits commits = metadata.getAllCommits();
|
||||
if (commits.getNumCommits() > commitsRetained) {
|
||||
if (commitTimeline.getTotalInstants() > commitsRetained) {
|
||||
String earliestCommitToRetain =
|
||||
commits.nthCommit(commits.getNumCommits() - commitsRetained);
|
||||
Map<String, List<FileStatus>> fileVersions =
|
||||
metadata.getAllVersionsInPartition(fs, partitionPath);
|
||||
for (String file : fileVersions.keySet()) {
|
||||
List<FileStatus> fileList = fileVersions.get(file);
|
||||
String lastVersion = FSUtils.getCommitTime(fileList.get(0).getPath().getName());
|
||||
commitTimeline.nthInstant(commitTimeline.getTotalInstants() - commitsRetained).get();
|
||||
List<List<HoodieDataFile>> fileVersions =
|
||||
fileSystemView.streamEveryVersionInPartition(partitionPath).collect(Collectors.toList());
|
||||
for (List<HoodieDataFile> fileList : fileVersions) {
|
||||
String lastVersion = FSUtils.getCommitTime(fileList.get(0).getFileName());
|
||||
String lastVersionBeforeEarliestCommitToRetain =
|
||||
getLatestVersionBeforeCommit(fileList, earliestCommitToRetain);
|
||||
|
||||
// Ensure there are more than 1 version of the file (we only clean old files from updates)
|
||||
// i.e always spare the last commit.
|
||||
for (FileStatus afile : fileList) {
|
||||
String fileCommitTime = FSUtils.getCommitTime(afile.getPath().getName());
|
||||
for (HoodieDataFile afile : fileList) {
|
||||
String fileCommitTime = afile.getCommitTime();
|
||||
// Dont delete the latest commit and also the last commit before the earliest commit we are retaining
|
||||
// The window of commit retain == max query run time. So a query could be running which still
|
||||
// uses this file.
|
||||
@@ -160,11 +160,12 @@ public class HoodieCleaner {
|
||||
}
|
||||
|
||||
// Always keep the last commit
|
||||
if (HoodieCommits.isCommit1After(earliestCommitToRetain, fileCommitTime)) {
|
||||
if (commitTimeline.compareInstants(earliestCommitToRetain, fileCommitTime,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// this is a commit, that should be cleaned.
|
||||
deletePaths.add(String
|
||||
.format("%s/%s/%s", config.getBasePath(), partitionPath,
|
||||
FSUtils.maskWithoutTaskPartitionId(fileCommitTime, file)));
|
||||
.format("%s/%s/%s", config.getBasePath(), partitionPath, FSUtils
|
||||
.maskWithoutTaskPartitionId(fileCommitTime, afile.getFileId())));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -176,10 +177,10 @@ public class HoodieCleaner {
|
||||
/**
|
||||
* Gets the latest version < commitTime. This version file could still be used by queries.
|
||||
*/
|
||||
private String getLatestVersionBeforeCommit(List<FileStatus> fileList, String commitTime) {
|
||||
for (FileStatus file : fileList) {
|
||||
String fileCommitTime = FSUtils.getCommitTime(file.getPath().getName());
|
||||
if (HoodieCommits.isCommit1After(commitTime, fileCommitTime)) {
|
||||
private String getLatestVersionBeforeCommit(List<HoodieDataFile> fileList, String commitTime) {
|
||||
for (HoodieDataFile file : fileList) {
|
||||
String fileCommitTime = FSUtils.getCommitTime(file.getFileName());
|
||||
if (commitTimeline.compareInstants(commitTime, fileCommitTime, HoodieTimeline.GREATER)) {
|
||||
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the one we want
|
||||
return fileCommitTime;
|
||||
}
|
||||
|
||||
@@ -16,10 +16,13 @@
|
||||
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveCommitTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieArchivedCommitTimeline;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.common.file.HoodieAppendLog;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieCommitException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -30,65 +33,70 @@ import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Log to hold older historical commits, to bound the growth of .commit files
|
||||
*/
|
||||
public class HoodieCommitArchiveLog {
|
||||
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
|
||||
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits.archived";
|
||||
|
||||
private final Path archiveFilePath;
|
||||
private final FileSystem fs;
|
||||
private final HoodieWriteConfig config;
|
||||
|
||||
public HoodieCommitArchiveLog(HoodieWriteConfig config) {
|
||||
this.archiveFilePath =
|
||||
new Path(config.getBasePath(),
|
||||
HoodieTableMetadata.METAFOLDER_NAME + "/" +HOODIE_COMMIT_ARCHIVE_LOG_FILE);
|
||||
this.fs = FSUtils.getFs();
|
||||
public HoodieCommitArchiveLog(HoodieWriteConfig config,
|
||||
FileSystem fs) {
|
||||
this.fs = fs;
|
||||
this.config = config;
|
||||
this.archiveFilePath = HoodieArchivedCommitTimeline
|
||||
.getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if commits need to be archived. If yes, archive commits.
|
||||
*/
|
||||
public boolean archiveIfRequired() {
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, config.getBasePath());
|
||||
List<String> commitsToArchive = getCommitsToArchive(metadata);
|
||||
if (!commitsToArchive.isEmpty()) {
|
||||
List<String> commitsToArchive = getCommitsToArchive().collect(Collectors.toList());
|
||||
if (commitsToArchive.iterator().hasNext()) {
|
||||
log.info("Archiving commits " + commitsToArchive);
|
||||
archive(metadata, commitsToArchive);
|
||||
return deleteCommits(metadata, commitsToArchive);
|
||||
archive(commitsToArchive);
|
||||
return deleteCommits(commitsToArchive);
|
||||
} else {
|
||||
log.info("No Commits to archive");
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
private List<String> getCommitsToArchive(HoodieTableMetadata metadata) {
|
||||
private Stream<String> getCommitsToArchive() {
|
||||
int maxCommitsToKeep = config.getMaxCommitsToKeep();
|
||||
int minCommitsToKeep = config.getMinCommitsToKeep();
|
||||
|
||||
List<String> commits = metadata.getAllCommits().getCommitList();
|
||||
List<String> commitsToArchive = new ArrayList<String>();
|
||||
if (commits.size() > maxCommitsToKeep) {
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
|
||||
if (commitTimeline.hasInstants() && commitTimeline.getTotalInstants() > maxCommitsToKeep) {
|
||||
// Actually do the commits
|
||||
commitsToArchive = commits.subList(0, commits.size() - minCommitsToKeep);
|
||||
return commitTimeline.getInstants()
|
||||
.limit(commitTimeline.getTotalInstants() - minCommitsToKeep);
|
||||
}
|
||||
return commitsToArchive;
|
||||
return Stream.empty();
|
||||
}
|
||||
|
||||
private boolean deleteCommits(HoodieTableMetadata metadata, List<String> commitsToArchive) {
|
||||
private boolean deleteCommits(List<String> commitsToArchive) {
|
||||
log.info("Deleting commits " + commitsToArchive);
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
|
||||
boolean success = true;
|
||||
for(String commitToArchive:commitsToArchive) {
|
||||
Path commitFile =
|
||||
new Path(metadata.getBasePath() + "/" +
|
||||
HoodieTableMetadata.METAFOLDER_NAME + "/" +
|
||||
FSUtils.makeCommitFileName(commitToArchive));
|
||||
Path commitFile = new Path(metaClient.getMetaPath(),
|
||||
((HoodieActiveCommitTimeline) commitTimeline)
|
||||
.getCompletedFileName(commitToArchive));
|
||||
try {
|
||||
if (fs.exists(commitFile)) {
|
||||
success &= fs.delete(commitFile, false);
|
||||
@@ -112,14 +120,19 @@ public class HoodieCommitArchiveLog {
|
||||
.compression(HoodieAppendLog.CompressionType.RECORD, new BZip2Codec()));
|
||||
}
|
||||
|
||||
private void archive(HoodieTableMetadata metadata, List<String> commits)
|
||||
throws HoodieCommitException {
|
||||
private void archive(List<String> commits) throws HoodieCommitException {
|
||||
HoodieTableMetaClient metaClient =
|
||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
|
||||
HoodieAppendLog.Writer writer = null;
|
||||
try {
|
||||
writer = openWriter();
|
||||
for (String commitTime : commits) {
|
||||
Text k = new Text(commitTime);
|
||||
Text v = new Text(metadata.getCommitMetadata(commitTime).toJsonString());
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.readInstantDetails(commitTime).get());
|
||||
Text v = new Text(commitMetadata.toJsonString());
|
||||
writer.append(k, v);
|
||||
log.info("Wrote " + k);
|
||||
}
|
||||
|
||||
@@ -16,9 +16,12 @@
|
||||
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
@@ -36,15 +39,19 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
protected final String commitTime;
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final FileSystem fs;
|
||||
protected final HoodieTableMetadata metadata;
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
protected final HoodieTimeline hoodieTimeline;
|
||||
protected final TableFileSystemView fileSystemView;
|
||||
protected final Schema schema;
|
||||
|
||||
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTableMetadata metadata) {
|
||||
HoodieTableMetaClient metaClient) {
|
||||
this.commitTime = commitTime;
|
||||
this.config = config;
|
||||
this.fs = FSUtils.getFs();
|
||||
this.metadata = metadata;
|
||||
this.metaClient = metaClient;
|
||||
this.hoodieTimeline = metaClient.getActiveCommitTimeline();
|
||||
this.fileSystemView = new ReadOptimizedTableView(fs, metaClient);
|
||||
this.schema =
|
||||
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
}
|
||||
|
||||
@@ -16,12 +16,12 @@
|
||||
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieInsertException;
|
||||
@@ -45,7 +45,7 @@ public class HoodieInsertHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
private int recordsWritten = 0;
|
||||
|
||||
public HoodieInsertHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTableMetadata metadata, String partitionPath) {
|
||||
HoodieTableMetaClient metadata, String partitionPath) {
|
||||
super(config, commitTime, metadata);
|
||||
this.status = new WriteStatus();
|
||||
status.setFileId(UUID.randomUUID().toString());
|
||||
|
||||
@@ -16,12 +16,12 @@
|
||||
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
@@ -52,10 +52,10 @@ import java.util.Iterator;
|
||||
|
||||
public HoodieUpdateHandle(HoodieWriteConfig config,
|
||||
String commitTime,
|
||||
HoodieTableMetadata metadata,
|
||||
HoodieTableMetaClient metaClient,
|
||||
Iterator<HoodieRecord<T>> recordItr,
|
||||
String fileId) {
|
||||
super(config, commitTime, metadata);
|
||||
super(config, commitTime, metaClient);
|
||||
WriteStatus writeStatus = new WriteStatus();
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
this.writeStatus = writeStatus;
|
||||
@@ -74,7 +74,9 @@ import java.util.Iterator;
|
||||
HoodieRecord<T> record = newRecordsItr.next();
|
||||
// If the first record, we need to extract some info out
|
||||
if (oldFilePath == null) {
|
||||
String latestValidFilePath = metadata.getFilenameForRecord(fs, record, fileId);
|
||||
String latestValidFilePath = fileSystemView
|
||||
.getLatestDataFilesForFileId(record.getPartitionPath(), fileId).findFirst()
|
||||
.get().getFileName();
|
||||
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
|
||||
oldFilePath = new Path(
|
||||
config.getBasePath() + "/" + record.getPartitionPath() + "/"
|
||||
@@ -102,14 +104,14 @@ import java.util.Iterator;
|
||||
}
|
||||
// Create the writer for writing the new version file
|
||||
storageWriter = HoodieStorageWriterFactory
|
||||
.getStorageWriter(commitTime, newFilePath, metadata, config, schema);
|
||||
.getStorageWriter(commitTime, newFilePath, metaClient, config, schema);
|
||||
|
||||
} catch (Exception e) {
|
||||
logger.error("Error in update task at commit " + commitTime, e);
|
||||
writeStatus.setGlobalError(e);
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
|
||||
+ commitTime + " on HDFS path " + metadata.getBasePath());
|
||||
+ commitTime + " on HDFS path " + metaClient.getBasePath());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -16,11 +16,11 @@
|
||||
|
||||
package com.uber.hoodie.io.storage;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||
import com.uber.hoodie.common.BloomFilter;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
@@ -32,7 +32,7 @@ import java.io.IOException;
|
||||
|
||||
public class HoodieStorageWriterFactory {
|
||||
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
|
||||
String commitTime, Path path, HoodieTableMetadata metadata, HoodieWriteConfig config, Schema schema)
|
||||
String commitTime, Path path, HoodieTableMetaClient metaClient, HoodieWriteConfig config, Schema schema)
|
||||
throws IOException {
|
||||
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
|
||||
// Currently only parquet is supported
|
||||
|
||||
@@ -16,6 +16,11 @@
|
||||
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
@@ -23,7 +28,6 @@ import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieInsertException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
@@ -33,7 +37,6 @@ import com.uber.hoodie.io.HoodieUpdateHandle;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -52,6 +55,7 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Option;
|
||||
import scala.Tuple2;
|
||||
@@ -133,8 +137,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
|
||||
|
||||
public HoodieCopyOnWriteTable(String commitTime, HoodieWriteConfig config, HoodieTableMetadata metadata) {
|
||||
super(commitTime, config, metadata);
|
||||
public HoodieCopyOnWriteTable(String commitTime, HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
super(commitTime, config, metaClient);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -287,24 +291,25 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
if (metadata.getAllCommits().getNumCommits() > 0) { // if we have some commits
|
||||
String latestCommitTime = metadata.getAllCommits().lastCommit();
|
||||
FileStatus[] allFiles = metadata.getLatestVersionInPartition(fs, partitionPath, latestCommitTime);
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
TableFileSystemView fileSystemView = new ReadOptimizedTableView(fs, metaClient);
|
||||
|
||||
if (allFiles != null && allFiles.length > 0) {
|
||||
for (FileStatus fileStatus : allFiles) {
|
||||
if (fileStatus.getLen() < config.getParquetSmallFileLimit()) {
|
||||
String filename = fileStatus.getPath().getName();
|
||||
if (commitTimeline.hasInstants()) { // if we have some commits
|
||||
String latestCommitTime = commitTimeline.lastInstant().get();
|
||||
List<HoodieDataFile> allFiles = fileSystemView.streamLatestVersionInPartition(partitionPath, latestCommitTime).collect(
|
||||
Collectors.toList());
|
||||
|
||||
for (HoodieDataFile file : allFiles) {
|
||||
if (file.getFileSize() < config.getParquetSmallFileLimit()) {
|
||||
String filename = file.getFileName();
|
||||
SmallFile sf = new SmallFile();
|
||||
sf.location = new HoodieRecordLocation(
|
||||
FSUtils.getCommitTime(filename),
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename),
|
||||
FSUtils.getFileId(filename));
|
||||
sf.sizeBytes = fileStatus.getLen();
|
||||
sf.sizeBytes = file.getFileSize();
|
||||
smallFileLocations.add(sf);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return smallFileLocations;
|
||||
}
|
||||
@@ -317,11 +322,15 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
*/
|
||||
private long averageBytesPerRecord() {
|
||||
long avgSize = 0L;
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
try {
|
||||
if (metadata.getAllCommits().getNumCommits() > 0) {
|
||||
String latestCommitTime = metadata.getAllCommits().lastCommit();
|
||||
HoodieCommitMetadata commitMetadata = metadata.getCommitMetadata(latestCommitTime);
|
||||
avgSize =(long) Math.ceil((1.0 * commitMetadata.fetchTotalBytesWritten())/commitMetadata.fetchTotalRecordsWritten());
|
||||
if (commitTimeline.hasInstants()) {
|
||||
String latestCommitTime = commitTimeline.lastInstant().get();
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.readInstantDetails(latestCommitTime).get());
|
||||
avgSize = (long) Math.ceil(
|
||||
(1.0 * commitMetadata.fetchTotalBytesWritten()) / commitMetadata
|
||||
.fetchTotalRecordsWritten());
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
// make this fail safe.
|
||||
@@ -389,7 +398,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String fileLoc, Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
// these are updates
|
||||
HoodieUpdateHandle upsertHandle =
|
||||
new HoodieUpdateHandle<>(config, commitTime, metadata, recordItr, fileLoc);
|
||||
new HoodieUpdateHandle<>(config, commitTime, metaClient, recordItr, fileLoc);
|
||||
if (upsertHandle.getOldFilePath() == null) {
|
||||
throw new HoodieUpsertException("Error in finding the old file path at commit " +
|
||||
commitTime +" at fileLoc: " + fileLoc);
|
||||
@@ -424,7 +433,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
return new LazyInsertIterable<>(recordItr, config, commitTime, metadata);
|
||||
return new LazyInsertIterable<>(recordItr, config, commitTime, metaClient);
|
||||
}
|
||||
|
||||
|
||||
|
||||
@@ -16,11 +16,11 @@
|
||||
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
|
||||
@@ -39,12 +39,13 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
|
||||
protected final HoodieTableMetadata metadata;
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
|
||||
protected HoodieTable(String commitTime, HoodieWriteConfig config, HoodieTableMetadata metadata) {
|
||||
protected HoodieTable(String commitTime, HoodieWriteConfig config,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
this.commitTime = commitTime;
|
||||
this.config = config;
|
||||
this.metadata = metadata;
|
||||
this.metaClient = metaClient;
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -81,8 +82,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
* @param partitioner
|
||||
*/
|
||||
public abstract Iterator<List<WriteStatus>> handleUpsertPartition(Integer partition,
|
||||
Iterator<HoodieRecord<T>> recordIterator,
|
||||
Partitioner partitioner);
|
||||
Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
||||
|
||||
/**
|
||||
* Perform the ultimate IO for a given inserted (RDD) partition
|
||||
@@ -96,14 +96,13 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
Partitioner partitioner);
|
||||
|
||||
|
||||
public static HoodieTable getHoodieTable(HoodieTableType type,
|
||||
public static HoodieTable getHoodieTable(HoodieTableMetaClient metaClient,
|
||||
String commitTime,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTableMetadata metadata) {
|
||||
if (type == HoodieTableType.COPY_ON_WRITE) {
|
||||
return new HoodieCopyOnWriteTable(commitTime, config, metadata);
|
||||
HoodieWriteConfig config) {
|
||||
if (metaClient.getTableType() == HoodieTableType.COPY_ON_WRITE) {
|
||||
return new HoodieCopyOnWriteTable(commitTime, config, metaClient);
|
||||
} else {
|
||||
throw new HoodieException("Unsupported table type :"+ type);
|
||||
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -21,12 +21,15 @@ import com.google.common.collect.Iterables;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieCommits;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
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.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.ParquetUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
@@ -59,9 +62,11 @@ import java.util.HashSet;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeSet;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
@@ -87,7 +92,7 @@ public class TestHoodieClient implements Serializable {
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath);
|
||||
HoodieTestUtils.init(basePath);
|
||||
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
}
|
||||
@@ -200,7 +205,7 @@ public class TestHoodieClient implements Serializable {
|
||||
assertEquals("Latest commit should be 001",readClient.latestCommit(), newCommitTime);
|
||||
assertEquals("Must contain 200 records", readClient.readCommit(newCommitTime).count(), records.size());
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetadata(fs, basePath)).collect();
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetaClient(fs, basePath)).collect();
|
||||
checkTaggedRecords(taggedRecords, "001");
|
||||
|
||||
/**
|
||||
@@ -226,7 +231,7 @@ public class TestHoodieClient implements Serializable {
|
||||
assertEquals("Latest commit should be 004",readClient.latestCommit(), newCommitTime);
|
||||
|
||||
// Index should be able to locate all updates in correct locations.
|
||||
taggedRecords = index.tagLocation(jsc.parallelize(dedupedRecords, 1), new HoodieTableMetadata(fs, basePath)).collect();
|
||||
taggedRecords = index.tagLocation(jsc.parallelize(dedupedRecords, 1), new HoodieTableMetaClient(fs, basePath)).collect();
|
||||
checkTaggedRecords(taggedRecords, "004");
|
||||
|
||||
// Check the entire dataset has 100 records still
|
||||
@@ -273,7 +278,7 @@ public class TestHoodieClient implements Serializable {
|
||||
|
||||
assertEquals("Expecting a single commit.", new HoodieReadClient(jsc, basePath).listCommitsSince("000").size(), 1);
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetadata(fs, basePath)).collect();
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetaClient(fs, basePath)).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
|
||||
// Keep doing some writes and clean inline. Make sure we have expected number of files remaining.
|
||||
@@ -287,34 +292,39 @@ public class TestHoodieClient implements Serializable {
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
SortedMap<String, HoodieCommitMetadata> commitMetadata = metadata.getAllCommitMetadata();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
|
||||
|
||||
TableFileSystemView fsView = new ReadOptimizedTableView(fs, metadata);
|
||||
// Need to ensure the following
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
// compute all the versions of all files, from time 0
|
||||
HashMap<String, TreeSet<String>> fileIdToVersions = new HashMap<>();
|
||||
for (Map.Entry<String, HoodieCommitMetadata> entry : commitMetadata.entrySet()) {
|
||||
for (HoodieWriteStat wstat : entry.getValue().getWriteStats(partitionPath)) {
|
||||
for (String entry : timeline.getInstants().collect(Collectors.toList())) {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.readInstantDetails(entry).get());
|
||||
|
||||
for (HoodieWriteStat wstat : commitMetadata.getWriteStats(partitionPath)) {
|
||||
if (!fileIdToVersions.containsKey(wstat.getFileId())) {
|
||||
fileIdToVersions.put(wstat.getFileId(), new TreeSet<String>());
|
||||
fileIdToVersions.put(wstat.getFileId(), new TreeSet<>());
|
||||
}
|
||||
fileIdToVersions.get(wstat.getFileId()).add(entry.getKey());
|
||||
fileIdToVersions.get(wstat.getFileId()).add(FSUtils.getCommitTime(new Path(wstat.getFullPath()).getName()));
|
||||
}
|
||||
}
|
||||
|
||||
Map<String, List<FileStatus>> fileVersions = metadata.getAllVersionsInPartition(fs, partitionPath);
|
||||
for (Map.Entry<String, List<FileStatus>> entry : fileVersions.entrySet()) {
|
||||
List<FileStatus> versions = entry.getValue();
|
||||
|
||||
List<List<HoodieDataFile>> fileVersions = fsView.streamEveryVersionInPartition(partitionPath).collect(Collectors.toList());
|
||||
for (List<HoodieDataFile> entry : fileVersions) {
|
||||
// No file has no more than max versions
|
||||
assertTrue("fileId " + entry.getKey() + " has more than " + maxVersions + " versions",
|
||||
versions.size() <= maxVersions);
|
||||
String fileId = entry.iterator().next().getFileId();
|
||||
|
||||
assertTrue("fileId " + fileId + " has more than " + maxVersions + " versions",
|
||||
entry.size() <= maxVersions);
|
||||
|
||||
// Each file, has the latest N versions (i.e cleaning gets rid of older versions)
|
||||
List<String> commitedVersions = new ArrayList<>(fileIdToVersions.get(entry.getKey()));
|
||||
for (int i = 0; i < versions.size(); i++) {
|
||||
assertEquals("File " + entry.getKey() + " does not have latest versions" + versions + " on commits" + commitedVersions,
|
||||
FSUtils.getCommitTime(Iterables.get(versions, i).getPath().getName()),
|
||||
List<String> commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId));
|
||||
for (int i = 0; i < entry.size(); i++) {
|
||||
assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions,
|
||||
Iterables.get(entry, i).getCommitTime(),
|
||||
commitedVersions.get(commitedVersions.size() - 1 - i));
|
||||
}
|
||||
}
|
||||
@@ -349,7 +359,7 @@ public class TestHoodieClient implements Serializable {
|
||||
// verify that there is a commit
|
||||
assertEquals("Expecting a single commit.", new HoodieReadClient(jsc, basePath).listCommitsSince("000").size(), 1);
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetadata(fs, basePath)).collect();
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), new HoodieTableMetaClient(fs, basePath)).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
|
||||
// Keep doing some writes and clean inline. Make sure we have expected number of files remaining.
|
||||
@@ -362,23 +372,29 @@ public class TestHoodieClient implements Serializable {
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
HoodieCommits commits = metadata.getAllCommits();
|
||||
String earliestRetainedCommit = commits.lastCommit(maxCommits - 1);
|
||||
Set<String> acceptableCommits = new HashSet<>(commits.getCommitList());
|
||||
if (earliestRetainedCommit != null) {
|
||||
acceptableCommits.removeAll(commits.findCommitsInRange("000", earliestRetainedCommit));
|
||||
acceptableCommits.add(earliestRetainedCommit);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTimeline activeTimeline = metadata.getActiveCommitTimeline();
|
||||
Optional<String> earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1);
|
||||
Set<String> acceptableCommits =
|
||||
activeTimeline.getInstants().collect(Collectors.toSet());
|
||||
if (earliestRetainedCommit.isPresent()) {
|
||||
acceptableCommits.removeAll(
|
||||
activeTimeline.findInstantsInRange("000", earliestRetainedCommit.get())
|
||||
.collect(Collectors.toSet()));
|
||||
acceptableCommits.add(earliestRetainedCommit.get());
|
||||
}
|
||||
|
||||
TableFileSystemView fsView = new ReadOptimizedTableView(fs, metadata);
|
||||
// Need to ensure the following
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
Map<String, List<FileStatus>> fileVersions = metadata.getAllVersionsInPartition(fs, partitionPath);
|
||||
for (Map.Entry<String, List<FileStatus>> entry : fileVersions.entrySet()) {
|
||||
Set<String> commitTimes = new HashSet<>(entry.getValue().size());
|
||||
for(FileStatus value:entry.getValue()) {
|
||||
commitTimes.add(FSUtils.getCommitTime(value.getPath().getName()));
|
||||
List<List<HoodieDataFile>> fileVersions = fsView.streamEveryVersionInPartition(partitionPath).collect(Collectors.toList());
|
||||
for (List<HoodieDataFile> entry : fileVersions) {
|
||||
Set<String> commitTimes = new HashSet<>();
|
||||
for(HoodieDataFile value:entry) {
|
||||
System.out.println("Data File - " + value);
|
||||
commitTimes.add(value.getCommitTime());
|
||||
}
|
||||
System.out.println("Existing commits " + activeTimeline.getInstants().collect(Collectors.toList()));
|
||||
assertEquals("Only contain acceptable versions of file should be present",
|
||||
acceptableCommits, commitTimes);
|
||||
}
|
||||
@@ -620,13 +636,16 @@ public class TestHoodieClient implements Serializable {
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
assertEquals("2 files needs to be committed.", 2, statuses.size());
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
FileStatus[] files = metadata.getLatestVersionInPartition(fs, TEST_PARTITION_PATH, commitTime3);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieTimeline activeTimeline = metadata.getActiveCommitTimeline();
|
||||
TableFileSystemView fileSystemView = new ReadOptimizedTableView(fs, metadata);
|
||||
List<HoodieDataFile> files = fileSystemView.streamLatestVersionInPartition(TEST_PARTITION_PATH, commitTime3).collect(
|
||||
Collectors.toList());
|
||||
int numTotalInsertsInCommit3 = 0;
|
||||
for (FileStatus file: files) {
|
||||
if (file.getPath().getName().contains(file1)) {
|
||||
assertEquals("Existing file should be expanded", commitTime3, FSUtils.getCommitTime(file.getPath().getName()));
|
||||
records = ParquetUtils.readAvroRecords(file.getPath());
|
||||
for (HoodieDataFile file: files) {
|
||||
if (file.getFileName().contains(file1)) {
|
||||
assertEquals("Existing file should be expanded", commitTime3, file.getCommitTime());
|
||||
records = ParquetUtils.readAvroRecords(new Path(file.getPath()));
|
||||
for (GenericRecord record: records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String recordCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
@@ -641,8 +660,8 @@ public class TestHoodieClient implements Serializable {
|
||||
}
|
||||
assertEquals("All keys added in commit 2 must be updated in commit3 correctly", 0, keys2.size());
|
||||
} else {
|
||||
assertEquals("New file must be written for commit 3", commitTime3, FSUtils.getCommitTime(file.getPath().getName()));
|
||||
records = ParquetUtils.readAvroRecords(file.getPath());
|
||||
assertEquals("New file must be written for commit 3", commitTime3, file.getCommitTime());
|
||||
records = ParquetUtils.readAvroRecords(new Path(file.getPath()));
|
||||
for (GenericRecord record: records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
assertEquals("only expect commit3", commitTime3, record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString());
|
||||
|
||||
@@ -18,7 +18,7 @@ package com.uber.hoodie.common;
|
||||
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import java.io.File;
|
||||
@@ -53,18 +53,18 @@ public class HoodieClientTestUtils {
|
||||
}
|
||||
|
||||
private static void fakeMetaFile(String basePath, String commitTime, String suffix) throws IOException {
|
||||
String parentPath = basePath + "/"+ HoodieTableMetadata.METAFOLDER_NAME;
|
||||
String parentPath = basePath + "/"+ HoodieTableMetaClient.METAFOLDER_NAME;
|
||||
new File(parentPath).mkdirs();
|
||||
new File(parentPath + "/" + commitTime + suffix).createNewFile();
|
||||
}
|
||||
|
||||
|
||||
public static void fakeCommitFile(String basePath, String commitTime) throws IOException {
|
||||
fakeMetaFile(basePath, commitTime, HoodieTableMetadata.COMMIT_FILE_SUFFIX);
|
||||
fakeMetaFile(basePath, commitTime, HoodieTableMetaClient.COMMIT_EXTENSION);
|
||||
}
|
||||
|
||||
public static void fakeInFlightFile(String basePath, String commitTime) throws IOException {
|
||||
fakeMetaFile(basePath, commitTime, HoodieTableMetadata.INFLIGHT_FILE_SUFFIX);
|
||||
fakeMetaFile(basePath, commitTime, HoodieTableMetaClient.INFLIGHT_FILE_SUFFIX);
|
||||
}
|
||||
|
||||
public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId) throws Exception {
|
||||
|
||||
@@ -19,7 +19,7 @@ package com.uber.hoodie.common;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
|
||||
@@ -29,8 +29,6 @@ import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
@@ -144,7 +142,7 @@ public class HoodieTestDataGenerator {
|
||||
|
||||
public static void createCommitFile(String basePath, String commitTime) throws IOException {
|
||||
Path commitFile =
|
||||
new Path(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME + "/" + FSUtils.makeCommitFileName(commitTime));
|
||||
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTableMetaClient.makeCommitFileName(commitTime));
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
FSDataOutputStream os = fs.create(commitFile, true);
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
|
||||
@@ -16,13 +16,13 @@
|
||||
|
||||
package com.uber.hoodie.func;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.TestRawTripPayload;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
|
||||
@@ -48,14 +48,14 @@ public class TestUpdateMapFunction {
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
this.basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath);
|
||||
HoodieTestUtils.init(basePath);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSchemaEvolutionOnUpdate() throws Exception {
|
||||
// Create a bunch of records with a old version of schema
|
||||
HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable("100", config, metadata);
|
||||
|
||||
String recordStr1 =
|
||||
@@ -79,13 +79,13 @@ public class TestUpdateMapFunction {
|
||||
rowChange3));
|
||||
Iterator<List<WriteStatus>> insertResult = table.handleInsert(records.iterator());
|
||||
Path commitFile =
|
||||
new Path(config.getBasePath() + "/.hoodie/" + FSUtils.makeCommitFileName("100"));
|
||||
new Path(config.getBasePath() + "/.hoodie/" + HoodieTableMetaClient.makeCommitFileName("100"));
|
||||
FSUtils.getFs().create(commitFile);
|
||||
|
||||
// Now try an update with an evolved schema
|
||||
// Evolved schema does not have guarantee on preserving the original field ordering
|
||||
config = makeHoodieClientConfig("/exampleEvolvedSchema.txt");
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath);
|
||||
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
String fileId = insertResult.next().get(0).getFileId();
|
||||
System.out.println(fileId);
|
||||
|
||||
|
||||
@@ -19,13 +19,13 @@ package com.uber.hoodie.index;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||
import com.uber.hoodie.common.BloomFilter;
|
||||
import com.uber.hoodie.common.TestRawTripPayload;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
@@ -74,7 +74,7 @@ public class TestHoodieBloomIndex {
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath);
|
||||
HoodieTestUtils.init(basePath);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -126,7 +126,7 @@ public class TestHoodieBloomIndex {
|
||||
new File(basePath + "/2015/03/12/3_0_20150312101010.parquet").createNewFile();
|
||||
new File(basePath + "/2015/03/12/4_0_20150312101010.parquet").createNewFile();
|
||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
JavaPairRDD<String, String> rdd = index.loadInvolvedFiles(partitions, metadata);
|
||||
// Still 0, as no valid commit
|
||||
assertEquals(rdd.count(), 0);
|
||||
@@ -135,7 +135,7 @@ public class TestHoodieBloomIndex {
|
||||
new File(basePath + "/.hoodie").mkdirs();
|
||||
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
|
||||
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
rdd = index.loadInvolvedFiles(partitions, metadata);
|
||||
final List<Tuple2<String, String>> filesList = rdd.collect();
|
||||
assertEquals(filesList.size(), 4);
|
||||
@@ -212,7 +212,7 @@ public class TestHoodieBloomIndex {
|
||||
// We have some records to be tagged (two different partitions)
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.emptyRDD();
|
||||
// Also create the metadata and config
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
|
||||
// Let's tag
|
||||
@@ -248,7 +248,7 @@ public class TestHoodieBloomIndex {
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
|
||||
|
||||
// Also create the metadata and config
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
|
||||
// Let's tag
|
||||
@@ -266,7 +266,7 @@ public class TestHoodieBloomIndex {
|
||||
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
|
||||
|
||||
// We do the tag again
|
||||
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, metadata);
|
||||
|
||||
// Check results
|
||||
@@ -309,7 +309,7 @@ public class TestHoodieBloomIndex {
|
||||
JavaRDD<HoodieKey> keysRDD = jsc.parallelize(Arrays.asList(key1, key2, key3, key4));
|
||||
|
||||
// Also create the metadata and config
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
|
||||
// Let's tag
|
||||
@@ -327,7 +327,7 @@ public class TestHoodieBloomIndex {
|
||||
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
|
||||
|
||||
// We do the tag again
|
||||
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, metadata);
|
||||
|
||||
// Check results
|
||||
@@ -375,7 +375,7 @@ public class TestHoodieBloomIndex {
|
||||
|
||||
// We do the tag
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, metadata);
|
||||
@@ -421,8 +421,8 @@ public class TestHoodieBloomIndex {
|
||||
|
||||
if (createCommitTime) {
|
||||
// Also make sure the commit is valid
|
||||
new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME).mkdirs();
|
||||
new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME + "/" + commitTime + ".commit").createNewFile();
|
||||
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME).mkdirs();
|
||||
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + ".commit").createNewFile();
|
||||
}
|
||||
return filename;
|
||||
}
|
||||
|
||||
@@ -16,8 +16,8 @@
|
||||
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
@@ -34,10 +34,12 @@ public class TestHoodieCleaner {
|
||||
|
||||
private String basePath = null;
|
||||
private String[] partitionPaths = {"2016/01/01", "2016/02/02"};
|
||||
private HoodieTableMetaClient metaClient;
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
this.basePath = HoodieTestUtils.initializeTempHoodieBasePath();
|
||||
this.metaClient = HoodieTestUtils.initOnTemp();
|
||||
this.basePath = metaClient.getBasePath();
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -53,7 +55,7 @@ public class TestHoodieCleaner {
|
||||
String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000");
|
||||
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000");
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieCleaner cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
|
||||
@@ -67,7 +69,7 @@ public class TestHoodieCleaner {
|
||||
String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "001"); // insert
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0C0); // update
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[1], "001", file1P1C0); // update
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
|
||||
assertEquals("Must clean 1 file" , 1, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must clean 1 file" , 1, cleaner.clean(partitionPaths[1]));
|
||||
@@ -82,7 +84,7 @@ public class TestHoodieCleaner {
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update
|
||||
String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "002");
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
|
||||
assertEquals("Must clean two files" , 2, cleaner.clean(partitionPaths[0]));
|
||||
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file1P0C0));
|
||||
@@ -110,7 +112,7 @@ public class TestHoodieCleaner {
|
||||
String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000");
|
||||
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000");
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieCleaner cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
|
||||
@@ -124,7 +126,7 @@ public class TestHoodieCleaner {
|
||||
String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "001"); // insert
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0C0); // update
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[1], "001", file1P1C0); // update
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[0]));
|
||||
assertEquals("Must not clean any files" , 0, cleaner.clean(partitionPaths[1]));
|
||||
@@ -139,7 +141,7 @@ public class TestHoodieCleaner {
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update
|
||||
String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "002");
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
|
||||
assertEquals(
|
||||
"Must not clean any file. We have to keep 1 version before the latest commit time to keep",
|
||||
@@ -153,7 +155,7 @@ public class TestHoodieCleaner {
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file1P0C0); // update
|
||||
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file2P0C1); // update
|
||||
String file4P0C3 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "003");
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
cleaner = new HoodieCleaner(metadata, config, FSUtils.getFs());
|
||||
assertEquals(
|
||||
"Must not clean one old file", 1, cleaner.clean(partitionPaths[0]));
|
||||
|
||||
@@ -17,10 +17,11 @@
|
||||
package com.uber.hoodie.io;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
@@ -32,8 +33,11 @@ import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
@@ -47,7 +51,7 @@ public class TestHoodieCommitArchiveLog {
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath);
|
||||
HoodieTestUtils.init(basePath);
|
||||
fs = FSUtils.getFs();
|
||||
}
|
||||
|
||||
@@ -56,7 +60,7 @@ public class TestHoodieCommitArchiveLog {
|
||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable("test-trip-table").build();
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
assertTrue(result);
|
||||
}
|
||||
@@ -67,20 +71,22 @@ public class TestHoodieCommitArchiveLog {
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable("test-trip-table").withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "102");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "103");
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
|
||||
|
||||
assertEquals("Loaded 4 commits and the count should match", 4,
|
||||
metadata.getAllCommits().getCommitList().size());
|
||||
timeline.getTotalInstants());
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
assertTrue(result);
|
||||
metadata = new HoodieTableMetadata(fs, basePath);
|
||||
timeline = timeline.reload();
|
||||
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4,
|
||||
metadata.getAllCommits().getCommitList().size());
|
||||
timeline.getTotalInstants());
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -89,7 +95,8 @@ public class TestHoodieCommitArchiveLog {
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable("test-trip-table").withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "102");
|
||||
@@ -97,24 +104,22 @@ public class TestHoodieCommitArchiveLog {
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "104");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
SortedMap<String, HoodieCommitMetadata> originalCommits = new TreeMap<>(metadata.getAllCommitMetadata());
|
||||
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
|
||||
List<String> originalCommits = timeline.getInstants().collect(
|
||||
Collectors.toList());
|
||||
|
||||
assertEquals("Loaded 6 commits and the count should match", 6,
|
||||
metadata.getAllCommits().getCommitList().size());
|
||||
assertEquals("Loaded 6 commits and the count should match", 6, timeline.getTotalInstants());
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
assertTrue(result);
|
||||
metadata = new HoodieTableMetadata(fs, basePath);
|
||||
timeline = timeline.reload();
|
||||
assertEquals(
|
||||
"Should archive commits when maxCommitsToKeep is 5 and now the commits length should be minCommitsToKeep which is 2",
|
||||
2, metadata.getAllCommits().getCommitList().size());
|
||||
2, timeline.getTotalInstants());
|
||||
assertEquals("Archive should not archive the last 2 commits",
|
||||
Lists.newArrayList("104", "105"), metadata.getAllCommits().getCommitList());
|
||||
Lists.newArrayList("104", "105"), timeline.getInstants().collect(Collectors.toList()));
|
||||
|
||||
// Remove all the commits from the original commits, make it ready to be checked against the read map
|
||||
for(String key:metadata.getAllCommitMetadata().keySet()) {
|
||||
originalCommits.remove(key);
|
||||
}
|
||||
timeline.getInstants().forEach(originalCommits::remove);
|
||||
|
||||
// Read back the commits to make sure
|
||||
SequenceFile.Reader reader = new SequenceFile.Reader(fs.getConf(),
|
||||
@@ -129,7 +134,7 @@ public class TestHoodieCommitArchiveLog {
|
||||
|
||||
assertEquals(
|
||||
"Read commits map should match the originalCommits - commitsLoadedAfterArchival",
|
||||
originalCommits, readCommits);
|
||||
originalCommits, new ArrayList<>(readCommits.keySet()));
|
||||
reader.close();
|
||||
}
|
||||
|
||||
@@ -139,7 +144,8 @@ public class TestHoodieCommitArchiveLog {
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.forTable("test-trip-table").withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "102");
|
||||
@@ -147,16 +153,15 @@ public class TestHoodieCommitArchiveLog {
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "104");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath);
|
||||
assertEquals("Loaded 6 commits and the count should match", 6,
|
||||
metadata.getAllCommits().getCommitList().size());
|
||||
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
|
||||
assertEquals("Loaded 6 commits and the count should match", 6, timeline.getTotalInstants());
|
||||
boolean result = archiveLog.archiveIfRequired();
|
||||
assertTrue(result);
|
||||
metadata = new HoodieTableMetadata(fs, basePath);
|
||||
assertTrue("Archived commits should always be safe", metadata.isCommitTsSafe("100"));
|
||||
assertTrue("Archived commits should always be safe", metadata.isCommitTsSafe("101"));
|
||||
assertTrue("Archived commits should always be safe", metadata.isCommitTsSafe("102"));
|
||||
assertTrue("Archived commits should always be safe", metadata.isCommitTsSafe("103"));
|
||||
timeline = timeline.reload();
|
||||
assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("100"));
|
||||
assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("101"));
|
||||
assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("102"));
|
||||
assertTrue("Archived commits should always be safe", timeline.containsOrBeforeTimelineStarts("103"));
|
||||
}
|
||||
|
||||
|
||||
|
||||
@@ -16,17 +16,17 @@
|
||||
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.BloomFilter;
|
||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||
import com.uber.hoodie.common.TestRawTripPayload;
|
||||
import com.uber.hoodie.common.model.HoodieCommits;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.ParquetUtils;
|
||||
@@ -76,7 +76,7 @@ public class TestCopyOnWriteTable {
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
this.basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath);
|
||||
HoodieTestUtils.init(basePath);
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -87,9 +87,10 @@ public class TestCopyOnWriteTable {
|
||||
HoodieRecord record = mock(HoodieRecord.class);
|
||||
when(record.getPartitionPath()).thenReturn(partitionPath);
|
||||
|
||||
String commitTime = HoodieTestUtils.getNewCommitTime();
|
||||
String commitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
HoodieInsertHandle io = new HoodieInsertHandle(config, commitTime, null, partitionPath);
|
||||
HoodieInsertHandle io = new HoodieInsertHandle(config, commitTime,
|
||||
new HoodieTableMetaClient(FSUtils.getFs(), basePath), partitionPath);
|
||||
Path newPath = io.makeNewPath(record.getPartitionPath(), unitNumber, fileName);
|
||||
assertTrue(newPath.toString().equals(this.basePath + "/" + partitionPath + "/" + FSUtils
|
||||
.makeDataFileName(commitTime, unitNumber, fileName)));
|
||||
@@ -110,8 +111,8 @@ public class TestCopyOnWriteTable {
|
||||
public void testUpdateRecords() throws Exception {
|
||||
// Prepare the AvroParquetIO
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
String firstCommitTime = HoodieTestUtils.getNewCommitTime();
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
String partitionPath = "/2016/01/31";
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(firstCommitTime, config, metadata);
|
||||
|
||||
@@ -148,7 +149,7 @@ public class TestCopyOnWriteTable {
|
||||
assertTrue(filter.mightContain(record.getRecordKey()));
|
||||
}
|
||||
// Create a commit file
|
||||
new File(this.basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME + "/"
|
||||
new File(this.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ FSUtils.getCommitTime(parquetFile.getName()) + ".commit").createNewFile();
|
||||
|
||||
// Read the parquet file, check the record content
|
||||
@@ -172,8 +173,8 @@ public class TestCopyOnWriteTable {
|
||||
List<HoodieRecord> updatedRecords = Arrays.asList(updatedRecord1, insertedRecord1);
|
||||
|
||||
Thread.sleep(1000);
|
||||
String newCommitTime = HoodieTestUtils.getNewCommitTime();
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
String newCommitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
table = new HoodieCopyOnWriteTable(newCommitTime, config, metadata);
|
||||
Iterator<List<WriteStatus>> iter = table.handleUpdate(updatedRecord1.getCurrentLocation().getFileId(), updatedRecords.iterator());
|
||||
|
||||
@@ -181,9 +182,11 @@ public class TestCopyOnWriteTable {
|
||||
File updatedParquetFile = null;
|
||||
for (File file : new File(basePath + "/2016/01/31").listFiles()) {
|
||||
if (file.getName().endsWith(".parquet")) {
|
||||
if (FSUtils.getFileId(file.getName()).equals(FSUtils.getFileId(parquetFile.getName()))
|
||||
&& HoodieCommits
|
||||
.isCommit1After(FSUtils.getCommitTime(file.getName()), FSUtils.getCommitTime(parquetFile.getName()))) {
|
||||
if (FSUtils.getFileId(file.getName())
|
||||
.equals(FSUtils.getFileId(parquetFile.getName())) && metadata
|
||||
.getActiveCommitTimeline()
|
||||
.compareInstants(FSUtils.getCommitTime(file.getName()),
|
||||
FSUtils.getCommitTime(parquetFile.getName()), HoodieTimeline.GREATER)) {
|
||||
updatedParquetFile = file;
|
||||
break;
|
||||
}
|
||||
@@ -236,9 +239,9 @@ public class TestCopyOnWriteTable {
|
||||
|
||||
@Test public void testInsertWithPartialFailures() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
String commitTime = HoodieTestUtils.getNewCommitTime();
|
||||
String commitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(commitTime, config, metadata);
|
||||
|
||||
// Write a few records, and get atleast one file
|
||||
@@ -275,8 +278,8 @@ public class TestCopyOnWriteTable {
|
||||
|
||||
@Test public void testInsertRecords() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
String commitTime = HoodieTestUtils.getNewCommitTime();
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
String commitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(commitTime, config, metadata);
|
||||
|
||||
// Case 1:
|
||||
@@ -322,8 +325,8 @@ public class TestCopyOnWriteTable {
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withStorageConfig(
|
||||
HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024)
|
||||
.parquetPageSize(64 * 1024).build()).build();
|
||||
String commitTime = HoodieTestUtils.getNewCommitTime();
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
String commitTime = HoodieTestUtils.makeNewCommitTime();
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(commitTime, config, metadata);
|
||||
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
@@ -367,7 +370,7 @@ public class TestCopyOnWriteTable {
|
||||
HoodieClientTestUtils.fakeCommitFile(basePath, "001");
|
||||
HoodieClientTestUtils.fakeDataFile(basePath, TEST_PARTITION_PATH, "001", "file1", fileSize);
|
||||
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable("001", config, metadata);
|
||||
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[]{TEST_PARTITION_PATH});
|
||||
|
||||
@@ -28,8 +28,8 @@
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.codehaus.mojo</groupId>
|
||||
<artifactId>cobertura-maven-plugin</artifactId>
|
||||
<groupId>org.jacoco</groupId>
|
||||
<artifactId>jacoco-maven-plugin</artifactId>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
@@ -90,5 +90,14 @@
|
||||
<version>1.10.19</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.commons</groupId>
|
||||
<artifactId>commons-lang3</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.esotericsoftware</groupId>
|
||||
<artifactId>kryo</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
@@ -187,4 +188,8 @@ public class HoodieCommitMetadata implements Serializable {
|
||||
public int hashCode() {
|
||||
return partitionToWriteStats != null ? partitionToWriteStats.hashCode() : 0;
|
||||
}
|
||||
|
||||
public static HoodieCommitMetadata fromBytes(byte[] bytes) throws IOException {
|
||||
return fromJsonString(new String(bytes, Charset.forName("utf-8")));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,191 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.model;
|
||||
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Manages the commit meta and provides operations on the commit timeline
|
||||
*/
|
||||
public class HoodieCommits implements Serializable {
|
||||
|
||||
private List<String> commitList;
|
||||
|
||||
public HoodieCommits(List<String> commitList) {
|
||||
this.commitList = new ArrayList<>(commitList);
|
||||
Collections.sort(this.commitList);
|
||||
this.commitList = Collections.unmodifiableList(this.commitList);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the commits which are in the range (startsTs, endTs].
|
||||
*
|
||||
* @param startTs - exclusive start commit ts
|
||||
* @param endTs - inclusive end commit ts
|
||||
*/
|
||||
public List<String> findCommitsInRange(String startTs, String endTs) {
|
||||
if (commitList.isEmpty()) {
|
||||
return Collections.EMPTY_LIST;
|
||||
}
|
||||
int startIndex = 0;
|
||||
if (startTs != null) {
|
||||
startIndex = Collections.binarySearch(commitList, startTs);
|
||||
// If startIndex is negative
|
||||
if (startIndex < 0) {
|
||||
startIndex = -(startIndex + 1);
|
||||
}
|
||||
}
|
||||
|
||||
int endIndex = Collections.binarySearch(commitList, endTs);
|
||||
// If endIndex is negative
|
||||
if (endIndex < 0) {
|
||||
endIndex = -(endIndex + 1);
|
||||
}
|
||||
|
||||
if (endIndex < startIndex) {
|
||||
throw new IllegalArgumentException(
|
||||
"Start Commit Ts " + startTs + " cannot be less than end commit ts" + endTs);
|
||||
}
|
||||
List<String> returns = new ArrayList<>(commitList.subList(startIndex, endIndex));
|
||||
if(endIndex < commitList.size()) {
|
||||
// Be inclusive of the endIndex
|
||||
returns.add(commitList.get(endIndex));
|
||||
}
|
||||
return Collections.unmodifiableList(returns);
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds the list of commits on or before asOfTs
|
||||
*/
|
||||
public List<String> findCommitsAfter(String commitTimeStamp, int numCommits) {
|
||||
if (commitList.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
|
||||
int startIndex = Collections.binarySearch(commitList, commitTimeStamp);
|
||||
if (startIndex < 0) {
|
||||
startIndex = -(startIndex + 1);
|
||||
} else {
|
||||
// we found asOfTs at startIndex. We want to exclude it.
|
||||
startIndex++;
|
||||
}
|
||||
|
||||
|
||||
List<String> commits = new ArrayList<>();
|
||||
while (numCommits > 0 && startIndex < commitList.size()) {
|
||||
commits.add(commitList.get(startIndex));
|
||||
startIndex++;
|
||||
numCommits--;
|
||||
}
|
||||
|
||||
return Collections.unmodifiableList(commits);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieCommits{");
|
||||
sb.append("commitList=").append(commitList);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public boolean isEmpty() {
|
||||
return commitList.isEmpty();
|
||||
}
|
||||
|
||||
public int getNumCommits() {
|
||||
return commitList.size();
|
||||
}
|
||||
|
||||
public String firstCommit() {
|
||||
return commitList.isEmpty() ? null : commitList.get(0);
|
||||
}
|
||||
|
||||
public String nthCommit(int n) {
|
||||
return commitList.isEmpty() || n >= commitList.size() ? null : commitList.get(n);
|
||||
}
|
||||
|
||||
public String lastCommit() {
|
||||
return commitList.isEmpty() ? null : commitList.get(commitList.size() - 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the nth commit from the latest commit such that lastCommit(0) gteq lastCommit()
|
||||
*/
|
||||
public String lastCommit(int n) {
|
||||
if (commitList.size() < n + 1) {
|
||||
return null;
|
||||
}
|
||||
return commitList.get(commitList.size() - 1 - n);
|
||||
}
|
||||
|
||||
public boolean contains(String commitTs) {
|
||||
return commitList.contains(commitTs);
|
||||
}
|
||||
|
||||
public String max(String commit1, String commit2) {
|
||||
if (commit1 == null && commit2 == null) {
|
||||
return null;
|
||||
}
|
||||
if (commit1 == null) {
|
||||
return commit2;
|
||||
}
|
||||
if (commit2 == null) {
|
||||
return commit1;
|
||||
}
|
||||
return (isCommit1BeforeOrOn(commit1, commit2) ? commit2 : commit1);
|
||||
}
|
||||
|
||||
public static boolean isCommit1BeforeOrOn(String commit1, String commit2) {
|
||||
return commit1.compareTo(commit2) <= 0;
|
||||
}
|
||||
|
||||
public static boolean isCommit1After(String commit1, String commit2) {
|
||||
return commit1.compareTo(commit2) > 0;
|
||||
}
|
||||
|
||||
public List<String> getCommitList() {
|
||||
return commitList;
|
||||
}
|
||||
|
||||
public boolean isCommitBeforeEarliestCommit(String commitTs) {
|
||||
return isCommit1BeforeOrOn(commitTs, firstCommit());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o)
|
||||
return true;
|
||||
if (o == null || getClass() != o.getClass())
|
||||
return false;
|
||||
|
||||
HoodieCommits that = (HoodieCommits) o;
|
||||
|
||||
return commitList != null ? commitList.equals(that.commitList) : that.commitList == null;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return commitList != null ? commitList.hashCode() : 0;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,71 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.model;
|
||||
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
|
||||
import java.util.Comparator;
|
||||
|
||||
public class HoodieDataFile {
|
||||
private FileStatus fileStatus;
|
||||
|
||||
public HoodieDataFile(FileStatus fileStatus) {
|
||||
this.fileStatus = fileStatus;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return FSUtils.getFileId(fileStatus.getPath().getName());
|
||||
}
|
||||
|
||||
public String getCommitTime() {
|
||||
return FSUtils.getCommitTime(fileStatus.getPath().getName());
|
||||
}
|
||||
|
||||
public String getPath() {
|
||||
return fileStatus.getPath().toString();
|
||||
}
|
||||
|
||||
public String getFileName() {
|
||||
return fileStatus.getPath().getName();
|
||||
}
|
||||
|
||||
public FileStatus getFileStatus() {
|
||||
return fileStatus;
|
||||
}
|
||||
|
||||
public static Comparator<HoodieDataFile> getCommitTimeComparator() {
|
||||
return (o1, o2) -> {
|
||||
// reverse the order
|
||||
return o2.getCommitTime().compareTo(o1.getCommitTime());
|
||||
};
|
||||
}
|
||||
|
||||
public long getFileSize() {
|
||||
return fileStatus.getLen();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieDataFile{");
|
||||
sb.append("fileStatus=").append(fileStatus);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
@@ -1,57 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.model;
|
||||
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
|
||||
public class HoodieFile {
|
||||
|
||||
private final FileStatus fileStatus;
|
||||
private String fileNameWithoutCommitTs;
|
||||
private String commitTs;
|
||||
|
||||
public HoodieFile(FileStatus fileStatus) {
|
||||
this.fileStatus = fileStatus;
|
||||
String fileName = fileStatus.getPath().getName();
|
||||
this.fileNameWithoutCommitTs = FSUtils.getFileId(fileName);
|
||||
this.commitTs = FSUtils.getCommitTime(fileName);
|
||||
}
|
||||
|
||||
public String getFileNameWithoutCommitTs() {
|
||||
return fileNameWithoutCommitTs;
|
||||
}
|
||||
|
||||
public String getCommitTs() {
|
||||
return commitTs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieFile{");
|
||||
sb.append("fileStatus=").append(fileStatus);
|
||||
sb.append(", fileNameWithoutCommitTs='").append(fileNameWithoutCommitTs).append('\'');
|
||||
sb.append(", commitTs='").append(commitTs).append('\'');
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public FileStatus getFileStatus() {
|
||||
return fileStatus;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,31 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.model;
|
||||
|
||||
public enum HoodieStorageType {
|
||||
PARQUET(".parquet");
|
||||
|
||||
private final String extension;
|
||||
|
||||
HoodieStorageType(String extension) {
|
||||
this.extension = extension;
|
||||
}
|
||||
|
||||
public String getFileExtension() {
|
||||
return extension;
|
||||
}
|
||||
}
|
||||
@@ -1,477 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.model;
|
||||
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import com.uber.hoodie.exception.DatasetNotFoundException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.InvalidDatasetException;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
|
||||
/**
|
||||
* Manages all file system level interactions for the Hoodie tables.
|
||||
*/
|
||||
public class HoodieTableMetadata implements Serializable {
|
||||
public static final String MAX_COMMIT_TS = String.valueOf(Long.MAX_VALUE);
|
||||
public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name";
|
||||
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
|
||||
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
|
||||
|
||||
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
|
||||
private static final String HOODIE_HDRONE_PROFILE_DEFAULT_VALUE = "HOODIE";
|
||||
private static final java.lang.String HOODIE_HDRONE_PROFILE_PROP_NAME =
|
||||
"hoodie.hdrone.dataset.profile";
|
||||
|
||||
private static Logger log = LogManager.getLogger(HoodieTableMetadata.class);
|
||||
private transient final FileSystem fs;
|
||||
private transient final Path metadataFolder;
|
||||
private final Properties properties;
|
||||
private HoodieCommits commits;
|
||||
private List<String> inflightCommits;
|
||||
private String basePath;
|
||||
|
||||
public static final String METAFOLDER_NAME = ".hoodie";
|
||||
public static final String COMMIT_FILE_SUFFIX = ".commit";
|
||||
public static final String INFLIGHT_FILE_SUFFIX = ".inflight";
|
||||
|
||||
/**
|
||||
* Constructor which initializes the hoodie table metadata. It will initialize the meta-data if not already present.
|
||||
*
|
||||
* @param fs
|
||||
* @param basePath
|
||||
* @param tableName
|
||||
*/
|
||||
public HoodieTableMetadata(FileSystem fs, String basePath, String tableName) {
|
||||
this(fs, basePath, tableName, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructor which loads the hoodie table metadata, It requires the meta-data to be present already
|
||||
* @param fs
|
||||
* @param basePath
|
||||
*/
|
||||
public HoodieTableMetadata(FileSystem fs, String basePath) {
|
||||
this(fs, basePath, null, false);
|
||||
}
|
||||
|
||||
private HoodieTableMetadata(FileSystem fs, String basePath, String tableName,
|
||||
boolean initOnMissing) {
|
||||
this.fs = fs;
|
||||
this.basePath = basePath;
|
||||
|
||||
try {
|
||||
Path basePathDir = new Path(this.basePath);
|
||||
if (!fs.exists(basePathDir)) {
|
||||
if (initOnMissing) {
|
||||
fs.mkdirs(basePathDir);
|
||||
} else {
|
||||
throw new DatasetNotFoundException(this.basePath);
|
||||
}
|
||||
}
|
||||
|
||||
if (!fs.isDirectory(new Path(basePath))) {
|
||||
throw new DatasetNotFoundException(this.basePath);
|
||||
}
|
||||
|
||||
this.metadataFolder = new Path(this.basePath, METAFOLDER_NAME);
|
||||
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
|
||||
if (!fs.exists(propertyPath)) {
|
||||
if (initOnMissing) {
|
||||
// create .hoodie folder if it does not exist.
|
||||
createHoodieProperties(metadataFolder, tableName);
|
||||
} else {
|
||||
throw new InvalidDatasetException(this.basePath);
|
||||
}
|
||||
}
|
||||
|
||||
// Load meta data
|
||||
this.commits = new HoodieCommits(scanCommits(COMMIT_FILE_SUFFIX));
|
||||
this.inflightCommits = scanCommits(INFLIGHT_FILE_SUFFIX);
|
||||
this.properties = readHoodieProperties();
|
||||
log.info("All commits :" + commits);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not load HoodieMetadata from path " + basePath, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the commit metadata for this table. Reads all the commit files from HDFS.
|
||||
* Expensive operation, use with caution.
|
||||
*
|
||||
* @return SortedMap of CommitTime,HoodieCommitMetadata
|
||||
*/
|
||||
public SortedMap<String, HoodieCommitMetadata> getAllCommitMetadata() {
|
||||
try {
|
||||
TreeMap<String, HoodieCommitMetadata> metadataMap = new TreeMap<>();
|
||||
for (String commitTs : commits.getCommitList()) {
|
||||
metadataMap.put(commitTs, getCommitMetadata(commitTs));
|
||||
}
|
||||
return Collections.unmodifiableSortedMap(metadataMap);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not load all commits for table " + getTableName(),
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieCommitMetadata getCommitMetadata(String commitTime) throws IOException {
|
||||
FSDataInputStream is = fs.open(new Path(metadataFolder, FSUtils.makeCommitFileName(commitTime)));
|
||||
try {
|
||||
String jsonStr = IOUtils.toString(is);
|
||||
return HoodieCommitMetadata.fromJsonString(jsonStr);
|
||||
} finally {
|
||||
is.close();
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieTableType getTableType() {
|
||||
return HoodieTableType.valueOf(properties.getProperty(HOODIE_TABLE_TYPE_PROP_NAME));
|
||||
}
|
||||
|
||||
/**
|
||||
* Lookup the file name for specified <code>HoodieRecord</code>
|
||||
*
|
||||
* TODO(vc): This metadata needs to be cached in each executor, statically, and used across, if
|
||||
* we need to be nicer to the NameNode
|
||||
*/
|
||||
public String getFilenameForRecord(FileSystem fs, final HoodieRecord record) {
|
||||
String fileId = record.getCurrentLocation().getFileId();
|
||||
return getFilenameForRecord(fs, record, fileId);
|
||||
}
|
||||
|
||||
|
||||
public String getFilenameForRecord(FileSystem fs, final HoodieRecord record, String fileId) {
|
||||
try {
|
||||
FileStatus[] files = fs.listStatus(new Path(basePath, record.getPartitionPath()));
|
||||
Map<String, List<FileStatus>> fileIdToVersions =
|
||||
groupFilesByFileId(files, commits.lastCommit());
|
||||
// If the record is not found
|
||||
if(!fileIdToVersions.containsKey(fileId)) {
|
||||
throw new FileNotFoundException("Cannot find valid versions for fileId " + fileId);
|
||||
}
|
||||
|
||||
List<FileStatus> statuses = fileIdToVersions.get(fileId);
|
||||
return statuses.get(0).getPath().getName();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not get Filename for record " + record, e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Get only the latest file in the partition with precondition commitTime(file) lt maxCommitTime
|
||||
*
|
||||
* @param fs
|
||||
* @param partitionPathStr
|
||||
* @param maxCommitTime
|
||||
* @return
|
||||
*/
|
||||
public FileStatus[] getLatestVersionInPartition(FileSystem fs, String partitionPathStr,
|
||||
String maxCommitTime) {
|
||||
try {
|
||||
Path partitionPath = new Path(basePath, partitionPathStr);
|
||||
if(!fs.exists(partitionPath)) {
|
||||
return new FileStatus[0];
|
||||
}
|
||||
FileStatus[] files = fs.listStatus(partitionPath);
|
||||
Map<String, List<FileStatus>> fileIdToVersions =
|
||||
groupFilesByFileId(files, commits.lastCommit());
|
||||
HashMap<String, FileStatus> validFiles = new HashMap<>();
|
||||
for (String fileId : fileIdToVersions.keySet()) {
|
||||
List<FileStatus> versions = fileIdToVersions.get(fileId);
|
||||
for (FileStatus file : versions) {
|
||||
String filename = file.getPath().getName();
|
||||
String commitTime = FSUtils.getCommitTime(filename);
|
||||
if (HoodieCommits.isCommit1BeforeOrOn(commitTime, maxCommitTime)) {
|
||||
validFiles.put(fileId, file);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return validFiles.values().toArray(new FileStatus[validFiles.size()]);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not get latest versions in Partition " + partitionPathStr, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get ALL the data files in partition grouped by fileId and sorted by the commitTime
|
||||
* Given a partition path, provide all the files with a list of their commits, sorted by commit time.
|
||||
*/
|
||||
public Map<String, List<FileStatus>> getAllVersionsInPartition(FileSystem fs, String partitionPath) {
|
||||
try {
|
||||
FileStatus[] files = fs.listStatus(new Path(basePath, partitionPath));
|
||||
return groupFilesByFileId(files, commits.lastCommit());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not load all file versions in partition " + partitionPath, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all the versions of files, within the commit range provided.
|
||||
*
|
||||
* @param commitsToReturn - commits to include
|
||||
*/
|
||||
public FileStatus[] getLatestVersionInRange(FileStatus[] fileStatuses, List<String> commitsToReturn) {
|
||||
if (commitsToReturn.isEmpty()) {
|
||||
return new FileStatus[0];
|
||||
}
|
||||
try {
|
||||
Map<String, List<FileStatus>> fileIdToVersions =
|
||||
groupFilesByFileId(fileStatuses, commits.lastCommit());
|
||||
|
||||
List<FileStatus> statuses = new ArrayList<>();
|
||||
for (List<FileStatus> entry : fileIdToVersions.values()) {
|
||||
for (FileStatus status : entry) {
|
||||
String commitTime = FSUtils.getCommitTime(status.getPath().getName());
|
||||
if (commitsToReturn.contains(commitTime)) {
|
||||
statuses.add(status);
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return statuses.toArray(new FileStatus[statuses.size()]);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not filter files from commits " + commitsToReturn, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* Get the latest versions of all the files.
|
||||
*
|
||||
* @param fileStatuses
|
||||
* @return
|
||||
*/
|
||||
public FileStatus[] getLatestVersions(FileStatus[] fileStatuses) {
|
||||
try {
|
||||
Map<String, List<FileStatus>> fileIdToVersions =
|
||||
groupFilesByFileId(fileStatuses, commits.lastCommit());
|
||||
|
||||
List<FileStatus> statuses = new ArrayList<>();
|
||||
for(List<FileStatus> entry:fileIdToVersions.values()) {
|
||||
// first file is the latest one
|
||||
statuses.add(entry.get(0));
|
||||
}
|
||||
return statuses.toArray(new FileStatus[statuses.size()]);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not filter files for latest version ", e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the base path for the Hoodie Table
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public String getBasePath() {
|
||||
return basePath;
|
||||
}
|
||||
|
||||
|
||||
public boolean isCommitsEmpty() {
|
||||
return commits.isEmpty();
|
||||
}
|
||||
|
||||
public boolean isCommitTsSafe(String commitTs) {
|
||||
return !isCommitsEmpty() && (commits.isCommitBeforeEarliestCommit(commitTs) || commits
|
||||
.contains(commitTs));
|
||||
}
|
||||
|
||||
public List<String> findCommitsSinceTs(String startTs) {
|
||||
return commits.findCommitsInRange(startTs, MAX_COMMIT_TS);
|
||||
}
|
||||
|
||||
public List<String> findCommitsInRange(String startTs, String endTs) {
|
||||
return commits.findCommitsInRange(startTs, endTs);
|
||||
}
|
||||
|
||||
public List<String> findCommitsAfter(String startTs, Integer maxCommits) {
|
||||
return commits.findCommitsAfter(startTs, maxCommits);
|
||||
}
|
||||
|
||||
public HoodieCommits getAllCommits() {
|
||||
return commits;
|
||||
}
|
||||
|
||||
public List<String> getAllInflightCommits() {
|
||||
return inflightCommits;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieTableMetadata{");
|
||||
sb.append("commits=").append(commits);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public String getTableName() {
|
||||
return properties.getProperty(HOODIE_TABLE_NAME_PROP_NAME);
|
||||
}
|
||||
|
||||
public String getHDroneDatasetProfile() {
|
||||
return properties.getProperty(HOODIE_HDRONE_PROFILE_PROP_NAME, HOODIE_HDRONE_PROFILE_DEFAULT_VALUE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialize the hoodie meta directory and any necessary files inside the meta (including the hoodie.properties)
|
||||
*
|
||||
* @param metadataFolder
|
||||
* @param tableName
|
||||
* @throws IOException
|
||||
*/
|
||||
private void createHoodieProperties(Path metadataFolder, String tableName) throws IOException {
|
||||
if (!fs.exists(metadataFolder)) {
|
||||
fs.mkdirs(metadataFolder);
|
||||
}
|
||||
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
|
||||
FSDataOutputStream outputStream = fs.create(propertyPath);
|
||||
try {
|
||||
Properties props = new Properties();
|
||||
props.setProperty(HOODIE_TABLE_NAME_PROP_NAME, tableName);
|
||||
props.setProperty(HOODIE_TABLE_TYPE_PROP_NAME, DEFAULT_TABLE_TYPE.name());
|
||||
props
|
||||
.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
|
||||
} finally {
|
||||
outputStream.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Loads the hoodie table properties from the hoodie.properties file under the .hoodie path
|
||||
*/
|
||||
private Properties readHoodieProperties() throws IOException {
|
||||
Properties props = new Properties();
|
||||
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
|
||||
FSDataInputStream inputStream = fs.open(propertyPath);
|
||||
try {
|
||||
props.load(inputStream);
|
||||
} finally {
|
||||
inputStream.close();
|
||||
}
|
||||
return props;
|
||||
}
|
||||
|
||||
/**
|
||||
* Scan the commit times (only choosing commit file with the given suffix)
|
||||
*/
|
||||
private List<String> scanCommits(final String commitFileSuffix) throws IOException {
|
||||
log.info("Attempting to load the commits under " + metadataFolder + " with suffix " + commitFileSuffix);
|
||||
final List<String> commitFiles = new ArrayList<>();
|
||||
fs.listStatus(metadataFolder, new PathFilter() {
|
||||
@Override
|
||||
public boolean accept(Path path) {
|
||||
if (path.getName().endsWith(commitFileSuffix)) {
|
||||
commitFiles.add(path.getName().split("\\.")[0]);
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
});
|
||||
return commitFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Takes a bunch of file versions, and returns a map keyed by fileId, with the necessary
|
||||
* version safety checking. Returns a map of commitTime and Sorted list of FileStats
|
||||
* ( by reverse commit time )
|
||||
*
|
||||
* @param maxCommitTime maximum permissible commit time
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
private Map<String, List<FileStatus>> groupFilesByFileId(FileStatus[] files,
|
||||
String maxCommitTime) throws IOException {
|
||||
HashMap<String, List<FileStatus>> fileIdtoVersions = new HashMap<>();
|
||||
for (FileStatus file : files) {
|
||||
String filename = file.getPath().getName();
|
||||
String fileId = FSUtils.getFileId(filename);
|
||||
String commitTime = FSUtils.getCommitTime(filename);
|
||||
if (isCommitTsSafe(commitTime) && HoodieCommits
|
||||
.isCommit1BeforeOrOn(commitTime, maxCommitTime)) {
|
||||
if (!fileIdtoVersions.containsKey(fileId)) {
|
||||
fileIdtoVersions.put(fileId, new ArrayList<FileStatus>());
|
||||
}
|
||||
fileIdtoVersions.get(fileId).add(file);
|
||||
}
|
||||
}
|
||||
for (Map.Entry<String, List<FileStatus>> entry : fileIdtoVersions.entrySet()) {
|
||||
Collections.sort(fileIdtoVersions.get(entry.getKey()), new Comparator<FileStatus>() {
|
||||
@Override
|
||||
public int compare(FileStatus o1, FileStatus o2) {
|
||||
String o1CommitTime = FSUtils.getCommitTime(o1.getPath().getName());
|
||||
String o2CommitTime = FSUtils.getCommitTime(o2.getPath().getName());
|
||||
// Reverse the order
|
||||
return o2CommitTime.compareTo(o1CommitTime);
|
||||
}
|
||||
});
|
||||
}
|
||||
return fileIdtoVersions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o)
|
||||
return true;
|
||||
if (o == null || getClass() != o.getClass())
|
||||
return false;
|
||||
|
||||
HoodieTableMetadata metadata = (HoodieTableMetadata) o;
|
||||
|
||||
if (commits != null ? !commits.equals(metadata.commits) : metadata.commits != null)
|
||||
return false;
|
||||
return basePath != null ? basePath.equals(metadata.basePath) : metadata.basePath == null;
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = commits != null ? commits.hashCode() : 0;
|
||||
result = 31 * result + (basePath != null ? basePath.hashCode() : 0);
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -0,0 +1,138 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieStorageType;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Date;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Configurations on the Hoodie Table like type of ingestion, storage formats, hive table name etc
|
||||
* Configurations are loaded from hoodie.properties, these properties are usually set during initializing a path as hoodie base path
|
||||
* and never changes during the lifetime of a hoodie dataset.
|
||||
*
|
||||
* @see HoodieTableMetaClient
|
||||
* @since 0.3.0
|
||||
*/
|
||||
public class HoodieTableConfig implements Serializable {
|
||||
private final transient static Logger log = LogManager.getLogger(HoodieTableConfig.class);
|
||||
|
||||
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
|
||||
public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name";
|
||||
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
|
||||
public static final String HOODIE_RO_STORAGE_FORMAT_PROP_NAME =
|
||||
"hoodie.table.ro.storage.format";
|
||||
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
|
||||
public static final HoodieStorageType DEFAULT_RO_STORAGE_FORMAT = HoodieStorageType.PARQUET;
|
||||
private Properties props;
|
||||
|
||||
public HoodieTableConfig(FileSystem fs, String metaPath) {
|
||||
Properties props = new Properties();
|
||||
Path propertyPath = new Path(metaPath, HOODIE_PROPERTIES_FILE);
|
||||
log.info("Loading dataset properties from " + propertyPath);
|
||||
try {
|
||||
try (FSDataInputStream inputStream = fs.open(propertyPath)) {
|
||||
props.load(inputStream);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not load Hoodie properties from " + propertyPath, e);
|
||||
}
|
||||
this.props = props;
|
||||
}
|
||||
|
||||
/**
|
||||
* For serailizing and de-serializing
|
||||
* @deprecated
|
||||
*/
|
||||
public HoodieTableConfig() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialize the hoodie meta directory and any necessary files inside the meta (including the hoodie.properties)
|
||||
*
|
||||
* @param metadataFolder
|
||||
* @param properties
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void createHoodieProperties(FileSystem fs, Path metadataFolder,
|
||||
Properties properties) throws IOException {
|
||||
if (!fs.exists(metadataFolder)) {
|
||||
fs.mkdirs(metadataFolder);
|
||||
}
|
||||
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
|
||||
FSDataOutputStream outputStream = fs.create(propertyPath);
|
||||
try {
|
||||
if (!properties.containsKey(HOODIE_TABLE_NAME_PROP_NAME)) {
|
||||
throw new IllegalArgumentException(
|
||||
HOODIE_TABLE_NAME_PROP_NAME + " property needs to be specified");
|
||||
}
|
||||
if (!properties.containsKey(HOODIE_TABLE_TYPE_PROP_NAME)) {
|
||||
properties.setProperty(HOODIE_TABLE_TYPE_PROP_NAME, DEFAULT_TABLE_TYPE.name());
|
||||
}
|
||||
properties
|
||||
.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
|
||||
} finally {
|
||||
outputStream.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Read the table type from the table properties and if not found, return the default
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public HoodieTableType getTableType() {
|
||||
if (props.contains(HOODIE_TABLE_TYPE_PROP_NAME)) {
|
||||
return HoodieTableType.valueOf(props.getProperty(HOODIE_TABLE_TYPE_PROP_NAME));
|
||||
}
|
||||
return DEFAULT_TABLE_TYPE;
|
||||
}
|
||||
|
||||
/**
|
||||
* Read the table name
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public String getTableName() {
|
||||
return props.getProperty(HOODIE_TABLE_NAME_PROP_NAME);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the Read Optimized Storage Format
|
||||
*
|
||||
* @return HoodieStorageType for the Read Optimized Storage format
|
||||
*/
|
||||
public HoodieStorageType getROStorageFormat() {
|
||||
if (props.contains(HOODIE_RO_STORAGE_FORMAT_PROP_NAME)) {
|
||||
return HoodieStorageType.valueOf(props.getProperty(HOODIE_RO_STORAGE_FORMAT_PROP_NAME));
|
||||
}
|
||||
return DEFAULT_RO_STORAGE_FORMAT;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,275 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveCommitTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieArchivedCommitTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieCleanerTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieSavePointTimeline;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.DatasetNotFoundException;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Objects;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* <code>HoodieTableMetaClient</code> allows to access meta-data about a hoodie table
|
||||
* It returns meta-data about commits, savepoints, compactions, cleanups as a <code>HoodieTimeline</code>
|
||||
* Create an instance of the <code>HoodieTableMetaClient</code> with FileSystem and basePath to start getting the meta-data.
|
||||
* <p>
|
||||
* All the timelines are computed lazily, once computed the timeline is cached and never refreshed.
|
||||
* Use the <code>HoodieTimeline.reload()</code> to refresh timelines.
|
||||
*
|
||||
* @see HoodieTimeline
|
||||
* @since 0.3.0
|
||||
*/
|
||||
public class HoodieTableMetaClient implements Serializable {
|
||||
private final transient static Logger log = LogManager.getLogger(HoodieTableMetaClient.class);
|
||||
public static String METAFOLDER_NAME = ".hoodie";
|
||||
public static String COMMIT_EXTENSION = ".commit";
|
||||
public static String CLEAN_EXTENSION = ".clean";
|
||||
public static String SAVEPOINT_EXTENSION = ".savepoint";
|
||||
public static String INFLIGHT_FILE_SUFFIX = ".inflight";
|
||||
|
||||
private String basePath;
|
||||
private transient FileSystem fs;
|
||||
private String metaPath;
|
||||
private HoodieTableType tableType;
|
||||
private HoodieTableConfig tableConfig;
|
||||
private HoodieTimeline activeCommitTimeline;
|
||||
private HoodieTimeline archivedCommitTimeline;
|
||||
private HoodieTimeline savePointTimeline;
|
||||
private HoodieTimeline cleanerTimeline;
|
||||
|
||||
public HoodieTableMetaClient(FileSystem fs, String basePath) throws DatasetNotFoundException {
|
||||
// Do not load any timeline by default
|
||||
this(fs, basePath, false);
|
||||
}
|
||||
|
||||
public HoodieTableMetaClient(FileSystem fs, String basePath,
|
||||
boolean loadActiveCommitTimelineOnLoad) throws DatasetNotFoundException {
|
||||
log.info("Loading HoodieTableMetaClient from " + basePath);
|
||||
this.basePath = basePath;
|
||||
this.fs = fs;
|
||||
Path basePathDir = new Path(this.basePath);
|
||||
this.metaPath = basePath + File.separator + METAFOLDER_NAME;
|
||||
Path metaPathDir = new Path(this.metaPath);
|
||||
DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir);
|
||||
this.tableConfig = new HoodieTableConfig(fs, metaPath);
|
||||
this.tableType = tableConfig.getTableType();
|
||||
log.info("Finished Loading Table of type " + tableType + " from " + basePath);
|
||||
if (loadActiveCommitTimelineOnLoad) {
|
||||
log.info("Loading Active commit timeline for " + basePath);
|
||||
getActiveCommitTimeline();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* For serailizing and de-serializing
|
||||
* @deprecated
|
||||
*/
|
||||
public HoodieTableMetaClient() {
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is only used when this object is deserialized in a spark executor.
|
||||
* @deprecated
|
||||
*/
|
||||
private void readObject(java.io.ObjectInputStream in)
|
||||
throws IOException, ClassNotFoundException {
|
||||
in.defaultReadObject();
|
||||
this.fs = FSUtils.getFs();
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Base path
|
||||
*/
|
||||
public String getBasePath() {
|
||||
return basePath;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Hoodie Table Type
|
||||
*/
|
||||
public HoodieTableType getTableType() {
|
||||
return tableType;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Meta path
|
||||
*/
|
||||
public String getMetaPath() {
|
||||
return metaPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Table Config
|
||||
*/
|
||||
public HoodieTableConfig getTableConfig() {
|
||||
return tableConfig;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the active commits as a timeline
|
||||
*
|
||||
* @return Active commit timeline
|
||||
* @throws IOException
|
||||
*/
|
||||
public synchronized HoodieTimeline getActiveCommitTimeline() {
|
||||
if (activeCommitTimeline == null) {
|
||||
activeCommitTimeline = new HoodieActiveCommitTimeline(fs, metaPath);
|
||||
}
|
||||
return activeCommitTimeline;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the archived commits as a timeline. This is costly operation, as all data from the
|
||||
* archived files are read. This should not be used, unless for historical debugging purposes
|
||||
*
|
||||
* @return Active commit timeline
|
||||
* @throws IOException
|
||||
*/
|
||||
public HoodieTimeline getArchivedCommitTimeline() {
|
||||
if (archivedCommitTimeline == null) {
|
||||
archivedCommitTimeline = new HoodieArchivedCommitTimeline(fs, metaPath);
|
||||
}
|
||||
return archivedCommitTimeline;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the save points as a timeline.
|
||||
*
|
||||
* @return Savepoint timeline
|
||||
* @throws IOException
|
||||
*/
|
||||
public HoodieTimeline getSavePointsTimeline() {
|
||||
if (savePointTimeline == null) {
|
||||
savePointTimeline = new HoodieSavePointTimeline(fs, metaPath);
|
||||
}
|
||||
return savePointTimeline;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the cleaner activity as a timeline.
|
||||
*
|
||||
* @return Cleaner activity
|
||||
* @throws IOException
|
||||
*/
|
||||
public HoodieTimeline getCleanerTimeline() {
|
||||
if (cleanerTimeline == null) {
|
||||
cleanerTimeline = new HoodieCleanerTimeline(fs, metaPath);
|
||||
}
|
||||
return cleanerTimeline;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Helper method to initialize a given path as a hoodie dataset with configs passed in as as Properties
|
||||
*
|
||||
* @param fs
|
||||
* @param basePath
|
||||
* @param props
|
||||
* @return Instance of HoodieTableMetaClient
|
||||
* @throws IOException
|
||||
*/
|
||||
public static HoodieTableMetaClient initializePathAsHoodieDataset(FileSystem fs,
|
||||
String basePath, Properties props) throws IOException {
|
||||
log.info("Initializing " + basePath + " as hoodie dataset " + basePath);
|
||||
Path basePathDir = new Path(basePath);
|
||||
if (!fs.exists(basePathDir)) {
|
||||
fs.mkdirs(basePathDir);
|
||||
}
|
||||
Path metaPathDir = new Path(basePath, METAFOLDER_NAME);
|
||||
if (!fs.exists(metaPathDir)) {
|
||||
fs.mkdirs(metaPathDir);
|
||||
}
|
||||
HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||
log.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType()
|
||||
+ " from " + basePath);
|
||||
return metaClient;
|
||||
}
|
||||
|
||||
// HELPER METHODS TO CREATE META FILE NAMES
|
||||
public static String makeCommitFileName(String commitTime) {
|
||||
return commitTime + COMMIT_EXTENSION;
|
||||
}
|
||||
|
||||
public static String makeInflightCommitFileName(String commitTime) {
|
||||
return commitTime + INFLIGHT_FILE_SUFFIX;
|
||||
}
|
||||
|
||||
public static String makeCleanerFileName(String instant) {
|
||||
return instant + CLEAN_EXTENSION;
|
||||
}
|
||||
|
||||
public static String makeInflightCleanerFileName(String instant) {
|
||||
return instant + CLEAN_EXTENSION + INFLIGHT_FILE_SUFFIX;
|
||||
}
|
||||
|
||||
public static String makeInflightSavePointFileName(String commitTime) {
|
||||
return commitTime + SAVEPOINT_EXTENSION + INFLIGHT_FILE_SUFFIX;
|
||||
}
|
||||
|
||||
public static String makeSavePointFileName(String commitTime) {
|
||||
return commitTime + SAVEPOINT_EXTENSION;
|
||||
}
|
||||
|
||||
public static String getCommitFromCommitFile(String commitFileName) {
|
||||
return commitFileName.split("\\.")[0];
|
||||
}
|
||||
|
||||
public static FileStatus[] scanFiles(FileSystem fs, Path metaPath, PathFilter nameFilter)
|
||||
throws IOException {
|
||||
return fs.listStatus(metaPath, nameFilter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o)
|
||||
return true;
|
||||
if (o == null || getClass() != o.getClass())
|
||||
return false;
|
||||
HoodieTableMetaClient that = (HoodieTableMetaClient) o;
|
||||
return Objects.equals(basePath, that.basePath) && tableType == that.tableType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(basePath, tableType);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieTableMetaClient{");
|
||||
sb.append("basePath='").append(basePath).append('\'');
|
||||
sb.append(", metaPath='").append(metaPath).append('\'');
|
||||
sb.append(", tableType=").append(tableType);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,190 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table;
|
||||
|
||||
import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Optional;
|
||||
import java.util.function.BiPredicate;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* HoodieTimeline allows representation of meta-data events as a timeline.
|
||||
* Instants are specific points in time represented as strings.
|
||||
* in this format YYYYMMDDHHmmSS. e.g. 20170101193218
|
||||
* Any operation on the timeline starts with the inflight instant and then when complete marks
|
||||
* the completed instant and removes the inflight instant.
|
||||
* Completed instants are plainly referred to as just instants
|
||||
* <p>
|
||||
* Timelines as immutable once created. Any operation to change the timeline (like create/delete instants)
|
||||
* will not be reflected unless explicitly reloaded using the reload()
|
||||
*
|
||||
* @see com.uber.hoodie.common.table.HoodieTableMetaClient
|
||||
* @see HoodieDefaultTimeline
|
||||
* @since 0.3.0
|
||||
*/
|
||||
public interface HoodieTimeline extends Serializable {
|
||||
/**
|
||||
* Find all the completed instants after startTs and before or on endTs
|
||||
*
|
||||
* @param startTs
|
||||
* @param endTs
|
||||
* @return Stream of instants
|
||||
*/
|
||||
Stream<String> findInstantsInRange(String startTs, String endTs);
|
||||
|
||||
/**
|
||||
* Find all the completed instants after startTs
|
||||
*
|
||||
* @param commitTime
|
||||
* @param numCommits
|
||||
* @return Stream of instants
|
||||
*/
|
||||
Stream<String> findInstantsAfter(String commitTime, int numCommits);
|
||||
|
||||
/**
|
||||
* If the timeline has any completed instants
|
||||
*
|
||||
* @return true if timeline is not empty
|
||||
*/
|
||||
boolean hasInstants();
|
||||
|
||||
/**
|
||||
* If the timeline has any in-complete instants
|
||||
*
|
||||
* @return true if timeline has any in-complete instants
|
||||
*/
|
||||
boolean hasInflightInstants();
|
||||
|
||||
/**
|
||||
* @return total number of completed instants
|
||||
*/
|
||||
int getTotalInstants();
|
||||
|
||||
/**
|
||||
* @return first completed instant if available
|
||||
*/
|
||||
Optional<String> firstInstant();
|
||||
|
||||
/**
|
||||
* @param n
|
||||
* @return nth completed instant from the first completed instant
|
||||
*/
|
||||
Optional<String> nthInstant(int n);
|
||||
|
||||
/**
|
||||
* @return last completed instant if available
|
||||
*/
|
||||
Optional<String> lastInstant();
|
||||
|
||||
/**
|
||||
* @param n
|
||||
* @return nth completed instant going back from the last completed instant
|
||||
*/
|
||||
Optional<String> nthFromLastInstant(int n);
|
||||
|
||||
/**
|
||||
* @return true if the passed instant is present as a completed instant on the timeline
|
||||
*/
|
||||
boolean containsInstant(String instant);
|
||||
|
||||
/**
|
||||
* @return true if the passed instant is present as a completed instant on the timeline or
|
||||
* if the instant is before the first completed instant in the timeline
|
||||
*/
|
||||
boolean containsOrBeforeTimelineStarts(String instant);
|
||||
|
||||
/**
|
||||
* @return Get the stream of completed instants
|
||||
*/
|
||||
Stream<String> getInstants();
|
||||
|
||||
/**
|
||||
* @return Get the stream of in-flight instants
|
||||
*/
|
||||
Stream<String> getInflightInstants();
|
||||
|
||||
/**
|
||||
* @return true if the passed in instant is before the first completed instant in the timeline
|
||||
*/
|
||||
boolean isInstantBeforeTimelineStarts(String instant);
|
||||
|
||||
/**
|
||||
* Register the passed in instant as a in-flight
|
||||
*
|
||||
* @param instant
|
||||
*/
|
||||
void saveInstantAsInflight(String instant);
|
||||
|
||||
/**
|
||||
* Register the passed in instant as a completed instant.
|
||||
* It needs to have a corresponding in-flight instant, otherwise it will fail.
|
||||
* Pass a optional byte[] to save with the instant.
|
||||
*
|
||||
* @param instant
|
||||
* @param data
|
||||
*/
|
||||
void saveInstantAsComplete(String instant, Optional<byte[]> data);
|
||||
|
||||
/**
|
||||
* Un-Register a completed instant as in-flight. This is usually atomic way to
|
||||
* revert the effects of a operation on hoodie datasets
|
||||
*
|
||||
* @param instant
|
||||
*/
|
||||
void revertInstantToInflight(String instant);
|
||||
|
||||
/**
|
||||
* Remove the in-flight instant from the timeline
|
||||
*
|
||||
* @param instant
|
||||
*/
|
||||
void removeInflightFromTimeline(String instant);
|
||||
|
||||
/**
|
||||
* Reload the timeline. Timelines are immutable once created.
|
||||
*
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
HoodieTimeline reload() throws IOException;
|
||||
|
||||
/**
|
||||
* Read the completed instant details
|
||||
*
|
||||
* @param instant
|
||||
* @return
|
||||
*/
|
||||
Optional<byte[]> readInstantDetails(String instant);
|
||||
|
||||
/**
|
||||
* Helper methods to compare instants
|
||||
**/
|
||||
BiPredicate<String, String> GREATER_OR_EQUAL =
|
||||
(commit1, commit2) -> commit1.compareTo(commit2) >= 0;
|
||||
BiPredicate<String, String> GREATER = (commit1, commit2) -> commit1.compareTo(commit2) > 0;
|
||||
BiPredicate<String, String> LESSER_OR_EQUAL =
|
||||
(commit1, commit2) -> commit1.compareTo(commit2) <= 0;
|
||||
BiPredicate<String, String> LESSER = (commit1, commit2) -> commit1.compareTo(commit2) < 0;
|
||||
|
||||
default boolean compareInstants(String commit1, String commit2,
|
||||
BiPredicate<String, String> predicateToApply) {
|
||||
return predicateToApply.test(commit1, commit2);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,93 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Interface for viewing the table file system.
|
||||
* Dependening on the Hoodie Table Type - The view of the filesystem changes.
|
||||
* <p>
|
||||
* ReadOptimizedView - Lets queries run only on organized columnar data files at the expense of latency
|
||||
* WriteOptimizedView - Lets queries run on columnar data as well as delta files (sequential) at the expense of query execution time
|
||||
* @since 0.3.0
|
||||
*/
|
||||
public interface TableFileSystemView {
|
||||
/**
|
||||
* Stream all the data files for a specific FileId.
|
||||
* This usually has a single RO file and multiple WO files if present.
|
||||
*
|
||||
* @param partitionPath
|
||||
* @param fileId
|
||||
* @return
|
||||
*/
|
||||
Stream<HoodieDataFile> getLatestDataFilesForFileId(final String partitionPath,
|
||||
final String fileId);
|
||||
|
||||
/**
|
||||
* Stream all the latest version data files in the given partition
|
||||
* with precondition that commitTime(file) before maxCommitTime
|
||||
*
|
||||
* @param partitionPathStr
|
||||
* @param maxCommitTime
|
||||
* @return
|
||||
*/
|
||||
Stream<HoodieDataFile> streamLatestVersionInPartition(String partitionPathStr,
|
||||
String maxCommitTime);
|
||||
|
||||
/**
|
||||
* Stream all the data file versions grouped by FileId for a given partition
|
||||
*
|
||||
* @param partitionPath
|
||||
* @return
|
||||
*/
|
||||
Stream<List<HoodieDataFile>> streamEveryVersionInPartition(String partitionPath);
|
||||
|
||||
/**
|
||||
* Stream all the versions from the passed in fileStatus[] with commit times containing in commitsToReturn.
|
||||
*
|
||||
* @param fileStatuses
|
||||
* @param commitsToReturn
|
||||
* @return
|
||||
*/
|
||||
Stream<HoodieDataFile> streamLatestVersionInRange(FileStatus[] fileStatuses,
|
||||
List<String> commitsToReturn);
|
||||
|
||||
/**
|
||||
* Stream the latest version from the passed in FileStatus[] with commit times less than maxCommitToReturn
|
||||
*
|
||||
* @param fileStatuses
|
||||
* @param maxCommitToReturn
|
||||
* @return
|
||||
*/
|
||||
Stream<HoodieDataFile> streamLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
|
||||
String maxCommitToReturn);
|
||||
|
||||
/**
|
||||
* Stream latest versions from the passed in FileStatus[].
|
||||
* Similar to calling streamLatestVersionsBeforeOrOn(fileStatuses, currentTimeAsCommitTime)
|
||||
*
|
||||
* @param fileStatuses
|
||||
* @return
|
||||
*/
|
||||
Stream<HoodieDataFile> streamLatestVersions(FileStatus[] fileStatuses);
|
||||
}
|
||||
@@ -0,0 +1,77 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table.timeline;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Active commit timeline. Much optimized for reading than the ArchivedTimeline.
|
||||
*/
|
||||
public class HoodieActiveCommitTimeline extends HoodieDefaultTimeline {
|
||||
public HoodieActiveCommitTimeline(FileSystem fs, String metaPath) {
|
||||
super(fs, metaPath);
|
||||
String completedInstantExtension = HoodieTableMetaClient.COMMIT_EXTENSION;
|
||||
String inflightInstantExtension = INFLIGHT_EXTENSION;
|
||||
|
||||
FileStatus[] fileStatuses;
|
||||
try {
|
||||
fileStatuses = HoodieTableMetaClient.scanFiles(fs, new Path(metaPath),
|
||||
path -> path.toString().endsWith(completedInstantExtension) || path.toString()
|
||||
.endsWith(inflightInstantExtension));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to scan metadata", e);
|
||||
}
|
||||
this.instants = Arrays.stream(fileStatuses)
|
||||
.filter(status -> status.getPath().getName().endsWith(completedInstantExtension))
|
||||
.map(fileStatus -> fileStatus.getPath().getName().replaceAll(completedInstantExtension, ""))
|
||||
.sorted().collect(Collectors.toList());
|
||||
this.inflights = Arrays.stream(fileStatuses).filter(
|
||||
status -> status.getPath().getName().endsWith(inflightInstantExtension)).map(
|
||||
fileStatus -> fileStatus.getPath().getName()
|
||||
.replaceAll(inflightInstantExtension, "")).sorted()
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getInflightFileName(String instant) {
|
||||
return HoodieTableMetaClient.makeInflightCommitFileName(instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getCompletedFileName(String instant) {
|
||||
return HoodieTableMetaClient.makeCommitFileName(instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getTimelineName() {
|
||||
return "commit";
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline reload() throws IOException {
|
||||
return new HoodieActiveCommitTimeline(fs, metaPath);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,121 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table.timeline;
|
||||
|
||||
import com.google.common.io.Closeables;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Archived commit timeline. These commits are usually cleaned up and the meta data is archived for
|
||||
* future triaging
|
||||
*
|
||||
* @since 0.3.0
|
||||
*/
|
||||
public class HoodieArchivedCommitTimeline extends HoodieDefaultTimeline {
|
||||
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits.archived";
|
||||
private transient Map<String, byte[]> readCommits = new HashMap<>();
|
||||
|
||||
public HoodieArchivedCommitTimeline(FileSystem fs, String metaPath) {
|
||||
// Read back the commits to make sure
|
||||
Path archiveLogPath = getArchiveLogPath(metaPath);
|
||||
try {
|
||||
SequenceFile.Reader reader =
|
||||
new SequenceFile.Reader(fs.getConf(), SequenceFile.Reader.file(archiveLogPath));
|
||||
try {
|
||||
Text key = new Text();
|
||||
Text val = new Text();
|
||||
while (reader.next(key, val)) {
|
||||
// TODO - limit the number of commits loaded in memory. this could get very large.
|
||||
// This is okay because only tooling will load the archived commit timeline today
|
||||
readCommits.put(key.toString(), Arrays.copyOf(val.getBytes(), val.getLength()));
|
||||
}
|
||||
this.instants = new ArrayList<>(readCommits.keySet());
|
||||
this.inflights = new ArrayList<>(0);
|
||||
} finally {
|
||||
Closeables.closeQuietly(reader);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not load archived commit timeline from path " + archiveLogPath, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void saveInstantAsInflight(String instant) {
|
||||
throw new UnsupportedOperationException(
|
||||
"Could not save inflight instant in ArchivedTimeline " + instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void saveInstantAsComplete(String instant, Optional<byte[]> data) {
|
||||
throw new UnsupportedOperationException(
|
||||
"Could not save instant as complete in ArchivedTimeline " + instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void revertInstantToInflight(String instant) {
|
||||
throw new UnsupportedOperationException(
|
||||
"Could not revert instant in ArchivedTimeline " + instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeInflightFromTimeline(String instant) {
|
||||
throw new UnsupportedOperationException(
|
||||
"Could not delete inflight instant from ArchivedTimeline " + instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline reload() throws IOException {
|
||||
return new HoodieArchivedCommitTimeline(fs, metaPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<byte[]> readInstantDetails(String instant) {
|
||||
return Optional.ofNullable(readCommits.get(instant));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getInflightFileName(String instant) {
|
||||
throw new UnsupportedOperationException("No inflight filename for archived commits");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getCompletedFileName(String instant) {
|
||||
throw new UnsupportedOperationException("No inflight filename for archived commits");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getTimelineName() {
|
||||
return "archived-commits";
|
||||
}
|
||||
|
||||
public static Path getArchiveLogPath(String metaPath) {
|
||||
return new Path(metaPath, HOODIE_COMMIT_ARCHIVE_LOG_FILE);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,57 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table.timeline;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
|
||||
public class HoodieCleanerTimeline extends HoodieDefaultTimeline {
|
||||
public HoodieCleanerTimeline(FileSystem fs, String path) {
|
||||
super(fs, path, HoodieTableMetaClient.CLEAN_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline reload() throws IOException {
|
||||
return new HoodieCleanerTimeline(fs, metaPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<byte[]> readInstantDetails(String instant) {
|
||||
// TODO - Nothing about the clean written today - this should change
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getInflightFileName(String instant) {
|
||||
return HoodieTableMetaClient.makeInflightCleanerFileName(instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getCompletedFileName(String instant) {
|
||||
return HoodieTableMetaClient.makeCleanerFileName(instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getTimelineName() {
|
||||
return "cleaner";
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,348 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table.timeline;
|
||||
|
||||
import com.google.common.io.Closeables;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* HoodieTimeline allows representation of meta-data events as a timeline.
|
||||
* Instants are specific points in time represented as strings.
|
||||
* in this format YYYYMMDDHHmmSS. e.g. 20170101193218
|
||||
* Any operation on the timeline starts with the inflight instant and then when complete marks
|
||||
* the completed instant and removes the inflight instant.
|
||||
* Completed instants are plainly referred to as just instants
|
||||
* <p>
|
||||
* Timelines as immutable once created. Any operation to change the timeline (like create/delete instants)
|
||||
* will not be reflected unless explicitly reloaded using the reload()
|
||||
*
|
||||
* @see com.uber.hoodie.common.table.HoodieTableMetaClient
|
||||
* @see HoodieTimeline
|
||||
* @since 0.3.0
|
||||
*/
|
||||
public abstract class HoodieDefaultTimeline implements HoodieTimeline {
|
||||
private final transient static Logger log = LogManager.getLogger(HoodieDefaultTimeline.class);
|
||||
|
||||
public static final String INFLIGHT_EXTENSION = ".inflight";
|
||||
protected String metaPath;
|
||||
protected transient FileSystem fs;
|
||||
protected List<String> inflights;
|
||||
protected List<String> instants;
|
||||
|
||||
public HoodieDefaultTimeline(FileSystem fs, String metaPath, String fileExtension) {
|
||||
String completedInstantExtension = fileExtension;
|
||||
String inflightInstantExtension = fileExtension + INFLIGHT_EXTENSION;
|
||||
|
||||
FileStatus[] fileStatuses;
|
||||
try {
|
||||
fileStatuses = HoodieTableMetaClient.scanFiles(fs, new Path(metaPath),
|
||||
path -> path.toString().endsWith(completedInstantExtension) || path.toString()
|
||||
.endsWith(inflightInstantExtension));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to scan metadata", e);
|
||||
}
|
||||
this.instants = Arrays.stream(fileStatuses)
|
||||
.filter(status -> status.getPath().getName().endsWith(completedInstantExtension))
|
||||
.map(fileStatus -> fileStatus.getPath().getName().replaceAll(completedInstantExtension, ""))
|
||||
.sorted().collect(Collectors.toList());
|
||||
this.inflights = Arrays.stream(fileStatuses).filter(
|
||||
status -> status.getPath().getName().endsWith(inflightInstantExtension)).map(
|
||||
fileStatus -> fileStatus.getPath().getName()
|
||||
.replaceAll(inflightInstantExtension, "")).sorted()
|
||||
.collect(Collectors.toList());
|
||||
this.fs = fs;
|
||||
this.metaPath = metaPath;
|
||||
}
|
||||
|
||||
public HoodieDefaultTimeline(Stream<String> instants, Stream<String> inflights) {
|
||||
this.instants = instants.collect(Collectors.toList());
|
||||
this.inflights = inflights.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* This constructor only supports backwards compatibility in inflight commits in ActiveCommitTimeline.
|
||||
* This should never be used.
|
||||
*
|
||||
* @param fs
|
||||
* @param metaPath
|
||||
* @deprecated
|
||||
*/
|
||||
public HoodieDefaultTimeline(FileSystem fs, String metaPath) {
|
||||
this.fs = fs;
|
||||
this.metaPath = metaPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* For serailizing and de-serializing
|
||||
* @deprecated
|
||||
*/
|
||||
public HoodieDefaultTimeline() {
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* This method is only used when this object is deserialized in a spark executor.
|
||||
* @deprecated
|
||||
*/
|
||||
private void readObject(java.io.ObjectInputStream in)
|
||||
throws IOException, ClassNotFoundException {
|
||||
in.defaultReadObject();
|
||||
this.fs = FSUtils.getFs();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<String> findInstantsInRange(String startTs, String endTs) {
|
||||
return instants.stream().filter(
|
||||
s -> compareInstants(s, startTs, GREATER) && compareInstants(s, endTs,
|
||||
LESSER_OR_EQUAL));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<String> findInstantsAfter(String commitTime, int numCommits) {
|
||||
return instants.stream().filter(s -> compareInstants(s, commitTime, GREATER))
|
||||
.limit(numCommits);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasInstants() {
|
||||
return instants.stream().count() != 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasInflightInstants() {
|
||||
return inflights.stream().count() != 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getTotalInstants() {
|
||||
return new Long(instants.stream().count()).intValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<String> firstInstant() {
|
||||
return instants.stream().findFirst();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<String> nthInstant(int n) {
|
||||
if(!hasInstants() || n >= getTotalInstants()) {
|
||||
return Optional.empty();
|
||||
}
|
||||
return Optional.of(instants.get(n));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<String> lastInstant() {
|
||||
return hasInstants() ? nthInstant(getTotalInstants() - 1) : Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<String> nthFromLastInstant(int n) {
|
||||
if(getTotalInstants() < n + 1) {
|
||||
return Optional.empty();
|
||||
}
|
||||
return nthInstant(getTotalInstants() - 1 - n);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsInstant(String instant) {
|
||||
return instants.stream().anyMatch(s -> s.equals(instant));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsOrBeforeTimelineStarts(String instant) {
|
||||
return containsInstant(instant) || isInstantBeforeTimelineStarts(instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<String> getInstants() {
|
||||
return instants.stream();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<String> getInflightInstants() {
|
||||
return inflights.stream();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isInstantBeforeTimelineStarts(String instant) {
|
||||
Optional<String> firstCommit = firstInstant();
|
||||
return firstCommit.isPresent() && compareInstants(instant, firstCommit.get(), LESSER);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void saveInstantAsInflight(String instant) {
|
||||
log.info("Creating a new in-flight " + getTimelineName() + " " + instant);
|
||||
// Create the in-flight file
|
||||
createFileInMetaPath(getInflightFileName(instant), Optional.empty());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void saveInstantAsComplete(String instant, Optional<byte[]> data) {
|
||||
log.info("Marking complete " + getTimelineName() + " " + instant);
|
||||
moveInflightToComplete(instant, data, getCompletedFileName(instant),
|
||||
HoodieTableMetaClient.makeInflightCommitFileName(instant));
|
||||
log.info("Completed " + getTimelineName() + " " + instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void revertInstantToInflight(String instant) {
|
||||
log.info("Reverting instant to inflight " + getTimelineName() + " " + instant);
|
||||
moveCompleteToInflight(instant, getCompletedFileName(instant),
|
||||
getInflightFileName(instant));
|
||||
log.info("Reverted " + getTimelineName() + " " + instant + " to inflight");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void removeInflightFromTimeline(String instant) {
|
||||
log.info("Removing in-flight " + getTimelineName() + " " + instant);
|
||||
String inFlightCommitFileName = getInflightFileName(instant);
|
||||
Path inFlightCommitFilePath = new Path(metaPath, inFlightCommitFileName);
|
||||
try {
|
||||
fs.delete(inFlightCommitFilePath, false);
|
||||
log.info("Removed in-flight " + getTimelineName() + " " + instant);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not remove inflight commit " + inFlightCommitFilePath, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<byte[]> readInstantDetails(String instant) {
|
||||
Path detailPath = new Path(metaPath, getCompletedFileName(instant));
|
||||
return readDataFromPath(detailPath);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the in-flight instant file name
|
||||
*
|
||||
* @param instant
|
||||
* @return
|
||||
*/
|
||||
protected abstract String getInflightFileName(String instant);
|
||||
|
||||
/**
|
||||
* Get the completed instant file name
|
||||
*
|
||||
* @param instant
|
||||
* @return
|
||||
*/
|
||||
protected abstract String getCompletedFileName(String instant);
|
||||
|
||||
/**
|
||||
* Get the timeline name
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
protected abstract String getTimelineName();
|
||||
|
||||
|
||||
protected void moveInflightToComplete(String instant, Optional<byte[]> data,
|
||||
String commitFileName, String inflightFileName) {
|
||||
Path commitFilePath = new Path(metaPath, commitFileName);
|
||||
try {
|
||||
// open a new file and write the commit metadata in
|
||||
Path inflightCommitFile = new Path(metaPath, inflightFileName);
|
||||
createFileInMetaPath(inflightFileName, data);
|
||||
boolean success = fs.rename(inflightCommitFile, commitFilePath);
|
||||
if (!success) {
|
||||
throw new HoodieIOException(
|
||||
"Could not rename " + inflightCommitFile + " to " + commitFilePath);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not complete commit " + instant, e);
|
||||
}
|
||||
}
|
||||
|
||||
protected void moveCompleteToInflight(String instant, String commitFileName,
|
||||
String inflightFileName) {
|
||||
Path inFlightCommitFilePath = new Path(metaPath, inflightFileName);
|
||||
try {
|
||||
if (!fs.exists(inFlightCommitFilePath)) {
|
||||
Path commitFilePath = new Path(metaPath, commitFileName);
|
||||
boolean success = fs.rename(commitFilePath, inFlightCommitFilePath);
|
||||
if (!success) {
|
||||
throw new HoodieIOException(
|
||||
"Could not rename " + commitFilePath + " to " + inFlightCommitFilePath);
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not complete commit revert " + instant, e);
|
||||
}
|
||||
}
|
||||
|
||||
protected void createFileInMetaPath(String filename, Optional<byte[]> content) {
|
||||
Path fullPath = new Path(metaPath, filename);
|
||||
try {
|
||||
if (!content.isPresent()) {
|
||||
if (fs.createNewFile(fullPath)) {
|
||||
log.info("Created a new file in meta path: " + fullPath);
|
||||
return;
|
||||
}
|
||||
} else {
|
||||
FSDataOutputStream fsout = fs.create(fullPath, true);
|
||||
fsout.write(content.get());
|
||||
fsout.close();
|
||||
return;
|
||||
}
|
||||
throw new HoodieIOException("Failed to create file " + fullPath);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to create file " + fullPath, e);
|
||||
}
|
||||
}
|
||||
|
||||
protected Optional<byte[]> readDataFromPath(Path detailPath) {
|
||||
FSDataInputStream is = null;
|
||||
try {
|
||||
is = fs.open(detailPath);
|
||||
return Optional.of(IOUtils.toByteArray(is));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not read commit details from " + detailPath, e);
|
||||
} finally {
|
||||
if (is != null) {
|
||||
Closeables.closeQuietly(is);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return this.getClass().getName() + ": " + instants.stream().map(Object::toString)
|
||||
.collect(Collectors.joining(","));
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,50 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table.timeline;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class HoodieSavePointTimeline extends HoodieDefaultTimeline {
|
||||
public HoodieSavePointTimeline(FileSystem fs, String metaPath) {
|
||||
super(fs, metaPath, HoodieTableMetaClient.SAVEPOINT_EXTENSION);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline reload() throws IOException {
|
||||
return new HoodieSavePointTimeline(fs, metaPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getInflightFileName(String instant) {
|
||||
return HoodieTableMetaClient.makeInflightSavePointFileName(instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getCompletedFileName(String instant) {
|
||||
return HoodieTableMetaClient.makeSavePointFileName(instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getTimelineName() {
|
||||
return "savepoint";
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,192 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table.view;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collector;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Common abstract implementation for multiple TableFileSystemView Implementations.
|
||||
* 2 possible implementations are ReadOptimizedView and RealtimeView
|
||||
*
|
||||
* Concrete implementations extending this abstract class, should only implement
|
||||
* listDataFilesInPartition which includes files to be included in the view
|
||||
*
|
||||
* @see TableFileSystemView
|
||||
* @see ReadOptimizedTableView
|
||||
* @since 0.3.0
|
||||
*/
|
||||
public abstract class AbstractTableFileSystemView implements TableFileSystemView {
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
protected final transient FileSystem fs;
|
||||
protected final HoodieTimeline activeCommitTimeline;
|
||||
|
||||
public AbstractTableFileSystemView(FileSystem fs, HoodieTableMetaClient metaClient) {
|
||||
this.metaClient = metaClient;
|
||||
this.fs = fs;
|
||||
this.activeCommitTimeline = metaClient.getActiveCommitTimeline();
|
||||
}
|
||||
|
||||
public Stream<HoodieDataFile> getLatestDataFilesForFileId(final String partitionPath,
|
||||
String fileId) {
|
||||
Optional<String> lastInstant = activeCommitTimeline.lastInstant();
|
||||
if (lastInstant.isPresent()) {
|
||||
return streamLatestVersionInPartition(partitionPath, lastInstant.get())
|
||||
.filter(hoodieDataFile -> hoodieDataFile.getFileId().equals(fileId));
|
||||
}
|
||||
return Stream.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> streamLatestVersionInPartition(String partitionPathStr,
|
||||
String maxCommitTime) {
|
||||
try {
|
||||
return streamLatestVersionsBeforeOrOn(listDataFilesInPartition(partitionPathStr),
|
||||
maxCommitTime);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not get latest versions in Partition " + partitionPathStr, e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Stream<List<HoodieDataFile>> streamEveryVersionInPartition(String partitionPath) {
|
||||
try {
|
||||
if(activeCommitTimeline.lastInstant().isPresent()) {
|
||||
return streamFilesByFileId(listDataFilesInPartition(partitionPath),
|
||||
activeCommitTimeline.lastInstant().get());
|
||||
}
|
||||
return Stream.empty();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not load all file versions in partition " + partitionPath, e);
|
||||
}
|
||||
}
|
||||
|
||||
protected abstract FileStatus[] listDataFilesInPartition(String partitionPathStr)
|
||||
throws IOException;
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> streamLatestVersionInRange(FileStatus[] fileStatuses,
|
||||
List<String> commitsToReturn) {
|
||||
if (!activeCommitTimeline.hasInstants() || commitsToReturn.isEmpty()) {
|
||||
return Stream.empty();
|
||||
}
|
||||
try {
|
||||
return streamFilesByFileId(fileStatuses, activeCommitTimeline.lastInstant().get())
|
||||
.map((Function<List<HoodieDataFile>, Optional<HoodieDataFile>>) fss -> {
|
||||
for (HoodieDataFile fs : fss) {
|
||||
if (commitsToReturn.contains(fs.getCommitTime())) {
|
||||
return Optional.of(fs);
|
||||
}
|
||||
}
|
||||
return Optional.empty();
|
||||
}).filter(Optional::isPresent).map(Optional::get);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not filter files from commits " + commitsToReturn,
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> streamLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
|
||||
String maxCommitToReturn) {
|
||||
try {
|
||||
if (!activeCommitTimeline.hasInstants()) {
|
||||
return Stream.empty();
|
||||
}
|
||||
return streamFilesByFileId(fileStatuses, activeCommitTimeline.lastInstant().get())
|
||||
.map((Function<List<HoodieDataFile>, Optional<HoodieDataFile>>) fss -> {
|
||||
for (HoodieDataFile fs1 : fss) {
|
||||
if (activeCommitTimeline
|
||||
.compareInstants(fs1.getCommitTime(), maxCommitToReturn,
|
||||
HoodieTimeline.LESSER_OR_EQUAL)) {
|
||||
return Optional.of(fs1);
|
||||
}
|
||||
}
|
||||
return Optional.empty();
|
||||
}).filter(Optional::isPresent).map(Optional::get);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not filter files for latest version ", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> streamLatestVersions(FileStatus[] fileStatuses) {
|
||||
try {
|
||||
if (!activeCommitTimeline.hasInstants()) {
|
||||
return Stream.empty();
|
||||
}
|
||||
return streamFilesByFileId(fileStatuses, activeCommitTimeline.lastInstant().get())
|
||||
.map(statuses -> statuses.get(0));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not filter files for latest version ", e);
|
||||
}
|
||||
}
|
||||
|
||||
protected Stream<List<HoodieDataFile>> streamFilesByFileId(FileStatus[] files,
|
||||
String maxCommitTime) throws IOException {
|
||||
return groupFilesByFileId(files, maxCommitTime).values().stream();
|
||||
}
|
||||
|
||||
/**
|
||||
* Filters the list of FileStatus to exclude non-committed data files and group by FileID
|
||||
* and sort the actial files by commit time (newer commit first)
|
||||
*
|
||||
* @param files Files to filter and group from
|
||||
* @param maxCommitTime maximum permissible commit time
|
||||
* @return Grouped map by fileId
|
||||
*/
|
||||
private Map<String, List<HoodieDataFile>> groupFilesByFileId(FileStatus[] files,
|
||||
String maxCommitTime) throws IOException {
|
||||
return Arrays.stream(files).flatMap(fileStatus -> {
|
||||
HoodieDataFile dataFile = new HoodieDataFile(fileStatus);
|
||||
if (activeCommitTimeline.containsOrBeforeTimelineStarts(dataFile.getCommitTime())
|
||||
&& activeCommitTimeline.compareInstants(dataFile.getCommitTime(), maxCommitTime,
|
||||
HoodieTimeline.LESSER_OR_EQUAL)) {
|
||||
return Stream.of(Pair.of(dataFile.getFileId(), dataFile));
|
||||
}
|
||||
return Stream.empty();
|
||||
}).collect(Collectors
|
||||
.groupingBy(Pair::getKey, Collectors.mapping(Pair::getValue, toSortedFileStatus())));
|
||||
}
|
||||
|
||||
private Collector<HoodieDataFile, ?, List<HoodieDataFile>> toSortedFileStatus() {
|
||||
return Collectors.collectingAndThen(Collectors.toList(),
|
||||
l -> l.stream().sorted(HoodieDataFile.getCommitTimeComparator())
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
@@ -0,0 +1,47 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table.view;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* ReadOptimized view which includes only the ROStorageformat files
|
||||
*/
|
||||
public class ReadOptimizedTableView extends AbstractTableFileSystemView {
|
||||
public ReadOptimizedTableView(FileSystem fs, HoodieTableMetaClient metaClient) {
|
||||
super(fs, metaClient);
|
||||
}
|
||||
|
||||
protected FileStatus[] listDataFilesInPartition(String partitionPathStr) {
|
||||
Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr);
|
||||
try {
|
||||
return fs.listStatus(partitionPath, path -> path.getName()
|
||||
.contains(metaClient.getTableConfig().getROStorageFormat().getFileExtension()));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Failed to list data files in partition " + partitionPathStr, e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
@@ -16,7 +16,6 @@
|
||||
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
@@ -68,14 +67,6 @@ public class FSUtils {
|
||||
return String.format("*_*_%s.parquet", commitTime);
|
||||
}
|
||||
|
||||
public static String makeInflightCommitFileName(String commitTime) {
|
||||
return commitTime + HoodieTableMetadata.INFLIGHT_FILE_SUFFIX;
|
||||
}
|
||||
|
||||
public static String makeCommitFileName(String commitTime) {
|
||||
return commitTime + HoodieTableMetadata.COMMIT_FILE_SUFFIX;
|
||||
}
|
||||
|
||||
public static String getCommitFromCommitFile(String commitFileName) {
|
||||
return commitFileName.split("\\.")[0];
|
||||
}
|
||||
|
||||
@@ -16,6 +16,11 @@
|
||||
|
||||
package com.uber.hoodie.exception;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Exception thrown to indicate that a hoodie dataset was not found on the path provided
|
||||
@@ -29,4 +34,21 @@ public class DatasetNotFoundException extends HoodieException {
|
||||
private static String getErrorMessage(String basePath) {
|
||||
return "Hoodie dataset not found in path " + basePath;
|
||||
}
|
||||
|
||||
public static void checkValidDataset(FileSystem fs, Path basePathDir, Path metaPathDir)
|
||||
throws DatasetNotFoundException {
|
||||
// Check if the base path is found
|
||||
try {
|
||||
if (!fs.exists(basePathDir) || !fs.isDirectory(basePathDir)) {
|
||||
throw new DatasetNotFoundException(basePathDir.toString());
|
||||
}
|
||||
// Check if the meta path is found
|
||||
if (!fs.exists(metaPathDir) || !fs.isDirectory(metaPathDir)) {
|
||||
throw new DatasetNotFoundException(metaPathDir.toString());
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not check if dataset " + basePathDir + " is valid dataset", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -24,13 +24,17 @@ import java.io.IOException;
|
||||
* </p>
|
||||
*/
|
||||
public class HoodieIOException extends HoodieException {
|
||||
private final IOException ioException;
|
||||
private IOException ioException;
|
||||
|
||||
public HoodieIOException(String msg, IOException t) {
|
||||
super(msg, t);
|
||||
this.ioException = t;
|
||||
}
|
||||
|
||||
public HoodieIOException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public IOException getIOException() {
|
||||
return ioException;
|
||||
}
|
||||
|
||||
@@ -16,58 +16,63 @@
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import com.esotericsoftware.kryo.Kryo;
|
||||
import com.esotericsoftware.kryo.io.Input;
|
||||
import com.esotericsoftware.kryo.io.Output;
|
||||
import com.uber.hoodie.common.table.HoodieTableConfig;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileWriter;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Date;
|
||||
import java.util.Iterator;
|
||||
import java.util.Properties;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
public class HoodieTestUtils {
|
||||
|
||||
public static FileSystem fs = FSUtils.getFs();
|
||||
public static final String TEST_EXTENSION = ".test";
|
||||
public static final String RAW_TRIPS_TEST_NAME = "raw_trips";
|
||||
public static final int DEFAULT_TASK_PARTITIONID = 1;
|
||||
|
||||
public static final void initializeHoodieDirectory(String basePath) throws IOException {
|
||||
new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME).mkdirs();
|
||||
public static HoodieTableMetaClient init(String basePath) throws IOException {
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(HoodieTableMetadata.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME);
|
||||
properties.setProperty(HoodieTableMetadata.HOODIE_TABLE_TYPE_PROP_NAME, HoodieTableMetadata.DEFAULT_TABLE_TYPE.name());
|
||||
FileWriter fileWriter = new FileWriter(new File(basePath + "/.hoodie/hoodie.properties"));
|
||||
try {
|
||||
properties.store(fileWriter, "");
|
||||
} finally {
|
||||
fileWriter.close();
|
||||
}
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME);
|
||||
return HoodieTableMetaClient.initializePathAsHoodieDataset(fs, basePath, properties);
|
||||
}
|
||||
|
||||
public static final String initializeTempHoodieBasePath() throws IOException {
|
||||
public static HoodieTableMetaClient initOnTemp() throws IOException {
|
||||
// Create a temp folder as the base path
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
String basePath = folder.getRoot().getAbsolutePath();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath);
|
||||
return basePath;
|
||||
return HoodieTestUtils.init(basePath);
|
||||
}
|
||||
|
||||
public static final String getNewCommitTime() {
|
||||
public static String makeNewCommitTime() {
|
||||
return new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
}
|
||||
|
||||
public static final void createCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
for (String commitTime: commitTimes) {
|
||||
new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME+ "/" + FSUtils.makeCommitFileName(commitTime)).createNewFile();
|
||||
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + HoodieTableMetaClient.makeCommitFileName(commitTime)).createNewFile();
|
||||
}
|
||||
}
|
||||
|
||||
public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
for (String commitTime: commitTimes) {
|
||||
new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME+ "/" + FSUtils.makeInflightCommitFileName(commitTime)).createNewFile();
|
||||
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + HoodieTableMetaClient.makeInflightCommitFileName(commitTime)).createNewFile();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -92,10 +97,43 @@ public class HoodieTestUtils {
|
||||
}
|
||||
|
||||
public static final boolean doesCommitExist(String basePath, String commitTime) {
|
||||
return new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME+ "/" + commitTime + HoodieTableMetadata.COMMIT_FILE_SUFFIX).exists();
|
||||
return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + commitTime + HoodieTableMetaClient.COMMIT_EXTENSION).exists();
|
||||
}
|
||||
|
||||
public static final boolean doesInflightExist(String basePath, String commitTime) {
|
||||
return new File(basePath + "/" + HoodieTableMetadata.METAFOLDER_NAME+ "/" + commitTime + HoodieTableMetadata.INFLIGHT_FILE_SUFFIX).exists();
|
||||
return new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME+ "/" + commitTime + HoodieTableMetaClient.INFLIGHT_FILE_SUFFIX).exists();
|
||||
}
|
||||
|
||||
public static String makeInflightTestFileName(String instant) {
|
||||
return instant + TEST_EXTENSION + HoodieTableMetaClient.INFLIGHT_FILE_SUFFIX;
|
||||
}
|
||||
|
||||
public static String makeTestFileName(String instant) {
|
||||
return instant + TEST_EXTENSION;
|
||||
}
|
||||
|
||||
public static String makeCommitFileName(String instant) {
|
||||
return instant + ".commit";
|
||||
}
|
||||
|
||||
public static void assertStreamEquals(String message, Stream<?> expected, Stream<?> actual) {
|
||||
Iterator<?> iter1 = expected.iterator(), iter2 = actual.iterator();
|
||||
while(iter1.hasNext() && iter2.hasNext())
|
||||
assertEquals(message, iter1.next(), iter2.next());
|
||||
assert !iter1.hasNext() && !iter2.hasNext();
|
||||
}
|
||||
|
||||
public static <T extends Serializable> T serializeDeserialize(T object, Class<T> clazz) {
|
||||
// Using Kyro as the default serializer in Spark Jobs
|
||||
Kryo kryo = new Kryo();
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
Output output = new Output(baos);
|
||||
kryo.writeObject(output, object);
|
||||
output.close();
|
||||
|
||||
Input input = new Input(new ByteArrayInputStream(baos.toByteArray()));
|
||||
T deseralizedObject = kryo.readObject(input, clazz);
|
||||
input.close();
|
||||
return deseralizedObject;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,48 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.model;
|
||||
|
||||
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class TestHoodieCommits {
|
||||
|
||||
@Test
|
||||
public void testHoodieCommits() throws Exception {
|
||||
HoodieCommits commits = new HoodieCommits(Arrays.asList("001", "005", "004", "002"));
|
||||
assertFalse(commits.contains("003"));
|
||||
assertTrue(commits.contains("002"));
|
||||
assertEquals(Arrays.asList("004", "005"), commits.findCommitsAfter("003", 2));
|
||||
assertEquals(Arrays.asList("001", "002", "004"), commits.findCommitsInRange("000", "004"));
|
||||
assertEquals(commits.lastCommit(), commits.lastCommit(0));
|
||||
assertEquals("001", commits.lastCommit(3));
|
||||
assertEquals(null, commits.lastCommit(4));
|
||||
|
||||
assertEquals(commits.max("001", "000"), "001");
|
||||
assertFalse(HoodieCommits.isCommit1After("001", "002"));
|
||||
assertFalse(HoodieCommits.isCommit1After("001", "001"));
|
||||
assertTrue(HoodieCommits.isCommit1After("003", "002"));
|
||||
assertTrue(HoodieCommits.isCommit1BeforeOrOn("003", "003"));
|
||||
}
|
||||
}
|
||||
@@ -1,242 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.model;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.HoodieRecordMissingException;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestHoodieTableMetadata {
|
||||
private String basePath = null;
|
||||
private HoodieTableMetadata metadata = null;
|
||||
@Rule
|
||||
public final ExpectedException exception = ExpectedException.none();
|
||||
|
||||
@Before
|
||||
public void init() throws Exception {
|
||||
basePath = HoodieTestUtils.initializeTempHoodieBasePath();
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScanCommitTs() throws Exception {
|
||||
// Empty commit dir
|
||||
assertTrue(metadata.getAllCommits().isEmpty());
|
||||
|
||||
// Create some commit files
|
||||
new File(basePath + "/.hoodie/20160504123032.commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/20160503122032.commit").createNewFile();
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
List<String> list = metadata.getAllCommits().getCommitList();
|
||||
assertEquals(list.size(), 2);
|
||||
assertTrue(list.contains("20160504123032"));
|
||||
assertTrue(list.contains("20160503122032"));
|
||||
|
||||
// Check the .inflight files
|
||||
assertTrue(metadata.getAllInflightCommits().isEmpty());
|
||||
new File(basePath + "/.hoodie/20160505123032.inflight").createNewFile();
|
||||
new File(basePath + "/.hoodie/20160506122032.inflight").createNewFile();
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
list = metadata.getAllInflightCommits();
|
||||
assertEquals(list.size(), 2);
|
||||
assertTrue(list.contains("20160505123032"));
|
||||
assertTrue(list.contains("20160506122032"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetLastValidFileNameForRecord() throws Exception {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
String partitionPath = "2016/05/01";
|
||||
new File(basePath + "/" + partitionPath).mkdirs();
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
HoodieRecord record = mock(HoodieRecord.class);
|
||||
when(record.getPartitionPath()).thenReturn(partitionPath);
|
||||
when(record.getCurrentLocation()).thenReturn(new HoodieRecordLocation("001", fileId));
|
||||
|
||||
// First, no commit for this record
|
||||
exception.expect(HoodieIOException.class);
|
||||
metadata.getFilenameForRecord(fs, record);
|
||||
|
||||
// Only one commit, but is not safe
|
||||
String commitTime1 = "20160501123212";
|
||||
String fileName1 = FSUtils.makeDataFileName(commitTime1, 1, fileId);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
|
||||
assertNull(metadata.getFilenameForRecord(fs, record));
|
||||
|
||||
// Make this commit safe
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
assertTrue(metadata.getFilenameForRecord(fs, record).equals(fileName1));
|
||||
|
||||
// Do another commit, but not safe
|
||||
String commitTime2 = "20160502123012";
|
||||
String fileName2 = FSUtils.makeDataFileName(commitTime2, 1, fileId);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
|
||||
assertTrue(metadata.getFilenameForRecord(fs, record).equals(fileName1));
|
||||
|
||||
// Make it safe
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
assertTrue(metadata.getFilenameForRecord(fs, record).equals(fileName2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetAllPartitionPaths() throws IOException {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
|
||||
// Empty
|
||||
List<String> partitions = FSUtils.getAllPartitionPaths(fs, basePath);
|
||||
assertEquals(partitions.size(), 0);
|
||||
|
||||
// Add some dirs
|
||||
new File(basePath + "/2016/04/01").mkdirs();
|
||||
new File(basePath + "/2015/04/01").mkdirs();
|
||||
partitions = FSUtils.getAllPartitionPaths(fs, basePath);
|
||||
assertEquals(partitions.size(), 2);
|
||||
assertTrue(partitions.contains("2016/04/01"));
|
||||
assertTrue(partitions.contains("2015/04/01"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetFileVersionsInPartition() throws IOException {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
|
||||
String commitTime1 = "20160501123032";
|
||||
String commitTime2 = "20160502123032";
|
||||
String commitTime3 = "20160503123032";
|
||||
String commitTime4 = "20160504123032";
|
||||
|
||||
HoodieTestUtils.createCommitFiles(basePath, commitTime1, commitTime2, commitTime3, commitTime4);
|
||||
|
||||
String fileId1 = UUID.randomUUID().toString();
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile();
|
||||
|
||||
metadata = new HoodieTableMetadata(FSUtils.getFs(), basePath, "testTable");
|
||||
|
||||
Map<String, List<FileStatus>> fileVersions = metadata.getAllVersionsInPartition(FSUtils.getFs(), "2016/05/01");
|
||||
assertEquals(fileVersions.get(fileId1).size(), 2);
|
||||
assertEquals(fileVersions.get(fileId2).size(), 3);
|
||||
assertEquals(fileVersions.get(fileId3).size(), 2);
|
||||
String commitTs = FSUtils.getCommitTime(fileVersions.get(fileId1).get(fileVersions.get(fileId1).size() - 1).getPath().getName());
|
||||
assertTrue(commitTs.equals(commitTime1));
|
||||
commitTs = FSUtils.getCommitTime(fileVersions.get(fileId1).get(fileVersions.get(fileId1).size() - 2).getPath().getName());
|
||||
assertTrue(commitTs.equals(commitTime4));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetOnlyLatestVersionFiles() throws Exception {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
String commitTime1 = "20160501123032";
|
||||
String commitTime2 = "20160502123032";
|
||||
String commitTime3 = "20160503123032";
|
||||
String commitTime4 = "20160504123032";
|
||||
String fileId1 = UUID.randomUUID().toString();
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile();
|
||||
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
|
||||
|
||||
// Now we list the entire partition
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
FileStatus[] statuses = fs.listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(statuses.length, 7);
|
||||
|
||||
metadata = new HoodieTableMetadata(fs, basePath, "testTable");
|
||||
FileStatus[] statuses1 = metadata
|
||||
.getLatestVersionInPartition(fs, "2016/05/01", commitTime4);
|
||||
assertEquals(statuses1.length, 3);
|
||||
Set<String> filenames = Sets.newHashSet();
|
||||
for (FileStatus status : statuses1) {
|
||||
filenames.add(status.getPath().getName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
|
||||
|
||||
// Reset the max commit time
|
||||
FileStatus[] statuses2 = metadata
|
||||
.getLatestVersionInPartition(fs, "2016/05/01", commitTime3);
|
||||
assertEquals(statuses2.length, 3);
|
||||
filenames = Sets.newHashSet();
|
||||
for (FileStatus status : statuses2) {
|
||||
filenames.add(status.getPath().getName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCommitTimeComparison() {
|
||||
String commitTime1 = "20160504123032";
|
||||
String commitTime2 = "20151231203159";
|
||||
assertTrue(HoodieCommits.isCommit1After(commitTime1, commitTime2));
|
||||
assertTrue(HoodieCommits.isCommit1BeforeOrOn(commitTime1, commitTime1));
|
||||
assertTrue(HoodieCommits.isCommit1BeforeOrOn(commitTime2, commitTime1));
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanup() {
|
||||
if (basePath != null) {
|
||||
new File(basePath).delete();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,128 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table;
|
||||
|
||||
import com.esotericsoftware.kryo.Kryo;
|
||||
import com.esotericsoftware.kryo.io.Input;
|
||||
import com.esotericsoftware.kryo.io.Output;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieArchivedCommitTimeline;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.ArrayFile;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
public class HoodieTableMetaClientTest {
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private String basePath;
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
this.basePath = folder.getRoot().getAbsolutePath();
|
||||
metaClient = HoodieTestUtils.init(basePath);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void checkMetadata() {
|
||||
assertEquals("Table name should be raw_trips", HoodieTestUtils.RAW_TRIPS_TEST_NAME,
|
||||
metaClient.getTableConfig().getTableName());
|
||||
assertEquals("Basepath should be the one assigned", basePath, metaClient.getBasePath());
|
||||
assertEquals("Metapath should be ${basepath}/.hoodie", basePath + "/.hoodie",
|
||||
metaClient.getMetaPath());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void checkSerDe() throws IOException {
|
||||
// check if this object is serialized and se-serialized, we are able to read from the file system
|
||||
HoodieTableMetaClient deseralizedMetaClient =
|
||||
HoodieTestUtils.serializeDeserialize(metaClient, HoodieTableMetaClient.class);
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
commitTimeline.saveInstantAsInflight("1");
|
||||
commitTimeline.saveInstantAsComplete("1", Optional.of("test-detail".getBytes()));
|
||||
commitTimeline = commitTimeline.reload();
|
||||
assertEquals("Commit should be 1", "1", commitTimeline.getInstants().findFirst().get());
|
||||
assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(),
|
||||
commitTimeline.readInstantDetails("1").get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void checkCommitTimeline() throws IOException {
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
assertFalse("Should be empty commit timeline",
|
||||
commitTimeline.getInstants().findFirst().isPresent());
|
||||
assertFalse("Should be empty commit timeline",
|
||||
commitTimeline.getInflightInstants().findFirst().isPresent());
|
||||
commitTimeline.saveInstantAsInflight("1");
|
||||
commitTimeline.saveInstantAsComplete("1", Optional.of("test-detail".getBytes()));
|
||||
|
||||
// Commit timeline should not auto-reload every time getActiveCommitTimeline(), it should be cached
|
||||
commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
assertFalse("Should be empty commit timeline",
|
||||
commitTimeline.getInstants().findFirst().isPresent());
|
||||
assertFalse("Should be empty commit timeline",
|
||||
commitTimeline.getInflightInstants().findFirst().isPresent());
|
||||
|
||||
commitTimeline = commitTimeline.reload();
|
||||
assertTrue("Should be the 1 commit we made",
|
||||
commitTimeline.getInstants().findFirst().isPresent());
|
||||
assertEquals("Commit should be 1", "1", commitTimeline.getInstants().findFirst().get());
|
||||
assertArrayEquals("Commit value should be \"test-detail\"", "test-detail".getBytes(),
|
||||
commitTimeline.readInstantDetails("1").get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void checkArchiveCommitTimeline() throws IOException {
|
||||
Path archiveLogPath =
|
||||
HoodieArchivedCommitTimeline.getArchiveLogPath(metaClient.getMetaPath());
|
||||
SequenceFile.Writer writer = SequenceFile
|
||||
.createWriter(HoodieTestUtils.fs.getConf(), SequenceFile.Writer.file(archiveLogPath),
|
||||
SequenceFile.Writer.keyClass(Text.class),
|
||||
SequenceFile.Writer.valueClass(Text.class));
|
||||
|
||||
writer.append(new Text("1"), new Text("data1"));
|
||||
writer.append(new Text("2"), new Text("data2"));
|
||||
writer.append(new Text("3"), new Text("data3"));
|
||||
|
||||
IOUtils.closeStream(writer);
|
||||
|
||||
HoodieTimeline archivedTimeline = metaClient.getArchivedCommitTimeline();
|
||||
assertEquals(Lists.newArrayList("1", "2", "3"),
|
||||
archivedTimeline.getInstants().collect(Collectors.toList()));
|
||||
System.out.println(new String( archivedTimeline.readInstantDetails("1").get()));
|
||||
assertArrayEquals(new Text("data1").getBytes(), archivedTimeline.readInstantDetails("1").get());
|
||||
assertArrayEquals(new Text("data2").getBytes(), archivedTimeline.readInstantDetails("2").get());
|
||||
assertArrayEquals(new Text("data3").getBytes(), archivedTimeline.readInstantDetails("3").get());
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
@@ -0,0 +1,104 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table.string;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
public class HoodieDefaultTimelineTest {
|
||||
private HoodieTimeline timeline;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
@Rule
|
||||
public final ExpectedException exception = ExpectedException.none();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
this.metaClient = HoodieTestUtils.initOnTemp();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
HoodieTestUtils.fs.delete(new Path(this.metaClient.getBasePath()), true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLoadingInstantsFromFiles() throws IOException {
|
||||
timeline =
|
||||
new MockHoodieTimeline(HoodieTestUtils.fs, metaClient.getMetaPath(), ".test");
|
||||
timeline.saveInstantAsComplete("1", Optional.empty());
|
||||
timeline.saveInstantAsComplete("3", Optional.empty());
|
||||
timeline.saveInstantAsComplete("5", Optional.empty());
|
||||
timeline.saveInstantAsComplete("8", Optional.empty());
|
||||
timeline.saveInstantAsInflight("9");
|
||||
timeline = timeline.reload();
|
||||
|
||||
assertEquals("Total instants should be 4", 4, timeline.getTotalInstants());
|
||||
HoodieTestUtils
|
||||
.assertStreamEquals("Check the instants stream", Stream.of("1", "3", "5", "8"),
|
||||
timeline.getInstants());
|
||||
assertTrue("Inflights should be present in the timeline", timeline.hasInflightInstants());
|
||||
HoodieTestUtils.assertStreamEquals("Check the inflights stream", Stream.of("9"),
|
||||
timeline.getInflightInstants());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimelineOperationsBasic() throws Exception {
|
||||
timeline = new MockHoodieTimeline(Stream.empty(), Stream.empty());
|
||||
assertFalse(timeline.hasInstants());
|
||||
assertFalse(timeline.hasInflightInstants());
|
||||
assertEquals("", 0, timeline.getTotalInstants());
|
||||
assertEquals("", Optional.empty(), timeline.firstInstant());
|
||||
assertEquals("", Optional.empty(), timeline.nthInstant(5));
|
||||
assertEquals("", Optional.empty(), timeline.nthInstant(-1));
|
||||
assertEquals("", Optional.empty(), timeline.lastInstant());
|
||||
assertFalse("", timeline.containsInstant("01"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTimelineOperations() throws Exception {
|
||||
timeline = new MockHoodieTimeline(
|
||||
Stream.of("01", "03", "05", "07", "09", "11", "13", "15", "17", "19"),
|
||||
Stream.of("21", "23"));
|
||||
HoodieTestUtils.assertStreamEquals("", Stream.of("05", "07", "09", "11"),
|
||||
timeline.findInstantsInRange("04", "11"));
|
||||
HoodieTestUtils
|
||||
.assertStreamEquals("", Stream.of("09", "11"), timeline.findInstantsAfter("07", 2));
|
||||
assertTrue(timeline.hasInstants());
|
||||
assertTrue(timeline.hasInflightInstants());
|
||||
assertEquals("", 10, timeline.getTotalInstants());
|
||||
assertEquals("", "01", timeline.firstInstant().get());
|
||||
assertEquals("", "11", timeline.nthInstant(5).get());
|
||||
assertEquals("", "19", timeline.lastInstant().get());
|
||||
assertEquals("", "09", timeline.nthFromLastInstant(5).get());
|
||||
assertTrue("", timeline.containsInstant("09"));
|
||||
assertFalse("", timeline.isInstantBeforeTimelineStarts("02"));
|
||||
assertTrue("", timeline.isInstantBeforeTimelineStarts("00"));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,66 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table.string;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class MockHoodieTimeline extends HoodieDefaultTimeline {
|
||||
private String fileExt;
|
||||
|
||||
public MockHoodieTimeline(FileSystem fs, String metaPath, String fileExtension)
|
||||
throws IOException {
|
||||
super(fs, metaPath, fileExtension);
|
||||
this.fileExt = fileExtension;
|
||||
}
|
||||
|
||||
public MockHoodieTimeline(Stream<String> instants, Stream<String> inflights)
|
||||
throws IOException {
|
||||
super(instants, inflights);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline reload() throws IOException {
|
||||
return new MockHoodieTimeline(fs, metaPath, fileExt);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<byte[]> readInstantDetails(String instant) {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getInflightFileName(String instant) {
|
||||
return HoodieTestUtils.makeInflightTestFileName(instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getCompletedFileName(String instant) {
|
||||
return HoodieTestUtils.makeTestFileName(instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getTimelineName() {
|
||||
return "mock-test";
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,385 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table.view;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
public class ReadOptimizedTableViewTest {
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private String basePath;
|
||||
private TableFileSystemView fsView;
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
TemporaryFolder folder = new TemporaryFolder();
|
||||
folder.create();
|
||||
this.basePath = folder.getRoot().getAbsolutePath();
|
||||
metaClient = HoodieTestUtils.init(basePath);
|
||||
fsView = new ReadOptimizedTableView(HoodieTestUtils.fs, metaClient);
|
||||
}
|
||||
|
||||
private void refreshFsView() {
|
||||
metaClient = new HoodieTableMetaClient(HoodieTestUtils.fs, basePath, true);
|
||||
fsView = new ReadOptimizedTableView(HoodieTestUtils.fs, metaClient);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetLatestDataFilesForFileId() throws IOException {
|
||||
String partitionPath = "2016/05/01";
|
||||
new File(basePath + "/" + partitionPath).mkdirs();
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
|
||||
assertFalse("No commit, should not find any data file",
|
||||
fsView.getLatestDataFilesForFileId(partitionPath, fileId).findFirst().isPresent());
|
||||
|
||||
// Only one commit, but is not safe
|
||||
String commitTime1 = "1";
|
||||
String fileName1 = FSUtils.makeDataFileName(commitTime1, 1, fileId);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
|
||||
refreshFsView();
|
||||
assertFalse("No commit, should not find any data file",
|
||||
fsView.getLatestDataFilesForFileId(partitionPath, fileId).findFirst().isPresent());
|
||||
|
||||
// Make this commit safe
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
|
||||
commitTimeline.saveInstantAsComplete(commitTime1, Optional.empty());
|
||||
refreshFsView();
|
||||
assertEquals("", fileName1,
|
||||
fsView.getLatestDataFilesForFileId(partitionPath, fileId).findFirst().get()
|
||||
.getFileName());
|
||||
|
||||
// Do another commit, but not safe
|
||||
String commitTime2 = "2";
|
||||
String fileName2 = FSUtils.makeDataFileName(commitTime2, 1, fileId);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
|
||||
refreshFsView();
|
||||
assertEquals("", fileName1,
|
||||
fsView.getLatestDataFilesForFileId(partitionPath, fileId).findFirst().get()
|
||||
.getFileName());
|
||||
|
||||
// Make it safe
|
||||
commitTimeline.saveInstantAsComplete(commitTime2, Optional.empty());
|
||||
refreshFsView();
|
||||
assertEquals("", fileName2,
|
||||
fsView.getLatestDataFilesForFileId(partitionPath, fileId).findFirst().get()
|
||||
.getFileName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStreamLatestVersionInPartition() throws IOException {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
String commitTime1 = "1";
|
||||
String commitTime2 = "2";
|
||||
String commitTime3 = "3";
|
||||
String commitTime4 = "4";
|
||||
String fileId1 = UUID.randomUUID().toString();
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
|
||||
.createNewFile();
|
||||
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
|
||||
|
||||
// Now we list the entire partition
|
||||
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(statuses.length, 7);
|
||||
|
||||
refreshFsView();
|
||||
List<HoodieDataFile> statuses1 =
|
||||
fsView.streamLatestVersionInPartition("2016/05/01", commitTime4)
|
||||
.collect(Collectors.toList());
|
||||
assertEquals(statuses1.size(), 3);
|
||||
Set<String> filenames = Sets.newHashSet();
|
||||
for (HoodieDataFile status : statuses1) {
|
||||
filenames.add(status.getFileName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
|
||||
|
||||
// Reset the max commit time
|
||||
List<HoodieDataFile> statuses2 =
|
||||
fsView.streamLatestVersionInPartition("2016/05/01", commitTime3)
|
||||
.collect(Collectors.toList());
|
||||
assertEquals(statuses2.size(), 3);
|
||||
filenames = Sets.newHashSet();
|
||||
for (HoodieDataFile status : statuses2) {
|
||||
filenames.add(status.getFileName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStreamEveryVersionInPartition() throws IOException {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
String commitTime1 = "1";
|
||||
String commitTime2 = "2";
|
||||
String commitTime3 = "3";
|
||||
String commitTime4 = "4";
|
||||
String fileId1 = UUID.randomUUID().toString();
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
|
||||
.createNewFile();
|
||||
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
|
||||
|
||||
// Now we list the entire partition
|
||||
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(statuses.length, 7);
|
||||
|
||||
refreshFsView();
|
||||
List<List<HoodieDataFile>> statuses1 =
|
||||
fsView.streamEveryVersionInPartition("2016/05/01").collect(Collectors.toList());
|
||||
assertEquals(statuses1.size(), 3);
|
||||
|
||||
for (List<HoodieDataFile> status : statuses1) {
|
||||
String fileId = status.get(0).getFileId();
|
||||
Set<String> filenames = Sets.newHashSet();
|
||||
for (HoodieDataFile dataFile : status) {
|
||||
assertEquals("All same fileId should be grouped", fileId, dataFile.getFileId());
|
||||
filenames.add(dataFile.getFileName());
|
||||
}
|
||||
if (fileId.equals(fileId1)) {
|
||||
assertEquals(filenames,
|
||||
Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId1),
|
||||
FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
|
||||
} else if (fileId.equals(fileId2)) {
|
||||
assertEquals(filenames,
|
||||
Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId2),
|
||||
FSUtils.makeDataFileName(commitTime2, 1, fileId2),
|
||||
FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
} else {
|
||||
assertEquals(filenames,
|
||||
Sets.newHashSet(FSUtils.makeDataFileName(commitTime3, 1, fileId3),
|
||||
FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void streamLatestVersionInRange() throws IOException {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
String commitTime1 = "1";
|
||||
String commitTime2 = "2";
|
||||
String commitTime3 = "3";
|
||||
String commitTime4 = "4";
|
||||
String fileId1 = UUID.randomUUID().toString();
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
|
||||
.createNewFile();
|
||||
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
|
||||
|
||||
// Now we list the entire partition
|
||||
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(statuses.length, 7);
|
||||
|
||||
refreshFsView();
|
||||
List<HoodieDataFile> statuses1 =
|
||||
fsView.streamLatestVersionInRange(statuses, Lists.newArrayList(commitTime2, commitTime3))
|
||||
.collect(Collectors.toList());
|
||||
assertEquals(statuses1.size(), 2);
|
||||
Set<String> filenames = Sets.newHashSet();
|
||||
for (HoodieDataFile status : statuses1) {
|
||||
filenames.add(status.getFileName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void streamLatestVersionsBefore() throws IOException {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
String commitTime1 = "1";
|
||||
String commitTime2 = "2";
|
||||
String commitTime3 = "3";
|
||||
String commitTime4 = "4";
|
||||
String fileId1 = UUID.randomUUID().toString();
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
|
||||
.createNewFile();
|
||||
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
|
||||
|
||||
// Now we list the entire partition
|
||||
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(statuses.length, 7);
|
||||
|
||||
refreshFsView();
|
||||
List<HoodieDataFile> statuses1 =
|
||||
fsView.streamLatestVersionsBeforeOrOn(statuses, commitTime2)
|
||||
.collect(Collectors.toList());
|
||||
assertEquals(statuses1.size(), 2);
|
||||
Set<String> filenames = Sets.newHashSet();
|
||||
for (HoodieDataFile status : statuses1) {
|
||||
filenames.add(status.getFileName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, 1, fileId2)));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void streamLatestVersions() throws IOException {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
String commitTime1 = "1";
|
||||
String commitTime2 = "2";
|
||||
String commitTime3 = "3";
|
||||
String commitTime4 = "4";
|
||||
String fileId1 = UUID.randomUUID().toString();
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3))
|
||||
.createNewFile();
|
||||
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime3 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime4 + ".commit").createNewFile();
|
||||
|
||||
// Now we list the entire partition
|
||||
FileStatus[] statuses = HoodieTestUtils.fs.listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(statuses.length, 7);
|
||||
|
||||
refreshFsView();
|
||||
List<HoodieDataFile> statuses1 =
|
||||
fsView.streamLatestVersions(statuses)
|
||||
.collect(Collectors.toList());
|
||||
assertEquals(statuses1.size(), 3);
|
||||
Set<String> filenames = Sets.newHashSet();
|
||||
for (HoodieDataFile status : statuses1) {
|
||||
filenames.add(status.getFileName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
|
||||
}
|
||||
}
|
||||
@@ -52,13 +52,6 @@ public class TestFSUtils {
|
||||
assertTrue(FSUtils.getCommitTime(fullFileName).equals(commitTime));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetCommitFromCommitFile() {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
String commitFileName = FSUtils.makeCommitFileName(commitTime);
|
||||
assertTrue(FSUtils.getCommitFromCommitFile(commitFileName).equals(commitTime));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetFileNameWithoutMeta() {
|
||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||
|
||||
@@ -90,7 +90,7 @@
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-shade-plugin</artifactId>
|
||||
<version>2.3</version>
|
||||
<version>2.4</version>
|
||||
<executions>
|
||||
<execution>
|
||||
<phase>package</phase>
|
||||
@@ -98,6 +98,7 @@
|
||||
<goal>shade</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<dependencyReducedPomLocation>${project.build.directory}/dependency-reduced-pom.xml</dependencyReducedPomLocation>
|
||||
<minimizeJar>true</minimizeJar>
|
||||
<artifactSet>
|
||||
<includes>
|
||||
|
||||
@@ -16,8 +16,13 @@
|
||||
|
||||
package com.uber.hoodie.hadoop;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.InvalidDatasetException;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
@@ -52,6 +57,7 @@ import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static parquet.filter2.predicate.FilterApi.and;
|
||||
import static parquet.filter2.predicate.FilterApi.binaryColumn;
|
||||
@@ -73,11 +79,11 @@ public class HoodieInputFormat extends MapredParquetInputFormat
|
||||
public FileStatus[] listStatus(JobConf job) throws IOException {
|
||||
// Get all the file status from FileInputFormat and then do the filter
|
||||
FileStatus[] fileStatuses = super.listStatus(job);
|
||||
Map<HoodieTableMetadata, List<FileStatus>> groupedFileStatus = groupFileStatus(fileStatuses);
|
||||
Map<HoodieTableMetaClient, List<FileStatus>> groupedFileStatus = groupFileStatus(fileStatuses);
|
||||
LOG.info("Found a total of " + groupedFileStatus.size() + " groups");
|
||||
List<FileStatus> returns = new ArrayList<FileStatus>();
|
||||
for(Map.Entry<HoodieTableMetadata, List<FileStatus>> entry:groupedFileStatus.entrySet()) {
|
||||
HoodieTableMetadata metadata = entry.getKey();
|
||||
for(Map.Entry<HoodieTableMetaClient, List<FileStatus>> entry:groupedFileStatus.entrySet()) {
|
||||
HoodieTableMetaClient metadata = entry.getKey();
|
||||
if(metadata == null) {
|
||||
// Add all the paths which are not hoodie specific
|
||||
returns.addAll(entry.getValue());
|
||||
@@ -86,31 +92,35 @@ public class HoodieInputFormat extends MapredParquetInputFormat
|
||||
|
||||
FileStatus[] value = entry.getValue().toArray(new FileStatus[entry.getValue().size()]);
|
||||
LOG.info("Hoodie Metadata initialized with completed commit Ts as :" + metadata);
|
||||
String tableName = metadata.getTableName();
|
||||
String tableName = metadata.getTableConfig().getTableName();
|
||||
String mode = HoodieHiveUtil.readMode(Job.getInstance(job), tableName);
|
||||
TableFileSystemView fsView = new ReadOptimizedTableView(FSUtils.getFs(), metadata);
|
||||
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
|
||||
if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {
|
||||
// this is of the form commitTs_partition_sequenceNumber
|
||||
String lastIncrementalTs = HoodieHiveUtil.readStartCommitTime(Job.getInstance(job), tableName);
|
||||
// Total number of commits to return in this batch. Set this to -1 to get all the commits.
|
||||
Integer maxCommits = HoodieHiveUtil.readMaxCommits(Job.getInstance(job), tableName);
|
||||
LOG.info("Last Incremental timestamp was set as " + lastIncrementalTs);
|
||||
List<String>
|
||||
commitsToReturn = metadata.findCommitsAfter(lastIncrementalTs, maxCommits);
|
||||
FileStatus[] filteredFiles =
|
||||
metadata.getLatestVersionInRange(value, commitsToReturn);
|
||||
for (FileStatus filteredFile : filteredFiles) {
|
||||
List<String> commitsToReturn =
|
||||
timeline.findInstantsAfter(lastIncrementalTs, maxCommits)
|
||||
.collect(Collectors.toList());
|
||||
List<HoodieDataFile> filteredFiles =
|
||||
fsView.streamLatestVersionInRange(value, commitsToReturn)
|
||||
.collect(Collectors.toList());
|
||||
for (HoodieDataFile filteredFile : filteredFiles) {
|
||||
LOG.info("Processing incremental hoodie file - " + filteredFile.getPath());
|
||||
returns.add(filteredFile);
|
||||
returns.add(filteredFile.getFileStatus());
|
||||
}
|
||||
LOG.info(
|
||||
"Total paths to process after hoodie incremental filter " + filteredFiles.length);
|
||||
"Total paths to process after hoodie incremental filter " + filteredFiles.size());
|
||||
} else {
|
||||
// filter files on the latest commit found
|
||||
FileStatus[] filteredFiles = metadata.getLatestVersions(value);
|
||||
LOG.info("Total paths to process after hoodie filter " + filteredFiles.length);
|
||||
for (FileStatus filteredFile : filteredFiles) {
|
||||
List<HoodieDataFile> filteredFiles = fsView.streamLatestVersions(value).collect(Collectors.toList());
|
||||
LOG.info("Total paths to process after hoodie filter " + filteredFiles.size());
|
||||
for (HoodieDataFile filteredFile : filteredFiles) {
|
||||
LOG.info("Processing latest hoodie file - " + filteredFile.getPath());
|
||||
returns.add(filteredFile);
|
||||
returns.add(filteredFile.getFileStatus());
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -118,18 +128,18 @@ public class HoodieInputFormat extends MapredParquetInputFormat
|
||||
|
||||
}
|
||||
|
||||
private Map<HoodieTableMetadata, List<FileStatus>> groupFileStatus(FileStatus[] fileStatuses)
|
||||
private Map<HoodieTableMetaClient, List<FileStatus>> groupFileStatus(FileStatus[] fileStatuses)
|
||||
throws IOException {
|
||||
// This assumes the paths for different tables are grouped together
|
||||
Map<HoodieTableMetadata, List<FileStatus>> grouped = new HashMap<>();
|
||||
HoodieTableMetadata metadata = null;
|
||||
Map<HoodieTableMetaClient, List<FileStatus>> grouped = new HashMap<>();
|
||||
HoodieTableMetaClient metadata = null;
|
||||
String nonHoodieBasePath = null;
|
||||
for(FileStatus status:fileStatuses) {
|
||||
if ((metadata == null && nonHoodieBasePath == null) || (metadata == null && !status.getPath().toString()
|
||||
.contains(nonHoodieBasePath)) || (metadata != null && !status.getPath().toString()
|
||||
.contains(metadata.getBasePath()))) {
|
||||
try {
|
||||
metadata = getTableMetadata(status.getPath().getParent());
|
||||
metadata = getTableMetaClient(status.getPath().getParent());
|
||||
nonHoodieBasePath = null;
|
||||
} catch (InvalidDatasetException e) {
|
||||
LOG.info("Handling a non-hoodie path " + status.getPath());
|
||||
@@ -138,7 +148,7 @@ public class HoodieInputFormat extends MapredParquetInputFormat
|
||||
status.getPath().getParent().toString();
|
||||
}
|
||||
if(!grouped.containsKey(metadata)) {
|
||||
grouped.put(metadata, new ArrayList<FileStatus>());
|
||||
grouped.put(metadata, new ArrayList<>());
|
||||
}
|
||||
}
|
||||
grouped.get(metadata).add(status);
|
||||
@@ -242,12 +252,12 @@ public class HoodieInputFormat extends MapredParquetInputFormat
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
private HoodieTableMetadata getTableMetadata(Path dataPath) throws IOException {
|
||||
private HoodieTableMetaClient getTableMetaClient(Path dataPath) throws IOException {
|
||||
FileSystem fs = dataPath.getFileSystem(conf);
|
||||
// TODO - remove this hard-coding. Pass this in job conf, somehow. Or read the Table Location
|
||||
Path baseDir = dataPath.getParent().getParent().getParent();
|
||||
LOG.info("Reading hoodie metadata from path " + baseDir.toString());
|
||||
return new HoodieTableMetadata(fs, baseDir.toString());
|
||||
return new HoodieTableMetaClient(fs, baseDir.toString());
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@@ -41,7 +41,7 @@ public class InputFormatTestUtil {
|
||||
public static File prepareDataset(TemporaryFolder basePath, int numberOfFiles,
|
||||
String commitNumber) throws IOException {
|
||||
basePath.create();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath.getRoot().toString());
|
||||
HoodieTestUtils.init(basePath.getRoot().toString());
|
||||
File partitionPath = basePath.newFolder("2016", "05", "01");
|
||||
for (int i = 0; i < numberOfFiles; i++) {
|
||||
File dataFile =
|
||||
@@ -95,7 +95,7 @@ public class InputFormatTestUtil {
|
||||
public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema, int numberOfFiles, int numberOfRecords,
|
||||
String commitNumber) throws IOException {
|
||||
basePath.create();
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath.getRoot().toString());
|
||||
HoodieTestUtils.init(basePath.getRoot().toString());
|
||||
File partitionPath = basePath.newFolder("2016", "05", "01");
|
||||
AvroParquetWriter parquetWriter;
|
||||
for (int i = 0; i < numberOfFiles; i++) {
|
||||
|
||||
@@ -28,16 +28,8 @@
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.codehaus.mojo</groupId>
|
||||
<artifactId>cobertura-maven-plugin</artifactId>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
<configuration>
|
||||
<source>1.7</source>
|
||||
<target>1.7</target>
|
||||
</configuration>
|
||||
<groupId>org.jacoco</groupId>
|
||||
<artifactId>jacoco-maven-plugin</artifactId>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
|
||||
@@ -18,7 +18,7 @@ package com.uber.hoodie.utilities;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.utilities.exception.HoodieIncrementalPullException;
|
||||
import com.uber.hoodie.utilities.exception.HoodieIncrementalPullSQLException;
|
||||
@@ -44,7 +44,9 @@ import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Scanner;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Utility to pull data after a given commit, based on the supplied HiveQL and save the delta as another hive temporary table.
|
||||
@@ -263,9 +265,10 @@ public class HiveIncrementalPuller {
|
||||
if(!fs.exists(new Path(targetDataPath)) || !fs.exists(new Path(targetDataPath + "/.hoodie"))) {
|
||||
return "0";
|
||||
}
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, targetDataPath);
|
||||
String lastCommit = metadata.getAllCommits().lastCommit();
|
||||
return lastCommit == null ? "0" : lastCommit;
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, targetDataPath);
|
||||
|
||||
Optional<String> lastCommit = metadata.getActiveCommitTimeline().lastInstant();
|
||||
return lastCommit.orElse("0");
|
||||
}
|
||||
|
||||
private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime)
|
||||
@@ -295,12 +298,14 @@ public class HiveIncrementalPuller {
|
||||
}
|
||||
|
||||
private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) throws IOException {
|
||||
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, sourceTableLocation);
|
||||
List<String> commitsToSync =
|
||||
metadata.getAllCommits().findCommitsAfter(config.fromCommitTime, config.maxCommits);
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, sourceTableLocation);
|
||||
List<String> commitsToSync = metadata.getActiveCommitTimeline()
|
||||
.findInstantsAfter(config.fromCommitTime, config.maxCommits)
|
||||
.collect(Collectors.toList());
|
||||
if (commitsToSync.isEmpty()) {
|
||||
log.warn("Nothing to sync. All commits in " + config.sourceTable + " are " + metadata
|
||||
.getAllCommits().getCommitList() + " and from commit time is " + config.fromCommitTime);
|
||||
.getActiveCommitTimeline().getInstants().collect(Collectors.toList())
|
||||
+ " and from commit time is " + config.fromCommitTime);
|
||||
return null;
|
||||
}
|
||||
log.info("Syncing commits " + commitsToSync);
|
||||
|
||||
@@ -22,10 +22,10 @@ import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import com.uber.hoodie.HoodieWriteClient;
|
||||
import com.uber.hoodie.common.HoodieJsonPayload;
|
||||
import com.uber.hoodie.common.model.HoodieCommits;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
@@ -65,19 +65,20 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
private void sync() throws Exception {
|
||||
JavaSparkContext sc = getSparkContext(cfg);
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
HoodieTableMetadata targetHoodieMetadata =
|
||||
new HoodieTableMetadata(fs, cfg.targetPath, cfg.targetTableName);
|
||||
HoodieTableMetaClient targetHoodieMetadata = new HoodieTableMetaClient(fs, cfg.targetPath);
|
||||
HoodieTimeline timeline = targetHoodieMetadata.getActiveCommitTimeline();
|
||||
String lastCommitPulled = findLastCommitPulled(fs, cfg.dataPath);
|
||||
log.info("Last commit pulled on the source dataset is " + lastCommitPulled);
|
||||
if (!targetHoodieMetadata.getAllCommits().isEmpty() && HoodieCommits
|
||||
.isCommit1After(targetHoodieMetadata.getAllCommits().lastCommit(), lastCommitPulled)) {
|
||||
if (!timeline.getInstants().iterator().hasNext() && timeline
|
||||
.compareInstants(timeline.lastInstant().get(), lastCommitPulled,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// this should never be the case
|
||||
throw new IllegalStateException(
|
||||
"Last commit pulled from source table " + lastCommitPulled
|
||||
+ " is before the last commit in the target table " + targetHoodieMetadata
|
||||
.getAllCommits().lastCommit());
|
||||
+ " is before the last commit in the target table " + timeline.lastInstant()
|
||||
.get());
|
||||
}
|
||||
if (!cfg.override && targetHoodieMetadata.getAllCommits().contains(lastCommitPulled)) {
|
||||
if (!cfg.override && timeline.containsOrBeforeTimelineStarts(lastCommitPulled)) {
|
||||
throw new IllegalStateException(
|
||||
"Target Table already has the commit " + lastCommitPulled
|
||||
+ ". Not overriding as cfg.override is false");
|
||||
@@ -98,7 +99,7 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
return commitTimes.get(0);
|
||||
}
|
||||
|
||||
private void syncTill(String lastCommitPulled, HoodieTableMetadata target,
|
||||
private void syncTill(String lastCommitPulled, HoodieTableMetaClient target,
|
||||
JavaSparkContext sc) throws Exception {
|
||||
// Step 1 : Scan incrementally and get the input records as a RDD of source format
|
||||
String dataPath = cfg.dataPath + "/" + lastCommitPulled;
|
||||
@@ -159,13 +160,13 @@ public class HoodieDeltaStreamer implements Serializable {
|
||||
// })
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getHoodieClientConfig(HoodieTableMetadata metadata)
|
||||
private HoodieWriteConfig getHoodieClientConfig(HoodieTableMetaClient metadata)
|
||||
throws Exception {
|
||||
final String schemaStr = Files.toString(new File(cfg.schemaFile), Charset.forName("UTF-8"));
|
||||
return HoodieWriteConfig.newBuilder().withPath(metadata.getBasePath())
|
||||
.withSchema(schemaStr)
|
||||
.withParallelism(cfg.groupByParallelism, cfg.groupByParallelism)
|
||||
.forTable(metadata.getTableName()).withIndexConfig(
|
||||
.forTable(metadata.getTableConfig().getTableName()).withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.build();
|
||||
}
|
||||
|
||||
@@ -19,8 +19,12 @@ package com.uber.hoodie.utilities;
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieCommits;
|
||||
import com.uber.hoodie.common.model.HoodieTableMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableConfig;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -40,6 +44,8 @@ import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Hoodie snapshot copy job which copies latest files from all partitions to another place, for snapshot backup.
|
||||
@@ -57,11 +63,15 @@ public class HoodieSnapshotCopier implements Serializable {
|
||||
|
||||
public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDir) throws IOException {
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
final HoodieTableMetadata tableMetadata = new HoodieTableMetadata(fs, baseDir);
|
||||
|
||||
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs, baseDir);
|
||||
final TableFileSystemView fsView = new ReadOptimizedTableView(fs, tableMetadata);
|
||||
// Get the latest commit
|
||||
final String latestCommit = tableMetadata.getAllCommits().lastCommit();
|
||||
logger.info(String.format("Starting to snapshot latest version files which are also no-late-than %s.", latestCommit));
|
||||
final Optional<String> latestCommit = tableMetadata.getActiveCommitTimeline().lastInstant();
|
||||
if(!latestCommit.isPresent()) {
|
||||
logger.warn("No commits present. Nothing to snapshot");
|
||||
} else {
|
||||
logger.info(String.format("Starting to snapshot latest version files which are also no-late-than %s.", latestCommit.get()));
|
||||
}
|
||||
|
||||
List<String> partitions = FSUtils.getAllPartitionPaths(fs, baseDir);
|
||||
if (partitions.size() > 0) {
|
||||
@@ -80,8 +90,10 @@ public class HoodieSnapshotCopier implements Serializable {
|
||||
// Only take latest version files <= latestCommit.
|
||||
FileSystem fs = FSUtils.getFs();
|
||||
List<Tuple2<String, String>> filePaths = new ArrayList<>();
|
||||
for (FileStatus fileStatus : tableMetadata.getLatestVersionInPartition(fs, partition, latestCommit)) {
|
||||
filePaths.add(new Tuple2<>(partition, fileStatus.getPath().toString()));
|
||||
for (HoodieDataFile hoodieDataFile : fsView
|
||||
.streamLatestVersionInPartition(partition, latestCommit.get())
|
||||
.collect(Collectors.toList())) {
|
||||
filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath()));
|
||||
}
|
||||
return filePaths.iterator();
|
||||
}
|
||||
@@ -102,22 +114,25 @@ public class HoodieSnapshotCopier implements Serializable {
|
||||
});
|
||||
|
||||
// Also copy the .commit files
|
||||
logger.info(String.format("Copying .commit files which are no-late-than %s.", latestCommit));
|
||||
logger.info(String.format("Copying .commit files which are no-late-than %s.", latestCommit.get()));
|
||||
FileStatus[] commitFilesToCopy = fs.listStatus(
|
||||
new Path(baseDir + "/" + HoodieTableMetadata.METAFOLDER_NAME), new PathFilter() {
|
||||
new Path(baseDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME), new PathFilter() {
|
||||
@Override
|
||||
public boolean accept(Path commitFilePath) {
|
||||
if (commitFilePath.getName().equals(HoodieTableMetadata.HOODIE_PROPERTIES_FILE)) {
|
||||
if (commitFilePath.getName().equals(HoodieTableConfig.HOODIE_PROPERTIES_FILE)) {
|
||||
return true;
|
||||
} else {
|
||||
String commitTime = FSUtils.getCommitFromCommitFile(commitFilePath.getName());
|
||||
return HoodieCommits.isCommit1BeforeOrOn(commitTime, latestCommit);
|
||||
String commitTime =
|
||||
FSUtils.getCommitFromCommitFile(commitFilePath.getName());
|
||||
return tableMetadata.getActiveCommitTimeline()
|
||||
.compareInstants(commitTime, latestCommit.get(), HoodieTimeline.GREATER);
|
||||
}
|
||||
}
|
||||
});
|
||||
for (FileStatus commitStatus : commitFilesToCopy) {
|
||||
Path targetFilePath =
|
||||
new Path(outputDir + "/" + HoodieTableMetadata.METAFOLDER_NAME + "/" + commitStatus.getPath().getName());
|
||||
Path targetFilePath = new Path(
|
||||
outputDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitStatus
|
||||
.getPath().getName());
|
||||
if (! fs.exists(targetFilePath.getParent())) {
|
||||
fs.mkdirs(targetFilePath.getParent());
|
||||
}
|
||||
|
||||
@@ -46,7 +46,7 @@ public class TestHoodieSnapshotCopier {
|
||||
folder.create();
|
||||
rootPath = folder.getRoot().getAbsolutePath();
|
||||
basePath = rootPath + "/" + HoodieTestUtils.RAW_TRIPS_TEST_NAME;
|
||||
HoodieTestUtils.initializeHoodieDirectory(basePath);
|
||||
HoodieTestUtils.init(basePath);
|
||||
outputPath = rootPath + "/output";
|
||||
fs = FSUtils.getFs();
|
||||
// Start a local Spark job
|
||||
|
||||
93
pom.xml
93
pom.xml
@@ -126,8 +126,8 @@
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-compiler-plugin</artifactId>
|
||||
<configuration>
|
||||
<source>1.7</source>
|
||||
<target>1.7</target>
|
||||
<source>1.8</source>
|
||||
<target>1.8</target>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
@@ -146,9 +146,15 @@
|
||||
<artifactId>maven-surefire-plugin</artifactId>
|
||||
<version>${maven-surefire-plugin.version}</version>
|
||||
<configuration>
|
||||
<!-- Sets the VM argument line used when unit tests are run. -->
|
||||
<argLine>${surefireArgLine}</argLine>
|
||||
<systemPropertyVariables>
|
||||
<log4j.configuration>file:${project.build.testOutputDirectory}/log4j-surefire.properties</log4j.configuration>
|
||||
</systemPropertyVariables>
|
||||
<!-- Excludes integration tests when unit tests are run. -->
|
||||
<excludes>
|
||||
<exclude>**/IT*.java</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
@@ -164,24 +170,74 @@
|
||||
<version>${maven-jar-plugin.version}</version>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.codehaus.mojo</groupId>
|
||||
<artifactId>cobertura-maven-plugin</artifactId>
|
||||
<version>2.7</version>
|
||||
<configuration>
|
||||
<formats>
|
||||
<format>html</format>
|
||||
<format>xml</format>
|
||||
</formats>
|
||||
</configuration>
|
||||
<groupId>org.jacoco</groupId>
|
||||
<artifactId>jacoco-maven-plugin</artifactId>
|
||||
<version>0.7.8</version>
|
||||
<executions>
|
||||
<!--
|
||||
Prepares the property pointing to the JaCoCo runtime agent which
|
||||
is passed as VM argument when Maven the Surefire plugin is executed.
|
||||
-->
|
||||
<execution>
|
||||
<id>pre-unit-test</id>
|
||||
<goals>
|
||||
<goal>prepare-agent</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<!-- Sets the path to the file which contains the execution data. -->
|
||||
<destFile>${project.build.directory}/coverage-reports/jacoco-ut.exec</destFile>
|
||||
<!--
|
||||
Sets the name of the property containing the settings
|
||||
for JaCoCo runtime agent.
|
||||
-->
|
||||
<propertyName>surefireArgLine</propertyName>
|
||||
</configuration>
|
||||
</execution>
|
||||
<!--
|
||||
Ensures that the code coverage report for unit tests is created after
|
||||
unit tests have been run.
|
||||
-->
|
||||
<execution>
|
||||
<id>post-unit-test</id>
|
||||
<phase>test</phase>
|
||||
<goals>
|
||||
<goal>cobertura</goal>
|
||||
<goal>report</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<!-- Sets the path to the file which contains the execution data. -->
|
||||
<dataFile>${project.build.directory}/coverage-reports/jacoco-ut.exec</dataFile>
|
||||
<!-- Sets the output directory for the code coverage report. -->
|
||||
<outputDirectory>${project.reporting.outputDirectory}/jacoco-ut</outputDirectory>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<!--<plugin>-->
|
||||
<!--<groupId>org.codehaus.mojo</groupId>-->
|
||||
<!--<artifactId>cobertura-maven-plugin</artifactId>-->
|
||||
<!--<version>2.7</version>-->
|
||||
<!--<configuration>-->
|
||||
<!--<formats>-->
|
||||
<!--<format>html</format>-->
|
||||
<!--<format>xml</format>-->
|
||||
<!--</formats>-->
|
||||
<!--</configuration>-->
|
||||
<!--<executions>-->
|
||||
<!--<execution>-->
|
||||
<!--<phase>test</phase>-->
|
||||
<!--<goals>-->
|
||||
<!--<goal>cobertura</goal>-->
|
||||
<!--</goals>-->
|
||||
<!--</execution>-->
|
||||
<!--</executions>-->
|
||||
<!--<dependencies>-->
|
||||
<!--<dependency>-->
|
||||
<!--<groupId>org.ow2.asm</groupId>-->
|
||||
<!--<artifactId>asm</artifactId>-->
|
||||
<!--<version>5.0.3</version>-->
|
||||
<!--</dependency>-->
|
||||
<!--</dependencies>-->
|
||||
<!--</plugin>-->
|
||||
<plugin>
|
||||
<!-- excludes are inherited -->
|
||||
<groupId>org.apache.rat</groupId>
|
||||
@@ -422,6 +478,11 @@
|
||||
<artifactId>hive-metastore</artifactId>
|
||||
<version>${hive.version}-cdh${cdh.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.commons</groupId>
|
||||
<artifactId>commons-lang3</artifactId>
|
||||
<version>3.4</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
@@ -446,6 +507,14 @@
|
||||
<scope>test</scope>
|
||||
<version>1.10.19</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<!--Used to test execution in task executor after de-serializing-->
|
||||
<groupId>com.esotericsoftware</groupId>
|
||||
<artifactId>kryo</artifactId>
|
||||
<version>4.0.0</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
</dependencies>
|
||||
|
||||
</dependencyManagement>
|
||||
|
||||
Reference in New Issue
Block a user