Reformatting code per Google Code Style all over
This commit is contained in:
committed by
vinoth chandar
parent
5a62480a92
commit
e45679f5e2
@@ -15,7 +15,9 @@
|
|||||||
~ limitations under the License.
|
~ limitations under the License.
|
||||||
-->
|
-->
|
||||||
|
|
||||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
||||||
|
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||||
|
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||||
<parent>
|
<parent>
|
||||||
<artifactId>hoodie</artifactId>
|
<artifactId>hoodie</artifactId>
|
||||||
<groupId>com.uber.hoodie</groupId>
|
<groupId>com.uber.hoodie</groupId>
|
||||||
@@ -117,7 +119,7 @@
|
|||||||
</goals>
|
</goals>
|
||||||
</execution>
|
</execution>
|
||||||
</executions>
|
</executions>
|
||||||
</plugin>
|
</plugin>
|
||||||
<plugin>
|
<plugin>
|
||||||
<groupId>org.apache.rat</groupId>
|
<groupId>org.apache.rat</groupId>
|
||||||
<artifactId>apache-rat-plugin</artifactId>
|
<artifactId>apache-rat-plugin</artifactId>
|
||||||
|
|||||||
@@ -17,38 +17,38 @@
|
|||||||
package com.uber.hoodie.cli;
|
package com.uber.hoodie.cli;
|
||||||
|
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
|
import java.io.IOException;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
public class HoodieCLI {
|
public class HoodieCLI {
|
||||||
public static Configuration conf;
|
|
||||||
public static FileSystem fs;
|
public static Configuration conf;
|
||||||
public static CLIState state = CLIState.INIT;
|
public static FileSystem fs;
|
||||||
public static HoodieTableMetaClient tableMetadata;
|
public static CLIState state = CLIState.INIT;
|
||||||
public static HoodieTableMetaClient syncTableMetadata;
|
public static HoodieTableMetaClient tableMetadata;
|
||||||
|
public static HoodieTableMetaClient syncTableMetadata;
|
||||||
|
|
||||||
|
|
||||||
public enum CLIState {
|
public enum CLIState {
|
||||||
INIT, DATASET, SYNC
|
INIT, DATASET, SYNC
|
||||||
|
}
|
||||||
|
|
||||||
|
public static boolean initConf() {
|
||||||
|
if (HoodieCLI.conf == null) {
|
||||||
|
HoodieCLI.conf = new Configuration();
|
||||||
|
return true;
|
||||||
}
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
public static boolean initConf() {
|
public static void initFS(boolean force) throws IOException {
|
||||||
if (HoodieCLI.conf == null) {
|
if (fs == null || force) {
|
||||||
HoodieCLI.conf = new Configuration();
|
fs = FileSystem.get(conf);
|
||||||
return true;
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public static void initFS(boolean force) throws IOException {
|
public static void setTableMetadata(HoodieTableMetaClient tableMetadata) {
|
||||||
if(fs == null || force) {
|
HoodieCLI.tableMetadata = tableMetadata;
|
||||||
fs = FileSystem.get(conf);
|
}
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void setTableMetadata(HoodieTableMetaClient tableMetadata) {
|
|
||||||
HoodieCLI.tableMetadata = tableMetadata;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -25,13 +25,13 @@ import org.springframework.stereotype.Component;
|
|||||||
@Order(Ordered.HIGHEST_PRECEDENCE)
|
@Order(Ordered.HIGHEST_PRECEDENCE)
|
||||||
public class HoodieHistoryFileNameProvider extends DefaultHistoryFileNameProvider {
|
public class HoodieHistoryFileNameProvider extends DefaultHistoryFileNameProvider {
|
||||||
|
|
||||||
public String getHistoryFileName() {
|
public String getHistoryFileName() {
|
||||||
return "hoodie-cmd.log";
|
return "hoodie-cmd.log";
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getProviderName() {
|
public String getProviderName() {
|
||||||
return "Hoodie file name provider";
|
return "Hoodie file name provider";
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,18 +17,17 @@
|
|||||||
package com.uber.hoodie.cli;
|
package com.uber.hoodie.cli;
|
||||||
|
|
||||||
import dnl.utils.text.table.TextTable;
|
import dnl.utils.text.table.TextTable;
|
||||||
|
|
||||||
import java.io.ByteArrayOutputStream;
|
import java.io.ByteArrayOutputStream;
|
||||||
import java.io.PrintStream;
|
import java.io.PrintStream;
|
||||||
import java.nio.charset.Charset;
|
import java.nio.charset.Charset;
|
||||||
|
|
||||||
public class HoodiePrintHelper {
|
public class HoodiePrintHelper {
|
||||||
|
|
||||||
public static String print(String[] header, String[][] rows) {
|
public static String print(String[] header, String[][] rows) {
|
||||||
TextTable textTable = new TextTable(header, rows);
|
TextTable textTable = new TextTable(header, rows);
|
||||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||||
PrintStream ps = new PrintStream(baos);
|
PrintStream ps = new PrintStream(baos);
|
||||||
textTable.printTable(ps, 4);
|
textTable.printTable(ps, 4);
|
||||||
return new String(baos.toByteArray(), Charset.forName("utf-8"));
|
return new String(baos.toByteArray(), Charset.forName("utf-8"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,7 +16,6 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.cli;
|
package com.uber.hoodie.cli;
|
||||||
|
|
||||||
import com.uber.hoodie.common.table.HoodieTableConfig;
|
|
||||||
import org.springframework.core.Ordered;
|
import org.springframework.core.Ordered;
|
||||||
import org.springframework.core.annotation.Order;
|
import org.springframework.core.annotation.Order;
|
||||||
import org.springframework.shell.plugin.support.DefaultPromptProvider;
|
import org.springframework.shell.plugin.support.DefaultPromptProvider;
|
||||||
@@ -26,27 +25,27 @@ import org.springframework.stereotype.Component;
|
|||||||
@Order(Ordered.HIGHEST_PRECEDENCE)
|
@Order(Ordered.HIGHEST_PRECEDENCE)
|
||||||
public class HoodiePrompt extends DefaultPromptProvider {
|
public class HoodiePrompt extends DefaultPromptProvider {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getPrompt() {
|
public String getPrompt() {
|
||||||
if (HoodieCLI.tableMetadata != null) {
|
if (HoodieCLI.tableMetadata != null) {
|
||||||
String tableName = HoodieCLI.tableMetadata.getTableConfig().getTableName();
|
String tableName = HoodieCLI.tableMetadata.getTableConfig().getTableName();
|
||||||
switch (HoodieCLI.state) {
|
switch (HoodieCLI.state) {
|
||||||
case INIT:
|
case INIT:
|
||||||
return "hoodie->";
|
return "hoodie->";
|
||||||
case DATASET:
|
case DATASET:
|
||||||
return "hoodie:" + tableName + "->";
|
return "hoodie:" + tableName + "->";
|
||||||
case SYNC:
|
case SYNC:
|
||||||
return "hoodie:" + tableName + " <==> "
|
return "hoodie:" + tableName + " <==> "
|
||||||
+ HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->";
|
+ HoodieCLI.syncTableMetadata.getTableConfig().getTableName() + "->";
|
||||||
}
|
}
|
||||||
return "hoodie:" + tableName + "->";
|
return "hoodie:" + tableName + "->";
|
||||||
}
|
|
||||||
return "hoodie->";
|
|
||||||
}
|
}
|
||||||
|
return "hoodie->";
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getProviderName() {
|
public String getProviderName() {
|
||||||
return "Hoodie provider";
|
return "Hoodie provider";
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -22,34 +22,39 @@ import org.springframework.shell.plugin.support.DefaultBannerProvider;
|
|||||||
import org.springframework.shell.support.util.OsUtils;
|
import org.springframework.shell.support.util.OsUtils;
|
||||||
import org.springframework.stereotype.Component;
|
import org.springframework.stereotype.Component;
|
||||||
|
|
||||||
@Component @Order(Ordered.HIGHEST_PRECEDENCE) public class HoodieSplashScreen
|
@Component
|
||||||
|
@Order(Ordered.HIGHEST_PRECEDENCE)
|
||||||
|
public class HoodieSplashScreen
|
||||||
extends DefaultBannerProvider {
|
extends DefaultBannerProvider {
|
||||||
private static String screen = "============================================" + OsUtils.LINE_SEPARATOR +
|
|
||||||
"* *" + OsUtils.LINE_SEPARATOR +
|
|
||||||
"* _ _ _ _ *" + OsUtils.LINE_SEPARATOR +
|
|
||||||
"* | | | | | (_) *" + OsUtils.LINE_SEPARATOR +
|
|
||||||
"* | |__| | ___ ___ __| |_ ___ *" + OsUtils.LINE_SEPARATOR +
|
|
||||||
"* | __ |/ _ \\ / _ \\ / _` | |/ _ \\ *" +
|
|
||||||
OsUtils.LINE_SEPARATOR +
|
|
||||||
"* | | | | (_) | (_) | (_| | | __/ *" + OsUtils.LINE_SEPARATOR +
|
|
||||||
"* |_| |_|\\___/ \\___/ \\__,_|_|\\___| *" +
|
|
||||||
OsUtils.LINE_SEPARATOR +
|
|
||||||
"* *" + OsUtils.LINE_SEPARATOR +
|
|
||||||
"============================================" + OsUtils.LINE_SEPARATOR;
|
|
||||||
|
|
||||||
public String getBanner() {
|
private static String screen =
|
||||||
return screen;
|
"============================================" + OsUtils.LINE_SEPARATOR +
|
||||||
}
|
"* *" + OsUtils.LINE_SEPARATOR +
|
||||||
|
"* _ _ _ _ *" + OsUtils.LINE_SEPARATOR +
|
||||||
|
"* | | | | | (_) *" + OsUtils.LINE_SEPARATOR +
|
||||||
|
"* | |__| | ___ ___ __| |_ ___ *" + OsUtils.LINE_SEPARATOR +
|
||||||
|
"* | __ |/ _ \\ / _ \\ / _` | |/ _ \\ *" +
|
||||||
|
OsUtils.LINE_SEPARATOR +
|
||||||
|
"* | | | | (_) | (_) | (_| | | __/ *" + OsUtils.LINE_SEPARATOR +
|
||||||
|
"* |_| |_|\\___/ \\___/ \\__,_|_|\\___| *" +
|
||||||
|
OsUtils.LINE_SEPARATOR +
|
||||||
|
"* *" + OsUtils.LINE_SEPARATOR +
|
||||||
|
"============================================" + OsUtils.LINE_SEPARATOR;
|
||||||
|
|
||||||
public String getVersion() {
|
public String getBanner() {
|
||||||
return "1.0";
|
return screen;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getWelcomeMessage() {
|
public String getVersion() {
|
||||||
return "Welcome to Hoodie CLI. Please type help if you are looking for help. ";
|
return "1.0";
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override public String getProviderName() {
|
public String getWelcomeMessage() {
|
||||||
return "Hoodie Banner";
|
return "Welcome to Hoodie CLI. Please type help if you are looking for help. ";
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getProviderName() {
|
||||||
|
return "Hoodie Banner";
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,18 +16,16 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.cli;
|
package com.uber.hoodie.cli;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
import org.springframework.shell.Bootstrap;
|
import org.springframework.shell.Bootstrap;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
public class Main {
|
public class Main {
|
||||||
/**
|
|
||||||
* Main class that delegates to Spring Shell's Bootstrap class in order to simplify debugging inside an IDE
|
/**
|
||||||
*
|
* Main class that delegates to Spring Shell's Bootstrap class in order to simplify debugging
|
||||||
* @param args
|
* inside an IDE
|
||||||
* @throws IOException
|
*/
|
||||||
*/
|
public static void main(String[] args) throws IOException {
|
||||||
public static void main(String[] args) throws IOException {
|
Bootstrap.main(args);
|
||||||
Bootstrap.main(args);
|
}
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -24,6 +24,10 @@ import com.uber.hoodie.common.table.HoodieTimeline;
|
|||||||
import com.uber.hoodie.common.table.log.HoodieLogFormat;
|
import com.uber.hoodie.common.table.log.HoodieLogFormat;
|
||||||
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
import org.apache.avro.generic.IndexedRecord;
|
import org.apache.avro.generic.IndexedRecord;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
@@ -34,90 +38,90 @@ import org.springframework.shell.core.annotation.CliCommand;
|
|||||||
import org.springframework.shell.core.annotation.CliOption;
|
import org.springframework.shell.core.annotation.CliOption;
|
||||||
import org.springframework.stereotype.Component;
|
import org.springframework.stereotype.Component;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
@Component
|
@Component
|
||||||
public class ArchivedCommitsCommand implements CommandMarker {
|
public class ArchivedCommitsCommand implements CommandMarker {
|
||||||
|
|
||||||
@CliAvailabilityIndicator({"show archived commits"})
|
@CliAvailabilityIndicator({"show archived commits"})
|
||||||
public boolean isShowArchivedCommitAvailable() {
|
public boolean isShowArchivedCommitAvailable() {
|
||||||
return HoodieCLI.tableMetadata != null;
|
return HoodieCLI.tableMetadata != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliCommand(value = "show archived commits", help = "Read commits from archived files and show details")
|
||||||
|
public String showCommits(
|
||||||
|
@CliOption(key = {
|
||||||
|
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10")
|
||||||
|
final Integer limit) throws IOException {
|
||||||
|
|
||||||
|
System.out
|
||||||
|
.println("===============> Showing only " + limit + " archived commits <===============");
|
||||||
|
FileStatus[] fsStatuses = FSUtils.getFs().globStatus(
|
||||||
|
new Path(HoodieCLI.tableMetadata.getBasePath() + "/.hoodie/.commits_.archive*"));
|
||||||
|
List<String[]> allCommits = new ArrayList<>();
|
||||||
|
for (FileStatus fs : fsStatuses) {
|
||||||
|
//read the archived file
|
||||||
|
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(),
|
||||||
|
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema(), false);
|
||||||
|
|
||||||
|
List<IndexedRecord> readRecords = new ArrayList<>();
|
||||||
|
//read the avro blocks
|
||||||
|
while (reader.hasNext()) {
|
||||||
|
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
|
||||||
|
List<IndexedRecord> records = blk.getRecords();
|
||||||
|
readRecords.addAll(records);
|
||||||
|
}
|
||||||
|
List<String[]> readCommits = readRecords.stream().map(r -> (GenericRecord) r)
|
||||||
|
.map(r -> readCommit(r)).limit(limit).collect(Collectors.toList());
|
||||||
|
allCommits.addAll(readCommits);
|
||||||
}
|
}
|
||||||
|
return HoodiePrintHelper.print(
|
||||||
|
new String[]{"CommitTime", "CommitType", "CommitDetails"},
|
||||||
|
allCommits.toArray(new String[allCommits.size()][]));
|
||||||
|
}
|
||||||
|
|
||||||
@CliCommand(value = "show archived commits", help = "Read commits from archived files and show details")
|
private String[] readCommit(GenericRecord record) {
|
||||||
public String showCommits(
|
List<String> commitDetails = new ArrayList<>();
|
||||||
@CliOption(key = {"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10")
|
try {
|
||||||
final Integer limit) throws IOException {
|
switch (record.get("actionType").toString()) {
|
||||||
|
case HoodieTimeline.CLEAN_ACTION: {
|
||||||
System.out.println("===============> Showing only " + limit + " archived commits <===============");
|
commitDetails.add(record.get("commitTime").toString());
|
||||||
FileStatus [] fsStatuses = FSUtils.getFs().globStatus(new Path(HoodieCLI.tableMetadata.getBasePath() + "/.hoodie/.commits_.archive*"));
|
commitDetails.add(record.get("actionType").toString());
|
||||||
List<String[]> allCommits = new ArrayList<>();
|
commitDetails.add(record.get("hoodieCleanMetadata").toString());
|
||||||
for(FileStatus fs : fsStatuses) {
|
break;
|
||||||
//read the archived file
|
|
||||||
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(),
|
|
||||||
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema(), false);
|
|
||||||
|
|
||||||
List<IndexedRecord> readRecords = new ArrayList<>();
|
|
||||||
//read the avro blocks
|
|
||||||
while (reader.hasNext()) {
|
|
||||||
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
|
|
||||||
List<IndexedRecord> records = blk.getRecords();
|
|
||||||
readRecords.addAll(records);
|
|
||||||
}
|
|
||||||
List<String[]> readCommits = readRecords.stream().map(r -> (GenericRecord)r).map(r -> readCommit(r)).limit(limit).collect(Collectors.toList());
|
|
||||||
allCommits.addAll(readCommits);
|
|
||||||
}
|
}
|
||||||
return HoodiePrintHelper.print(
|
case HoodieTimeline.COMMIT_ACTION: {
|
||||||
new String[] {"CommitTime", "CommitType", "CommitDetails"}, allCommits.toArray(new String[allCommits.size()][]));
|
commitDetails.add(record.get("commitTime").toString());
|
||||||
}
|
commitDetails.add(record.get("actionType").toString());
|
||||||
|
commitDetails.add(record.get("hoodieCommitMetadata").toString());
|
||||||
private String[] readCommit(GenericRecord record) {
|
break;
|
||||||
List<String> commitDetails = new ArrayList<>();
|
|
||||||
try {
|
|
||||||
switch (record.get("actionType").toString()) {
|
|
||||||
case HoodieTimeline.CLEAN_ACTION: {
|
|
||||||
commitDetails.add(record.get("commitTime").toString());
|
|
||||||
commitDetails.add(record.get("actionType").toString());
|
|
||||||
commitDetails.add(record.get("hoodieCleanMetadata").toString());
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case HoodieTimeline.COMMIT_ACTION: {
|
|
||||||
commitDetails.add(record.get("commitTime").toString());
|
|
||||||
commitDetails.add(record.get("actionType").toString());
|
|
||||||
commitDetails.add(record.get("hoodieCommitMetadata").toString());
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case HoodieTimeline.COMPACTION_ACTION: {
|
|
||||||
commitDetails.add(record.get("commitTime").toString());
|
|
||||||
commitDetails.add(record.get("actionType").toString());
|
|
||||||
commitDetails.add(record.get("hoodieCompactionMetadata").toString());
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case HoodieTimeline.DELTA_COMMIT_ACTION: {
|
|
||||||
commitDetails.add(record.get("commitTime").toString());
|
|
||||||
commitDetails.add(record.get("actionType").toString());
|
|
||||||
commitDetails.add(record.get("hoodieCommitMetadata").toString());
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case HoodieTimeline.ROLLBACK_ACTION: {
|
|
||||||
commitDetails.add(record.get("commitTime").toString());
|
|
||||||
commitDetails.add(record.get("actionType").toString());
|
|
||||||
commitDetails.add(record.get("hoodieRollbackMetadata").toString());
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case HoodieTimeline.SAVEPOINT_ACTION: {
|
|
||||||
commitDetails.add(record.get("commitTime").toString());
|
|
||||||
commitDetails.add(record.get("actionType").toString());
|
|
||||||
commitDetails.add(record.get("hoodieSavePointMetadata").toString());
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} catch (Exception e) {
|
|
||||||
e.printStackTrace();
|
|
||||||
}
|
}
|
||||||
return commitDetails.toArray(new String[commitDetails.size()]);
|
case HoodieTimeline.COMPACTION_ACTION: {
|
||||||
|
commitDetails.add(record.get("commitTime").toString());
|
||||||
|
commitDetails.add(record.get("actionType").toString());
|
||||||
|
commitDetails.add(record.get("hoodieCompactionMetadata").toString());
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case HoodieTimeline.DELTA_COMMIT_ACTION: {
|
||||||
|
commitDetails.add(record.get("commitTime").toString());
|
||||||
|
commitDetails.add(record.get("actionType").toString());
|
||||||
|
commitDetails.add(record.get("hoodieCommitMetadata").toString());
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case HoodieTimeline.ROLLBACK_ACTION: {
|
||||||
|
commitDetails.add(record.get("commitTime").toString());
|
||||||
|
commitDetails.add(record.get("actionType").toString());
|
||||||
|
commitDetails.add(record.get("hoodieRollbackMetadata").toString());
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case HoodieTimeline.SAVEPOINT_ACTION: {
|
||||||
|
commitDetails.add(record.get("commitTime").toString());
|
||||||
|
commitDetails.add(record.get("actionType").toString());
|
||||||
|
commitDetails.add(record.get("hoodieSavePointMetadata").toString());
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
e.printStackTrace();
|
||||||
}
|
}
|
||||||
|
return commitDetails.toArray(new String[commitDetails.size()]);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
@@ -24,89 +24,90 @@ import com.uber.hoodie.common.table.HoodieTimeline;
|
|||||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||||
import com.uber.hoodie.common.util.AvroUtils;
|
import com.uber.hoodie.common.util.AvroUtils;
|
||||||
import org.springframework.shell.core.CommandMarker;
|
|
||||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
|
||||||
import org.springframework.shell.core.annotation.CliCommand;
|
|
||||||
import org.springframework.shell.core.annotation.CliOption;
|
|
||||||
import org.springframework.stereotype.Component;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
import org.springframework.shell.core.CommandMarker;
|
||||||
|
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||||
|
import org.springframework.shell.core.annotation.CliCommand;
|
||||||
|
import org.springframework.shell.core.annotation.CliOption;
|
||||||
|
import org.springframework.stereotype.Component;
|
||||||
|
|
||||||
@Component
|
@Component
|
||||||
public class CleansCommand implements CommandMarker {
|
public class CleansCommand implements CommandMarker {
|
||||||
@CliAvailabilityIndicator({"cleans show"})
|
|
||||||
public boolean isShowAvailable() {
|
|
||||||
return HoodieCLI.tableMetadata != null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@CliAvailabilityIndicator({"cleans refresh"})
|
@CliAvailabilityIndicator({"cleans show"})
|
||||||
public boolean isRefreshAvailable() {
|
public boolean isShowAvailable() {
|
||||||
return HoodieCLI.tableMetadata != null;
|
return HoodieCLI.tableMetadata != null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@CliAvailabilityIndicator({"clean showpartitions"})
|
@CliAvailabilityIndicator({"cleans refresh"})
|
||||||
public boolean isCommitShowAvailable() {
|
public boolean isRefreshAvailable() {
|
||||||
return HoodieCLI.tableMetadata != null;
|
return HoodieCLI.tableMetadata != null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@CliCommand(value = "cleans show", help = "Show the cleans")
|
@CliAvailabilityIndicator({"clean showpartitions"})
|
||||||
public String showCleans() throws IOException {
|
public boolean isCommitShowAvailable() {
|
||||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
return HoodieCLI.tableMetadata != null;
|
||||||
HoodieTimeline timeline = activeTimeline.getCleanerTimeline().filterCompletedInstants();
|
}
|
||||||
List<HoodieInstant> cleans = timeline.getInstants().collect(Collectors.toList());
|
|
||||||
String[][] rows = new String[cleans.size()][];
|
|
||||||
Collections.reverse(cleans);
|
|
||||||
for (int i = 0; i < cleans.size(); i++) {
|
|
||||||
HoodieInstant clean = cleans.get(i);
|
|
||||||
HoodieCleanMetadata cleanMetadata =
|
|
||||||
AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get());
|
|
||||||
rows[i] = new String[] {clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(),
|
|
||||||
String.valueOf(cleanMetadata.getTotalFilesDeleted()),
|
|
||||||
String.valueOf(cleanMetadata.getTimeTakenInMillis())};
|
|
||||||
}
|
|
||||||
return HoodiePrintHelper.print(
|
|
||||||
new String[] {"CleanTime", "EarliestCommandRetained", "Total Files Deleted",
|
|
||||||
"Total Time Taken"}, rows);
|
|
||||||
}
|
|
||||||
|
|
||||||
@CliCommand(value = "cleans refresh", help = "Refresh the commits")
|
@CliCommand(value = "cleans show", help = "Show the cleans")
|
||||||
public String refreshCleans() throws IOException {
|
public String showCleans() throws IOException {
|
||||||
HoodieTableMetaClient metadata =
|
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||||
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
|
HoodieTimeline timeline = activeTimeline.getCleanerTimeline().filterCompletedInstants();
|
||||||
HoodieCLI.setTableMetadata(metadata);
|
List<HoodieInstant> cleans = timeline.getInstants().collect(Collectors.toList());
|
||||||
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
|
String[][] rows = new String[cleans.size()][];
|
||||||
|
Collections.reverse(cleans);
|
||||||
|
for (int i = 0; i < cleans.size(); i++) {
|
||||||
|
HoodieInstant clean = cleans.get(i);
|
||||||
|
HoodieCleanMetadata cleanMetadata =
|
||||||
|
AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get());
|
||||||
|
rows[i] = new String[]{clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(),
|
||||||
|
String.valueOf(cleanMetadata.getTotalFilesDeleted()),
|
||||||
|
String.valueOf(cleanMetadata.getTimeTakenInMillis())};
|
||||||
}
|
}
|
||||||
|
return HoodiePrintHelper.print(
|
||||||
|
new String[]{"CleanTime", "EarliestCommandRetained", "Total Files Deleted",
|
||||||
|
"Total Time Taken"}, rows);
|
||||||
|
}
|
||||||
|
|
||||||
@CliCommand(value = "clean showpartitions", help = "Show partition level details of a clean")
|
@CliCommand(value = "cleans refresh", help = "Refresh the commits")
|
||||||
public String showCleanPartitions(
|
public String refreshCleans() throws IOException {
|
||||||
@CliOption(key = {"clean"}, help = "clean to show")
|
HoodieTableMetaClient metadata =
|
||||||
final String commitTime) throws Exception {
|
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
|
||||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
HoodieCLI.setTableMetadata(metadata);
|
||||||
HoodieTimeline timeline = activeTimeline.getCleanerTimeline().filterCompletedInstants();
|
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
|
||||||
HoodieInstant cleanInstant =
|
}
|
||||||
new HoodieInstant(false, HoodieTimeline.CLEAN_ACTION, commitTime);
|
|
||||||
|
|
||||||
if (!timeline.containsInstant(cleanInstant)) {
|
@CliCommand(value = "clean showpartitions", help = "Show partition level details of a clean")
|
||||||
return "Clean " + commitTime + " not found in metadata " + timeline;
|
public String showCleanPartitions(
|
||||||
}
|
@CliOption(key = {"clean"}, help = "clean to show")
|
||||||
HoodieCleanMetadata cleanMetadata =
|
final String commitTime) throws Exception {
|
||||||
AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(cleanInstant).get());
|
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||||
List<String[]> rows = new ArrayList<>();
|
HoodieTimeline timeline = activeTimeline.getCleanerTimeline().filterCompletedInstants();
|
||||||
for (Map.Entry<String, HoodieCleanPartitionMetadata> entry : cleanMetadata.getPartitionMetadata().entrySet()) {
|
HoodieInstant cleanInstant =
|
||||||
String path = entry.getKey();
|
new HoodieInstant(false, HoodieTimeline.CLEAN_ACTION, commitTime);
|
||||||
HoodieCleanPartitionMetadata stats = entry.getValue();
|
|
||||||
String policy = stats.getPolicy();
|
if (!timeline.containsInstant(cleanInstant)) {
|
||||||
String totalSuccessDeletedFiles = String.valueOf(stats.getSuccessDeleteFiles().size());
|
return "Clean " + commitTime + " not found in metadata " + timeline;
|
||||||
String totalFailedDeletedFiles = String.valueOf(stats.getFailedDeleteFiles().size());
|
|
||||||
rows.add(new String[] {path, policy, totalSuccessDeletedFiles, totalFailedDeletedFiles});
|
|
||||||
}
|
|
||||||
return HoodiePrintHelper.print(
|
|
||||||
new String[] {"Partition Path", "Cleaning policy", "Total Files Successfully Deleted",
|
|
||||||
"Total Failed Deletions"}, rows.toArray(new String[rows.size()][]));
|
|
||||||
}
|
}
|
||||||
|
HoodieCleanMetadata cleanMetadata =
|
||||||
|
AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(cleanInstant).get());
|
||||||
|
List<String[]> rows = new ArrayList<>();
|
||||||
|
for (Map.Entry<String, HoodieCleanPartitionMetadata> entry : cleanMetadata
|
||||||
|
.getPartitionMetadata().entrySet()) {
|
||||||
|
String path = entry.getKey();
|
||||||
|
HoodieCleanPartitionMetadata stats = entry.getValue();
|
||||||
|
String policy = stats.getPolicy();
|
||||||
|
String totalSuccessDeletedFiles = String.valueOf(stats.getSuccessDeleteFiles().size());
|
||||||
|
String totalFailedDeletedFiles = String.valueOf(stats.getFailedDeleteFiles().size());
|
||||||
|
rows.add(new String[]{path, policy, totalSuccessDeletedFiles, totalFailedDeletedFiles});
|
||||||
|
}
|
||||||
|
return HoodiePrintHelper.print(
|
||||||
|
new String[]{"Partition Path", "Cleaning policy", "Total Files Successfully Deleted",
|
||||||
|
"Total Failed Deletions"}, rows.toArray(new String[rows.size()][]));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -27,7 +27,12 @@ import com.uber.hoodie.common.table.HoodieTimeline;
|
|||||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||||
import com.uber.hoodie.common.util.NumericUtils;
|
import com.uber.hoodie.common.util.NumericUtils;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import org.apache.spark.launcher.SparkLauncher;
|
import org.apache.spark.launcher.SparkLauncher;
|
||||||
import org.springframework.shell.core.CommandMarker;
|
import org.springframework.shell.core.CommandMarker;
|
||||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||||
@@ -35,228 +40,236 @@ import org.springframework.shell.core.annotation.CliCommand;
|
|||||||
import org.springframework.shell.core.annotation.CliOption;
|
import org.springframework.shell.core.annotation.CliOption;
|
||||||
import org.springframework.stereotype.Component;
|
import org.springframework.stereotype.Component;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
@Component
|
@Component
|
||||||
public class CommitsCommand implements CommandMarker {
|
public class CommitsCommand implements CommandMarker {
|
||||||
@CliAvailabilityIndicator({"commits show"})
|
|
||||||
public boolean isShowAvailable() {
|
@CliAvailabilityIndicator({"commits show"})
|
||||||
return HoodieCLI.tableMetadata != null;
|
public boolean isShowAvailable() {
|
||||||
|
return HoodieCLI.tableMetadata != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliAvailabilityIndicator({"commits refresh"})
|
||||||
|
public boolean isRefreshAvailable() {
|
||||||
|
return HoodieCLI.tableMetadata != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliAvailabilityIndicator({"commit rollback"})
|
||||||
|
public boolean isRollbackAvailable() {
|
||||||
|
return HoodieCLI.tableMetadata != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliAvailabilityIndicator({"commit show"})
|
||||||
|
public boolean isCommitShowAvailable() {
|
||||||
|
return HoodieCLI.tableMetadata != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliCommand(value = "commits show", help = "Show the commits")
|
||||||
|
public String showCommits(
|
||||||
|
@CliOption(key = {
|
||||||
|
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10")
|
||||||
|
final Integer limit) throws IOException {
|
||||||
|
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||||
|
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline()
|
||||||
|
.filterCompletedInstants();
|
||||||
|
List<HoodieInstant> commits = timeline.getInstants().collect(Collectors.toList());
|
||||||
|
String[][] rows = new String[commits.size()][];
|
||||||
|
Collections.reverse(commits);
|
||||||
|
for (int i = 0; i < commits.size(); i++) {
|
||||||
|
HoodieInstant commit = commits.get(i);
|
||||||
|
HoodieCommitMetadata commitMetadata =
|
||||||
|
HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get());
|
||||||
|
rows[i] = new String[]{commit.getTimestamp(),
|
||||||
|
NumericUtils.humanReadableByteCount(commitMetadata.fetchTotalBytesWritten()),
|
||||||
|
String.valueOf(commitMetadata.fetchTotalFilesInsert()),
|
||||||
|
String.valueOf(commitMetadata.fetchTotalFilesUpdated()),
|
||||||
|
String.valueOf(commitMetadata.fetchTotalPartitionsWritten()),
|
||||||
|
String.valueOf(commitMetadata.fetchTotalRecordsWritten()),
|
||||||
|
String.valueOf(commitMetadata.fetchTotalUpdateRecordsWritten()),
|
||||||
|
String.valueOf(commitMetadata.fetchTotalWriteErrors())};
|
||||||
|
}
|
||||||
|
return HoodiePrintHelper.print(
|
||||||
|
new String[]{"CommitTime", "Total Written (B)", "Total Files Added",
|
||||||
|
"Total Files Updated", "Total Partitions Written", "Total Records Written",
|
||||||
|
"Total Update Records Written", "Total Errors"}, rows);
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliCommand(value = "commits refresh", help = "Refresh the commits")
|
||||||
|
public String refreshCommits() throws IOException {
|
||||||
|
HoodieTableMetaClient metadata =
|
||||||
|
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
|
||||||
|
HoodieCLI.setTableMetadata(metadata);
|
||||||
|
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliCommand(value = "commit rollback", help = "Rollback a commit")
|
||||||
|
public String rollbackCommit(
|
||||||
|
@CliOption(key = {"commit"}, help = "Commit to rollback")
|
||||||
|
final String commitTime,
|
||||||
|
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path")
|
||||||
|
final String sparkPropertiesPath) throws Exception {
|
||||||
|
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||||
|
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline()
|
||||||
|
.filterCompletedInstants();
|
||||||
|
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION,
|
||||||
|
commitTime);
|
||||||
|
|
||||||
|
if (!timeline.containsInstant(commitInstant)) {
|
||||||
|
return "Commit " + commitTime + " not found in Commits " + timeline;
|
||||||
}
|
}
|
||||||
|
|
||||||
@CliAvailabilityIndicator({"commits refresh"})
|
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||||
public boolean isRefreshAvailable() {
|
sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(),
|
||||||
return HoodieCLI.tableMetadata != null;
|
commitTime,
|
||||||
|
HoodieCLI.tableMetadata.getBasePath());
|
||||||
|
Process process = sparkLauncher.launch();
|
||||||
|
InputStreamConsumer.captureOutput(process);
|
||||||
|
int exitCode = process.waitFor();
|
||||||
|
// Refresh the current
|
||||||
|
refreshCommits();
|
||||||
|
if (exitCode != 0) {
|
||||||
|
return "Commit " + commitTime + " failed to roll back";
|
||||||
}
|
}
|
||||||
|
return "Commit " + commitTime + " rolled back";
|
||||||
|
}
|
||||||
|
|
||||||
@CliAvailabilityIndicator({"commit rollback"})
|
@CliCommand(value = "commit showpartitions", help = "Show partition level details of a commit")
|
||||||
public boolean isRollbackAvailable() {
|
public String showCommitPartitions(
|
||||||
return HoodieCLI.tableMetadata != null;
|
@CliOption(key = {"commit"}, help = "Commit to show")
|
||||||
|
final String commitTime) throws Exception {
|
||||||
|
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||||
|
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline()
|
||||||
|
.filterCompletedInstants();
|
||||||
|
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION,
|
||||||
|
commitTime);
|
||||||
|
|
||||||
|
if (!timeline.containsInstant(commitInstant)) {
|
||||||
|
return "Commit " + commitTime + " not found in Commits " + timeline;
|
||||||
}
|
}
|
||||||
|
HoodieCommitMetadata meta =
|
||||||
@CliAvailabilityIndicator({"commit show"})
|
HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get());
|
||||||
public boolean isCommitShowAvailable() {
|
List<String[]> rows = new ArrayList<String[]>();
|
||||||
return HoodieCLI.tableMetadata != null;
|
for (Map.Entry<String, List<HoodieWriteStat>> entry : meta.getPartitionToWriteStats()
|
||||||
}
|
.entrySet()) {
|
||||||
|
String path = entry.getKey();
|
||||||
@CliCommand(value = "commits show", help = "Show the commits")
|
List<HoodieWriteStat> stats = entry.getValue();
|
||||||
public String showCommits(
|
long totalFilesAdded = 0;
|
||||||
@CliOption(key = {
|
long totalFilesUpdated = 0;
|
||||||
"limit"}, mandatory = false, help = "Limit commits", unspecifiedDefaultValue = "10")
|
long totalRecordsUpdated = 0;
|
||||||
final Integer limit) throws IOException {
|
long totalRecordsInserted = 0;
|
||||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
long totalBytesWritten = 0;
|
||||||
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
long totalWriteErrors = 0;
|
||||||
List<HoodieInstant> commits = timeline.getInstants().collect(Collectors.toList());
|
for (HoodieWriteStat stat : stats) {
|
||||||
String[][] rows = new String[commits.size()][];
|
if (stat.getPrevCommit().equals(HoodieWriteStat.NULL_COMMIT)) {
|
||||||
Collections.reverse(commits);
|
totalFilesAdded += 1;
|
||||||
for (int i = 0; i < commits.size(); i++) {
|
totalRecordsInserted += stat.getNumWrites();
|
||||||
HoodieInstant commit = commits.get(i);
|
|
||||||
HoodieCommitMetadata commitMetadata =
|
|
||||||
HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get());
|
|
||||||
rows[i] = new String[] {commit.getTimestamp(),
|
|
||||||
NumericUtils.humanReadableByteCount(commitMetadata.fetchTotalBytesWritten()),
|
|
||||||
String.valueOf(commitMetadata.fetchTotalFilesInsert()),
|
|
||||||
String.valueOf(commitMetadata.fetchTotalFilesUpdated()),
|
|
||||||
String.valueOf(commitMetadata.fetchTotalPartitionsWritten()),
|
|
||||||
String.valueOf(commitMetadata.fetchTotalRecordsWritten()),
|
|
||||||
String.valueOf(commitMetadata.fetchTotalUpdateRecordsWritten()),
|
|
||||||
String.valueOf(commitMetadata.fetchTotalWriteErrors())};
|
|
||||||
}
|
|
||||||
return HoodiePrintHelper.print(
|
|
||||||
new String[] {"CommitTime", "Total Written (B)", "Total Files Added",
|
|
||||||
"Total Files Updated", "Total Partitions Written", "Total Records Written",
|
|
||||||
"Total Update Records Written", "Total Errors"}, rows);
|
|
||||||
}
|
|
||||||
|
|
||||||
@CliCommand(value = "commits refresh", help = "Refresh the commits")
|
|
||||||
public String refreshCommits() throws IOException {
|
|
||||||
HoodieTableMetaClient metadata =
|
|
||||||
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
|
|
||||||
HoodieCLI.setTableMetadata(metadata);
|
|
||||||
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
|
|
||||||
}
|
|
||||||
|
|
||||||
@CliCommand(value = "commit rollback", help = "Rollback a commit")
|
|
||||||
public String rollbackCommit(
|
|
||||||
@CliOption(key = {"commit"}, help = "Commit to rollback")
|
|
||||||
final String commitTime,
|
|
||||||
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path")
|
|
||||||
final String sparkPropertiesPath) throws Exception {
|
|
||||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
|
||||||
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
|
||||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
|
||||||
|
|
||||||
if (!timeline.containsInstant(commitInstant)) {
|
|
||||||
return "Commit " + commitTime + " not found in Commits " + timeline;
|
|
||||||
}
|
|
||||||
|
|
||||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
|
||||||
sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK.toString(),
|
|
||||||
commitTime,
|
|
||||||
HoodieCLI.tableMetadata.getBasePath());
|
|
||||||
Process process = sparkLauncher.launch();
|
|
||||||
InputStreamConsumer.captureOutput(process);
|
|
||||||
int exitCode = process.waitFor();
|
|
||||||
// Refresh the current
|
|
||||||
refreshCommits();
|
|
||||||
if (exitCode != 0) {
|
|
||||||
return "Commit " + commitTime + " failed to roll back";
|
|
||||||
}
|
|
||||||
return "Commit " + commitTime + " rolled back";
|
|
||||||
}
|
|
||||||
|
|
||||||
@CliCommand(value = "commit showpartitions", help = "Show partition level details of a commit")
|
|
||||||
public String showCommitPartitions(
|
|
||||||
@CliOption(key = {"commit"}, help = "Commit to show")
|
|
||||||
final String commitTime) throws Exception {
|
|
||||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
|
||||||
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
|
||||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
|
||||||
|
|
||||||
if (!timeline.containsInstant(commitInstant)) {
|
|
||||||
return "Commit " + commitTime + " not found in Commits " + timeline;
|
|
||||||
}
|
|
||||||
HoodieCommitMetadata meta =
|
|
||||||
HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get());
|
|
||||||
List<String[]> rows = new ArrayList<String[]>();
|
|
||||||
for (Map.Entry<String, List<HoodieWriteStat>> entry : meta.getPartitionToWriteStats()
|
|
||||||
.entrySet()) {
|
|
||||||
String path = entry.getKey();
|
|
||||||
List<HoodieWriteStat> stats = entry.getValue();
|
|
||||||
long totalFilesAdded = 0;
|
|
||||||
long totalFilesUpdated = 0;
|
|
||||||
long totalRecordsUpdated = 0;
|
|
||||||
long totalRecordsInserted = 0;
|
|
||||||
long totalBytesWritten = 0;
|
|
||||||
long totalWriteErrors = 0;
|
|
||||||
for (HoodieWriteStat stat : stats) {
|
|
||||||
if (stat.getPrevCommit().equals(HoodieWriteStat.NULL_COMMIT)) {
|
|
||||||
totalFilesAdded += 1;
|
|
||||||
totalRecordsInserted += stat.getNumWrites();
|
|
||||||
} else {
|
|
||||||
totalFilesUpdated += 1;
|
|
||||||
totalRecordsUpdated += stat.getNumUpdateWrites();
|
|
||||||
}
|
|
||||||
totalBytesWritten += stat.getTotalWriteBytes();
|
|
||||||
totalWriteErrors += stat.getTotalWriteErrors();
|
|
||||||
}
|
|
||||||
rows.add(new String[] {path, String.valueOf(totalFilesAdded),
|
|
||||||
String.valueOf(totalFilesUpdated), String.valueOf(totalRecordsInserted),
|
|
||||||
String.valueOf(totalRecordsUpdated),
|
|
||||||
NumericUtils.humanReadableByteCount(totalBytesWritten),
|
|
||||||
String.valueOf(totalWriteErrors)});
|
|
||||||
|
|
||||||
}
|
|
||||||
return HoodiePrintHelper.print(
|
|
||||||
new String[] {"Partition Path", "Total Files Added", "Total Files Updated",
|
|
||||||
"Total Records Inserted", "Total Records Updated", "Total Bytes Written",
|
|
||||||
"Total Errors"}, rows.toArray(new String[rows.size()][]));
|
|
||||||
}
|
|
||||||
|
|
||||||
@CliCommand(value = "commit showfiles", help = "Show file level details of a commit")
|
|
||||||
public String showCommitFiles(
|
|
||||||
@CliOption(key = {"commit"}, help = "Commit to show")
|
|
||||||
final String commitTime) throws Exception {
|
|
||||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
|
||||||
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
|
||||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
|
||||||
|
|
||||||
if (!timeline.containsInstant(commitInstant)) {
|
|
||||||
return "Commit " + commitTime + " not found in Commits " + timeline;
|
|
||||||
}
|
|
||||||
HoodieCommitMetadata meta =
|
|
||||||
HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get());
|
|
||||||
List<String[]> rows = new ArrayList<String[]>();
|
|
||||||
for (Map.Entry<String, List<HoodieWriteStat>> entry : meta.getPartitionToWriteStats()
|
|
||||||
.entrySet()) {
|
|
||||||
String path = entry.getKey();
|
|
||||||
List<HoodieWriteStat> stats = entry.getValue();
|
|
||||||
for (HoodieWriteStat stat : stats) {
|
|
||||||
rows.add(new String[] {path, stat.getFileId(), stat.getPrevCommit(),
|
|
||||||
String.valueOf(stat.getNumUpdateWrites()), String.valueOf(stat.getNumWrites()),
|
|
||||||
String.valueOf(stat.getTotalWriteBytes()),
|
|
||||||
String.valueOf(stat.getTotalWriteErrors())});
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return HoodiePrintHelper.print(
|
|
||||||
new String[] {"Partition Path", "File ID", "Previous Commit", "Total Records Updated",
|
|
||||||
"Total Records Written", "Total Bytes Written", "Total Errors"},
|
|
||||||
rows.toArray(new String[rows.size()][]));
|
|
||||||
}
|
|
||||||
|
|
||||||
@CliAvailabilityIndicator({"commits compare"})
|
|
||||||
public boolean isCompareCommitsAvailable() {
|
|
||||||
return HoodieCLI.tableMetadata != null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@CliCommand(value = "commits compare", help = "Compare commits with another Hoodie dataset")
|
|
||||||
public String compareCommits(
|
|
||||||
@CliOption(key = {"path"}, help = "Path of the dataset to compare to")
|
|
||||||
final String path) throws Exception {
|
|
||||||
HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.fs, path);
|
|
||||||
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();;
|
|
||||||
HoodieTableMetaClient source = HoodieCLI.tableMetadata;
|
|
||||||
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();;
|
|
||||||
String targetLatestCommit =
|
|
||||||
targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get().getTimestamp();
|
|
||||||
String sourceLatestCommit =
|
|
||||||
sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get().getTimestamp();
|
|
||||||
|
|
||||||
if (sourceLatestCommit != null &&
|
|
||||||
HoodieTimeline.compareTimestamps(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) {
|
|
||||||
// source is behind the target
|
|
||||||
List<String> commitsToCatchup =
|
|
||||||
targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE)
|
|
||||||
.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
|
||||||
return "Source " + source.getTableConfig().getTableName() + " is behind by "
|
|
||||||
+ commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup;
|
|
||||||
} else {
|
} else {
|
||||||
List<String> commitsToCatchup =
|
totalFilesUpdated += 1;
|
||||||
sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE)
|
totalRecordsUpdated += stat.getNumUpdateWrites();
|
||||||
.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
|
||||||
return "Source " + source.getTableConfig().getTableName() + " is ahead by "
|
|
||||||
+ commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup;
|
|
||||||
}
|
}
|
||||||
}
|
totalBytesWritten += stat.getTotalWriteBytes();
|
||||||
|
totalWriteErrors += stat.getTotalWriteErrors();
|
||||||
|
}
|
||||||
|
rows.add(new String[]{path, String.valueOf(totalFilesAdded),
|
||||||
|
String.valueOf(totalFilesUpdated), String.valueOf(totalRecordsInserted),
|
||||||
|
String.valueOf(totalRecordsUpdated),
|
||||||
|
NumericUtils.humanReadableByteCount(totalBytesWritten),
|
||||||
|
String.valueOf(totalWriteErrors)});
|
||||||
|
|
||||||
@CliAvailabilityIndicator({"commits sync"})
|
|
||||||
public boolean isSyncCommitsAvailable() {
|
|
||||||
return HoodieCLI.tableMetadata != null;
|
|
||||||
}
|
}
|
||||||
|
return HoodiePrintHelper.print(
|
||||||
|
new String[]{"Partition Path", "Total Files Added", "Total Files Updated",
|
||||||
|
"Total Records Inserted", "Total Records Updated", "Total Bytes Written",
|
||||||
|
"Total Errors"}, rows.toArray(new String[rows.size()][]));
|
||||||
|
}
|
||||||
|
|
||||||
@CliCommand(value = "commits sync", help = "Compare commits with another Hoodie dataset")
|
@CliCommand(value = "commit showfiles", help = "Show file level details of a commit")
|
||||||
public String syncCommits(
|
public String showCommitFiles(
|
||||||
@CliOption(key = {"path"}, help = "Path of the dataset to compare to")
|
@CliOption(key = {"commit"}, help = "Commit to show")
|
||||||
final String path) throws Exception {
|
final String commitTime) throws Exception {
|
||||||
HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.fs, path);
|
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||||
HoodieCLI.state = HoodieCLI.CLIState.SYNC;
|
HoodieTimeline timeline = activeTimeline.getCommitsAndCompactionsTimeline()
|
||||||
return "Load sync state between " + HoodieCLI.tableMetadata.getTableConfig().getTableName()
|
.filterCompletedInstants();
|
||||||
+ " and " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName();
|
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION,
|
||||||
|
commitTime);
|
||||||
|
|
||||||
|
if (!timeline.containsInstant(commitInstant)) {
|
||||||
|
return "Commit " + commitTime + " not found in Commits " + timeline;
|
||||||
}
|
}
|
||||||
|
HoodieCommitMetadata meta =
|
||||||
|
HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitInstant).get());
|
||||||
|
List<String[]> rows = new ArrayList<String[]>();
|
||||||
|
for (Map.Entry<String, List<HoodieWriteStat>> entry : meta.getPartitionToWriteStats()
|
||||||
|
.entrySet()) {
|
||||||
|
String path = entry.getKey();
|
||||||
|
List<HoodieWriteStat> stats = entry.getValue();
|
||||||
|
for (HoodieWriteStat stat : stats) {
|
||||||
|
rows.add(new String[]{path, stat.getFileId(), stat.getPrevCommit(),
|
||||||
|
String.valueOf(stat.getNumUpdateWrites()), String.valueOf(stat.getNumWrites()),
|
||||||
|
String.valueOf(stat.getTotalWriteBytes()),
|
||||||
|
String.valueOf(stat.getTotalWriteErrors())});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return HoodiePrintHelper.print(
|
||||||
|
new String[]{"Partition Path", "File ID", "Previous Commit", "Total Records Updated",
|
||||||
|
"Total Records Written", "Total Bytes Written", "Total Errors"},
|
||||||
|
rows.toArray(new String[rows.size()][]));
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliAvailabilityIndicator({"commits compare"})
|
||||||
|
public boolean isCompareCommitsAvailable() {
|
||||||
|
return HoodieCLI.tableMetadata != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliCommand(value = "commits compare", help = "Compare commits with another Hoodie dataset")
|
||||||
|
public String compareCommits(
|
||||||
|
@CliOption(key = {"path"}, help = "Path of the dataset to compare to")
|
||||||
|
final String path) throws Exception {
|
||||||
|
HoodieTableMetaClient target = new HoodieTableMetaClient(HoodieCLI.fs, path);
|
||||||
|
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsAndCompactionsTimeline()
|
||||||
|
.filterCompletedInstants();
|
||||||
|
;
|
||||||
|
HoodieTableMetaClient source = HoodieCLI.tableMetadata;
|
||||||
|
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsAndCompactionsTimeline()
|
||||||
|
.filterCompletedInstants();
|
||||||
|
;
|
||||||
|
String targetLatestCommit =
|
||||||
|
targetTimeline.getInstants().iterator().hasNext() ? "0"
|
||||||
|
: targetTimeline.lastInstant().get().getTimestamp();
|
||||||
|
String sourceLatestCommit =
|
||||||
|
sourceTimeline.getInstants().iterator().hasNext() ? "0"
|
||||||
|
: sourceTimeline.lastInstant().get().getTimestamp();
|
||||||
|
|
||||||
|
if (sourceLatestCommit != null &&
|
||||||
|
HoodieTimeline
|
||||||
|
.compareTimestamps(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) {
|
||||||
|
// source is behind the target
|
||||||
|
List<String> commitsToCatchup =
|
||||||
|
targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE)
|
||||||
|
.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||||
|
return "Source " + source.getTableConfig().getTableName() + " is behind by "
|
||||||
|
+ commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup;
|
||||||
|
} else {
|
||||||
|
List<String> commitsToCatchup =
|
||||||
|
sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE)
|
||||||
|
.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||||
|
return "Source " + source.getTableConfig().getTableName() + " is ahead by "
|
||||||
|
+ commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliAvailabilityIndicator({"commits sync"})
|
||||||
|
public boolean isSyncCommitsAvailable() {
|
||||||
|
return HoodieCLI.tableMetadata != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliCommand(value = "commits sync", help = "Compare commits with another Hoodie dataset")
|
||||||
|
public String syncCommits(
|
||||||
|
@CliOption(key = {"path"}, help = "Path of the dataset to compare to")
|
||||||
|
final String path) throws Exception {
|
||||||
|
HoodieCLI.syncTableMetadata = new HoodieTableMetaClient(HoodieCLI.fs, path);
|
||||||
|
HoodieCLI.state = HoodieCLI.CLIState.SYNC;
|
||||||
|
return "Load sync state between " + HoodieCLI.tableMetadata.getTableConfig().getTableName()
|
||||||
|
+ " and " + HoodieCLI.syncTableMetadata.getTableConfig().getTableName();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,24 +18,24 @@ package com.uber.hoodie.cli.commands;
|
|||||||
|
|
||||||
import com.uber.hoodie.cli.HoodieCLI;
|
import com.uber.hoodie.cli.HoodieCLI;
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
|
import java.io.IOException;
|
||||||
import org.springframework.shell.core.CommandMarker;
|
import org.springframework.shell.core.CommandMarker;
|
||||||
import org.springframework.shell.core.annotation.CliCommand;
|
import org.springframework.shell.core.annotation.CliCommand;
|
||||||
import org.springframework.shell.core.annotation.CliOption;
|
import org.springframework.shell.core.annotation.CliOption;
|
||||||
import org.springframework.stereotype.Component;
|
import org.springframework.stereotype.Component;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
@Component
|
@Component
|
||||||
public class DatasetsCommand implements CommandMarker {
|
public class DatasetsCommand implements CommandMarker {
|
||||||
@CliCommand(value = "connect", help = "Connect to a hoodie dataset")
|
|
||||||
public String connect(
|
@CliCommand(value = "connect", help = "Connect to a hoodie dataset")
|
||||||
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset")
|
public String connect(
|
||||||
final String path) throws IOException {
|
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset")
|
||||||
boolean initialized = HoodieCLI.initConf();
|
final String path) throws IOException {
|
||||||
HoodieCLI.initFS(initialized);
|
boolean initialized = HoodieCLI.initConf();
|
||||||
HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.fs, path));
|
HoodieCLI.initFS(initialized);
|
||||||
HoodieCLI.state = HoodieCLI.CLIState.DATASET;
|
HoodieCLI.setTableMetadata(new HoodieTableMetaClient(HoodieCLI.fs, path));
|
||||||
return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName()
|
HoodieCLI.state = HoodieCLI.CLIState.DATASET;
|
||||||
+ " loaded";
|
return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName()
|
||||||
}
|
+ " loaded";
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -33,58 +33,59 @@ import org.springframework.stereotype.Component;
|
|||||||
@Component
|
@Component
|
||||||
public class HDFSParquetImportCommand implements CommandMarker {
|
public class HDFSParquetImportCommand implements CommandMarker {
|
||||||
|
|
||||||
private static Logger log = LogManager.getLogger(HDFSParquetImportCommand.class);
|
private static Logger log = LogManager.getLogger(HDFSParquetImportCommand.class);
|
||||||
|
|
||||||
@CliCommand(value = "hdfsparquetimport", help = "Imports hdfs dataset to a hoodie dataset")
|
@CliCommand(value = "hdfsparquetimport", help = "Imports hdfs dataset to a hoodie dataset")
|
||||||
public String convert(
|
public String convert(
|
||||||
@CliOption(key = "srcPath", mandatory = true, help = "Base path for the input dataset")
|
@CliOption(key = "srcPath", mandatory = true, help = "Base path for the input dataset")
|
||||||
final String srcPath,
|
final String srcPath,
|
||||||
@CliOption(key = "srcType", mandatory = true, help = "Source type for the input dataset")
|
@CliOption(key = "srcType", mandatory = true, help = "Source type for the input dataset")
|
||||||
final String srcType,
|
final String srcType,
|
||||||
@CliOption(key = "targetPath", mandatory = true, help = "Base path for the target hoodie dataset")
|
@CliOption(key = "targetPath", mandatory = true, help = "Base path for the target hoodie dataset")
|
||||||
final String targetPath,
|
final String targetPath,
|
||||||
@CliOption(key = "tableName", mandatory = true, help = "Table name")
|
@CliOption(key = "tableName", mandatory = true, help = "Table name")
|
||||||
final String tableName,
|
final String tableName,
|
||||||
@CliOption(key = "tableType", mandatory = true, help = "Table type")
|
@CliOption(key = "tableType", mandatory = true, help = "Table type")
|
||||||
final String tableType,
|
final String tableType,
|
||||||
@CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name")
|
@CliOption(key = "rowKeyField", mandatory = true, help = "Row key field name")
|
||||||
final String rowKeyField,
|
final String rowKeyField,
|
||||||
@CliOption(key = "partitionPathField", mandatory = true, help = "Partition path field name")
|
@CliOption(key = "partitionPathField", mandatory = true, help = "Partition path field name")
|
||||||
final String partitionPathField,
|
final String partitionPathField,
|
||||||
@CliOption(key = {"parallelism"}, mandatory = true, help = "Parallelism for hoodie insert")
|
@CliOption(key = {"parallelism"}, mandatory = true, help = "Parallelism for hoodie insert")
|
||||||
final String parallelism,
|
final String parallelism,
|
||||||
@CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file")
|
@CliOption(key = "schemaFilePath", mandatory = true, help = "Path for Avro schema file")
|
||||||
final String schemaFilePath,
|
final String schemaFilePath,
|
||||||
@CliOption(key = "format", mandatory = true, help = "Format for the input data")
|
@CliOption(key = "format", mandatory = true, help = "Format for the input data")
|
||||||
final String format,
|
final String format,
|
||||||
@CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory")
|
@CliOption(key = "sparkMemory", mandatory = true, help = "Spark executor memory")
|
||||||
final String sparkMemory,
|
final String sparkMemory,
|
||||||
@CliOption(key = "retry", mandatory = true, help = "Number of retries")
|
@CliOption(key = "retry", mandatory = true, help = "Number of retries")
|
||||||
final String retry)
|
final String retry)
|
||||||
throws Exception {
|
throws Exception {
|
||||||
|
|
||||||
validate(format, srcType);
|
validate(format, srcType);
|
||||||
|
|
||||||
boolean initialized = HoodieCLI.initConf();
|
boolean initialized = HoodieCLI.initConf();
|
||||||
HoodieCLI.initFS(initialized);
|
HoodieCLI.initFS(initialized);
|
||||||
String sparkPropertiesPath = Utils
|
String sparkPropertiesPath = Utils
|
||||||
.getDefaultPropertiesFile(scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
.getDefaultPropertiesFile(
|
||||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
scala.collection.JavaConversions.propertiesAsScalaMap(System.getProperties()));
|
||||||
|
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||||
|
|
||||||
sparkLauncher.addAppArgs(SparkCommand.IMPORT.toString(), srcPath, targetPath, tableName,
|
sparkLauncher.addAppArgs(SparkCommand.IMPORT.toString(), srcPath, targetPath, tableName,
|
||||||
tableType, rowKeyField, partitionPathField, parallelism, schemaFilePath, sparkMemory,
|
tableType, rowKeyField, partitionPathField, parallelism, schemaFilePath, sparkMemory,
|
||||||
retry);
|
retry);
|
||||||
Process process = sparkLauncher.launch();
|
Process process = sparkLauncher.launch();
|
||||||
InputStreamConsumer.captureOutput(process);
|
InputStreamConsumer.captureOutput(process);
|
||||||
int exitCode = process.waitFor();
|
int exitCode = process.waitFor();
|
||||||
if (exitCode != 0) {
|
if (exitCode != 0) {
|
||||||
return "Failed to import dataset to hoodie format";
|
return "Failed to import dataset to hoodie format";
|
||||||
}
|
|
||||||
return "Dataset imported to hoodie format";
|
|
||||||
}
|
}
|
||||||
|
return "Dataset imported to hoodie format";
|
||||||
|
}
|
||||||
|
|
||||||
private void validate(String format, String srcType) {
|
private void validate(String format, String srcType) {
|
||||||
(new HDFSParquetImporter.FormatValidator()).validate("format", format);
|
(new HDFSParquetImporter.FormatValidator()).validate("format", format);
|
||||||
(new HDFSParquetImporter.SourceTypeValidator()).validate("srcType", srcType);
|
(new HDFSParquetImporter.SourceTypeValidator()).validate("srcType", srcType);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,105 +16,109 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.cli.commands;
|
package com.uber.hoodie.cli.commands;
|
||||||
|
|
||||||
|
import com.uber.hoodie.cli.HoodieCLI;
|
||||||
import com.uber.hoodie.cli.utils.CommitUtil;
|
import com.uber.hoodie.cli.utils.CommitUtil;
|
||||||
import com.uber.hoodie.cli.utils.HiveUtil;
|
import com.uber.hoodie.cli.utils.HiveUtil;
|
||||||
import com.uber.hoodie.cli.HoodieCLI;
|
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import org.springframework.shell.core.CommandMarker;
|
import org.springframework.shell.core.CommandMarker;
|
||||||
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
|
||||||
import org.springframework.shell.core.annotation.CliCommand;
|
import org.springframework.shell.core.annotation.CliCommand;
|
||||||
import org.springframework.shell.core.annotation.CliOption;
|
import org.springframework.shell.core.annotation.CliOption;
|
||||||
import org.springframework.stereotype.Component;
|
import org.springframework.stereotype.Component;
|
||||||
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
@Component
|
@Component
|
||||||
public class HoodieSyncCommand implements CommandMarker {
|
public class HoodieSyncCommand implements CommandMarker {
|
||||||
@CliAvailabilityIndicator({"sync validate"})
|
|
||||||
public boolean isSyncVerificationAvailable() {
|
@CliAvailabilityIndicator({"sync validate"})
|
||||||
return HoodieCLI.tableMetadata != null && HoodieCLI.syncTableMetadata != null;
|
public boolean isSyncVerificationAvailable() {
|
||||||
|
return HoodieCLI.tableMetadata != null && HoodieCLI.syncTableMetadata != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliCommand(value = "sync validate", help = "Validate the sync by counting the number of records")
|
||||||
|
public String validateSync(
|
||||||
|
@CliOption(key = {"mode"}, unspecifiedDefaultValue = "complete", help = "Check mode")
|
||||||
|
final String mode,
|
||||||
|
@CliOption(key = {
|
||||||
|
"sourceDb"}, unspecifiedDefaultValue = "rawdata", help = "source database")
|
||||||
|
final String srcDb,
|
||||||
|
@CliOption(key = {
|
||||||
|
"targetDb"}, unspecifiedDefaultValue = "dwh_hoodie", help = "target database")
|
||||||
|
final String tgtDb,
|
||||||
|
@CliOption(key = {
|
||||||
|
"partitionCount"}, unspecifiedDefaultValue = "5", help = "total number of recent partitions to validate")
|
||||||
|
final int partitionCount,
|
||||||
|
@CliOption(key = {
|
||||||
|
"hiveServerUrl"}, mandatory = true, help = "hiveServerURL to connect to")
|
||||||
|
final String hiveServerUrl,
|
||||||
|
@CliOption(key = {
|
||||||
|
"hiveUser"}, mandatory = false, unspecifiedDefaultValue = "", help = "hive username to connect to")
|
||||||
|
final String hiveUser,
|
||||||
|
@CliOption(key = {
|
||||||
|
"hivePass"}, mandatory = true, unspecifiedDefaultValue = "", help = "hive password to connect to")
|
||||||
|
final String hivePass) throws Exception {
|
||||||
|
HoodieTableMetaClient target = HoodieCLI.syncTableMetadata;
|
||||||
|
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsAndCompactionsTimeline();
|
||||||
|
HoodieTableMetaClient source = HoodieCLI.tableMetadata;
|
||||||
|
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsAndCompactionsTimeline();
|
||||||
|
long sourceCount = 0;
|
||||||
|
long targetCount = 0;
|
||||||
|
if ("complete".equals(mode)) {
|
||||||
|
sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, hiveUser, hivePass);
|
||||||
|
targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, hiveUser, hivePass);
|
||||||
|
} else if ("latestPartitions".equals(mode)) {
|
||||||
|
sourceCount = HiveUtil
|
||||||
|
.countRecords(hiveServerUrl, source, srcDb, partitionCount, hiveUser, hivePass);
|
||||||
|
targetCount = HiveUtil
|
||||||
|
.countRecords(hiveServerUrl, target, tgtDb, partitionCount, hiveUser, hivePass);
|
||||||
}
|
}
|
||||||
|
|
||||||
@CliCommand(value = "sync validate", help = "Validate the sync by counting the number of records")
|
String targetLatestCommit =
|
||||||
public String validateSync(
|
targetTimeline.getInstants().iterator().hasNext() ? "0"
|
||||||
@CliOption(key = {"mode"}, unspecifiedDefaultValue = "complete", help = "Check mode")
|
: targetTimeline.lastInstant().get().getTimestamp();
|
||||||
final String mode,
|
String sourceLatestCommit =
|
||||||
@CliOption(key = {
|
sourceTimeline.getInstants().iterator().hasNext() ? "0"
|
||||||
"sourceDb"}, unspecifiedDefaultValue = "rawdata", help = "source database")
|
: sourceTimeline.lastInstant().get().getTimestamp();
|
||||||
final String srcDb,
|
|
||||||
@CliOption(key = {
|
|
||||||
"targetDb"}, unspecifiedDefaultValue = "dwh_hoodie", help = "target database")
|
|
||||||
final String tgtDb,
|
|
||||||
@CliOption(key = {
|
|
||||||
"partitionCount"}, unspecifiedDefaultValue = "5", help = "total number of recent partitions to validate")
|
|
||||||
final int partitionCount,
|
|
||||||
@CliOption(key = {
|
|
||||||
"hiveServerUrl"}, mandatory = true, help = "hiveServerURL to connect to")
|
|
||||||
final String hiveServerUrl,
|
|
||||||
@CliOption(key = {
|
|
||||||
"hiveUser"}, mandatory = false, unspecifiedDefaultValue = "", help = "hive username to connect to")
|
|
||||||
final String hiveUser,
|
|
||||||
@CliOption(key = {
|
|
||||||
"hivePass"}, mandatory = true, unspecifiedDefaultValue = "", help = "hive password to connect to")
|
|
||||||
final String hivePass) throws Exception {
|
|
||||||
HoodieTableMetaClient target = HoodieCLI.syncTableMetadata;
|
|
||||||
HoodieTimeline targetTimeline = target.getActiveTimeline().getCommitsAndCompactionsTimeline();
|
|
||||||
HoodieTableMetaClient source = HoodieCLI.tableMetadata;
|
|
||||||
HoodieTimeline sourceTimeline = source.getActiveTimeline().getCommitsAndCompactionsTimeline();
|
|
||||||
long sourceCount = 0;
|
|
||||||
long targetCount = 0;
|
|
||||||
if ("complete".equals(mode)) {
|
|
||||||
sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, hiveUser, hivePass);
|
|
||||||
targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, hiveUser, hivePass);
|
|
||||||
} else if ("latestPartitions".equals(mode)) {
|
|
||||||
sourceCount = HiveUtil.countRecords(hiveServerUrl, source, srcDb, partitionCount, hiveUser, hivePass);
|
|
||||||
targetCount = HiveUtil.countRecords(hiveServerUrl, target, tgtDb, partitionCount, hiveUser, hivePass);
|
|
||||||
}
|
|
||||||
|
|
||||||
String targetLatestCommit =
|
if (sourceLatestCommit != null && HoodieTimeline
|
||||||
targetTimeline.getInstants().iterator().hasNext() ? "0" : targetTimeline.lastInstant().get().getTimestamp();
|
.compareTimestamps(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) {
|
||||||
String sourceLatestCommit =
|
// source is behind the target
|
||||||
sourceTimeline.getInstants().iterator().hasNext() ? "0" : sourceTimeline.lastInstant().get().getTimestamp();
|
List<HoodieInstant> commitsToCatchup =
|
||||||
|
targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE).getInstants()
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
if (commitsToCatchup.isEmpty()) {
|
||||||
|
return "Count difference now is (count(" + target.getTableConfig().getTableName()
|
||||||
|
+ ") - count(" + source.getTableConfig().getTableName() + ") == " + (targetCount
|
||||||
|
- sourceCount);
|
||||||
|
} else {
|
||||||
|
long newInserts = CommitUtil.countNewRecords(target,
|
||||||
|
commitsToCatchup.stream().map(HoodieInstant::getTimestamp)
|
||||||
|
.collect(Collectors.toList()));
|
||||||
|
return "Count difference now is (count(" + target.getTableConfig().getTableName()
|
||||||
|
+ ") - count(" + source.getTableConfig().getTableName() + ") == " + (targetCount
|
||||||
|
- sourceCount) + ". Catch up count is " + newInserts;
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
List<HoodieInstant> commitsToCatchup =
|
||||||
|
sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE).getInstants()
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
if (commitsToCatchup.isEmpty()) {
|
||||||
|
return "Count difference now is (count(" + source.getTableConfig().getTableName()
|
||||||
|
+ ") - count(" + target.getTableConfig().getTableName() + ") == " + (sourceCount
|
||||||
|
- targetCount);
|
||||||
|
} else {
|
||||||
|
long newInserts = CommitUtil.countNewRecords(source,
|
||||||
|
commitsToCatchup.stream().map(HoodieInstant::getTimestamp)
|
||||||
|
.collect(Collectors.toList()));
|
||||||
|
return "Count difference now is (count(" + source.getTableConfig().getTableName()
|
||||||
|
+ ") - count(" + target.getTableConfig().getTableName() + ") == " + (sourceCount
|
||||||
|
- targetCount) + ". Catch up count is " + newInserts;
|
||||||
|
}
|
||||||
|
|
||||||
if (sourceLatestCommit != null && HoodieTimeline
|
|
||||||
.compareTimestamps(targetLatestCommit, sourceLatestCommit, HoodieTimeline.GREATER)) {
|
|
||||||
// source is behind the target
|
|
||||||
List<HoodieInstant> commitsToCatchup =
|
|
||||||
targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE).getInstants()
|
|
||||||
.collect(Collectors.toList());
|
|
||||||
if (commitsToCatchup.isEmpty()) {
|
|
||||||
return "Count difference now is (count(" + target.getTableConfig().getTableName()
|
|
||||||
+ ") - count(" + source.getTableConfig().getTableName() + ") == " + (targetCount
|
|
||||||
- sourceCount);
|
|
||||||
} else {
|
|
||||||
long newInserts = CommitUtil.countNewRecords(target,
|
|
||||||
commitsToCatchup.stream().map(HoodieInstant::getTimestamp)
|
|
||||||
.collect(Collectors.toList()));
|
|
||||||
return "Count difference now is (count(" + target.getTableConfig().getTableName()
|
|
||||||
+ ") - count(" + source.getTableConfig().getTableName() + ") == " + (targetCount
|
|
||||||
- sourceCount) + ". Catch up count is " + newInserts;
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
List<HoodieInstant> commitsToCatchup =
|
|
||||||
sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE).getInstants()
|
|
||||||
.collect(Collectors.toList());
|
|
||||||
if (commitsToCatchup.isEmpty()) {
|
|
||||||
return "Count difference now is (count(" + source.getTableConfig().getTableName()
|
|
||||||
+ ") - count(" + target.getTableConfig().getTableName() + ") == " + (sourceCount
|
|
||||||
- targetCount);
|
|
||||||
} else {
|
|
||||||
long newInserts = CommitUtil.countNewRecords(source,
|
|
||||||
commitsToCatchup.stream().map(HoodieInstant::getTimestamp)
|
|
||||||
.collect(Collectors.toList()));
|
|
||||||
return "Count difference now is (count(" + source.getTableConfig().getTableName()
|
|
||||||
+ ") - count(" + target.getTableConfig().getTableName() + ") == " + (sourceCount
|
|
||||||
- targetCount) + ". Catch up count is " + newInserts;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -22,7 +22,8 @@ import com.uber.hoodie.cli.utils.InputStreamConsumer;
|
|||||||
import com.uber.hoodie.cli.utils.SparkUtil;
|
import com.uber.hoodie.cli.utils.SparkUtil;
|
||||||
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.spark.launcher.SparkLauncher;
|
import org.apache.spark.launcher.SparkLauncher;
|
||||||
import org.springframework.shell.core.CommandMarker;
|
import org.springframework.shell.core.CommandMarker;
|
||||||
@@ -31,80 +32,80 @@ import org.springframework.shell.core.annotation.CliCommand;
|
|||||||
import org.springframework.shell.core.annotation.CliOption;
|
import org.springframework.shell.core.annotation.CliOption;
|
||||||
import org.springframework.stereotype.Component;
|
import org.springframework.stereotype.Component;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
@Component
|
@Component
|
||||||
public class RepairsCommand implements CommandMarker {
|
public class RepairsCommand implements CommandMarker {
|
||||||
|
|
||||||
@CliAvailabilityIndicator({"repair deduplicate"})
|
@CliAvailabilityIndicator({"repair deduplicate"})
|
||||||
public boolean isRepairDeduplicateAvailable() {
|
public boolean isRepairDeduplicateAvailable() {
|
||||||
return HoodieCLI.tableMetadata != null;
|
return HoodieCLI.tableMetadata != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliAvailabilityIndicator({"repair addpartitionmeta"})
|
||||||
|
public boolean isRepairAddPartitionMetaAvailable() {
|
||||||
|
return HoodieCLI.tableMetadata != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliCommand(value = "repair deduplicate", help = "De-duplicate a partition path contains duplicates & produce repaired files to replace with")
|
||||||
|
public String deduplicate(
|
||||||
|
@CliOption(key = {
|
||||||
|
"duplicatedPartitionPath"}, help = "Partition Path containing the duplicates", mandatory = true)
|
||||||
|
final String duplicatedPartitionPath,
|
||||||
|
@CliOption(key = {
|
||||||
|
"repairedOutputPath"}, help = "Location to place the repaired files", mandatory = true)
|
||||||
|
final String repairedOutputPath,
|
||||||
|
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path", mandatory = true)
|
||||||
|
final String sparkPropertiesPath) throws Exception {
|
||||||
|
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||||
|
sparkLauncher
|
||||||
|
.addAppArgs(SparkMain.SparkCommand.DEDUPLICATE.toString(), duplicatedPartitionPath,
|
||||||
|
repairedOutputPath, HoodieCLI.tableMetadata.getBasePath());
|
||||||
|
Process process = sparkLauncher.launch();
|
||||||
|
InputStreamConsumer.captureOutput(process);
|
||||||
|
int exitCode = process.waitFor();
|
||||||
|
|
||||||
|
if (exitCode != 0) {
|
||||||
|
return "Deduplicated files placed in: " + repairedOutputPath;
|
||||||
}
|
}
|
||||||
|
return "Deduplication failed ";
|
||||||
|
}
|
||||||
|
|
||||||
@CliAvailabilityIndicator({"repair addpartitionmeta"})
|
|
||||||
public boolean isRepairAddPartitionMetaAvailable() {
|
|
||||||
return HoodieCLI.tableMetadata != null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@CliCommand(value = "repair deduplicate", help = "De-duplicate a partition path contains duplicates & produce repaired files to replace with")
|
@CliCommand(value = "repair addpartitionmeta", help = "Add partition metadata to a dataset, if not present")
|
||||||
public String deduplicate(
|
public String addPartitionMeta(
|
||||||
@CliOption(key = {
|
@CliOption(key = {"dryrun"},
|
||||||
"duplicatedPartitionPath"}, help = "Partition Path containing the duplicates", mandatory = true)
|
help = "Should we actually add or just print what would be done",
|
||||||
final String duplicatedPartitionPath,
|
unspecifiedDefaultValue = "true")
|
||||||
@CliOption(key = {"repairedOutputPath"}, help = "Location to place the repaired files", mandatory = true)
|
final boolean dryRun) throws IOException {
|
||||||
final String repairedOutputPath,
|
|
||||||
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path", mandatory = true)
|
|
||||||
final String sparkPropertiesPath) throws Exception {
|
|
||||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
|
||||||
sparkLauncher
|
|
||||||
.addAppArgs(SparkMain.SparkCommand.DEDUPLICATE.toString(), duplicatedPartitionPath,
|
|
||||||
repairedOutputPath, HoodieCLI.tableMetadata.getBasePath());
|
|
||||||
Process process = sparkLauncher.launch();
|
|
||||||
InputStreamConsumer.captureOutput(process);
|
|
||||||
int exitCode = process.waitFor();
|
|
||||||
|
|
||||||
if (exitCode != 0) {
|
String latestCommit = HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline()
|
||||||
return "Deduplicated files placed in: " + repairedOutputPath;
|
.lastInstant().get().getTimestamp();
|
||||||
|
List<String> partitionPaths = FSUtils.getAllFoldersThreeLevelsDown(HoodieCLI.fs,
|
||||||
|
HoodieCLI.tableMetadata.getBasePath());
|
||||||
|
Path basePath = new Path(HoodieCLI.tableMetadata.getBasePath());
|
||||||
|
String[][] rows = new String[partitionPaths.size() + 1][];
|
||||||
|
|
||||||
|
int ind = 0;
|
||||||
|
for (String partition : partitionPaths) {
|
||||||
|
Path partitionPath = new Path(basePath, partition);
|
||||||
|
String[] row = new String[3];
|
||||||
|
row[0] = partition;
|
||||||
|
row[1] = "Yes";
|
||||||
|
row[2] = "None";
|
||||||
|
if (!HoodiePartitionMetadata.hasPartitionMetadata(HoodieCLI.fs, partitionPath)) {
|
||||||
|
row[1] = "No";
|
||||||
|
if (!dryRun) {
|
||||||
|
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(
|
||||||
|
HoodieCLI.fs,
|
||||||
|
latestCommit,
|
||||||
|
basePath,
|
||||||
|
partitionPath);
|
||||||
|
partitionMetadata.trySave(0);
|
||||||
}
|
}
|
||||||
return "Deduplication failed ";
|
}
|
||||||
|
rows[ind++] = row;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
return HoodiePrintHelper.print(
|
||||||
|
new String[]{"Partition Path", "Metadata Present?", "Action"}, rows);
|
||||||
@CliCommand(value = "repair addpartitionmeta", help = "Add partition metadata to a dataset, if not present")
|
}
|
||||||
public String addPartitionMeta(
|
|
||||||
@CliOption(key = {"dryrun"},
|
|
||||||
help = "Should we actually add or just print what would be done",
|
|
||||||
unspecifiedDefaultValue = "true")
|
|
||||||
final boolean dryRun) throws IOException {
|
|
||||||
|
|
||||||
String latestCommit = HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline().lastInstant().get().getTimestamp();
|
|
||||||
List<String> partitionPaths = FSUtils.getAllFoldersThreeLevelsDown(HoodieCLI.fs,
|
|
||||||
HoodieCLI.tableMetadata.getBasePath());
|
|
||||||
Path basePath = new Path(HoodieCLI.tableMetadata.getBasePath());
|
|
||||||
String[][] rows = new String[partitionPaths.size() + 1][];
|
|
||||||
|
|
||||||
int ind = 0;
|
|
||||||
for (String partition: partitionPaths) {
|
|
||||||
Path partitionPath = new Path(basePath, partition);
|
|
||||||
String[] row = new String[3];
|
|
||||||
row[0] = partition; row[1] = "Yes"; row[2] = "None";
|
|
||||||
if (!HoodiePartitionMetadata.hasPartitionMetadata(HoodieCLI.fs, partitionPath)) {
|
|
||||||
row[1] = "No";
|
|
||||||
if (!dryRun) {
|
|
||||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(
|
|
||||||
HoodieCLI.fs,
|
|
||||||
latestCommit,
|
|
||||||
basePath,
|
|
||||||
partitionPath);
|
|
||||||
partitionMetadata.trySave(0);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
rows[ind++] = row;
|
|
||||||
}
|
|
||||||
|
|
||||||
return HoodiePrintHelper.print(
|
|
||||||
new String[] {"Partition Path", "Metadata Present?", "Action"}, rows);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -27,6 +27,10 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
|||||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.index.HoodieIndex;
|
import com.uber.hoodie.index.HoodieIndex;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.apache.spark.launcher.SparkLauncher;
|
import org.apache.spark.launcher.SparkLauncher;
|
||||||
import org.springframework.shell.core.CommandMarker;
|
import org.springframework.shell.core.CommandMarker;
|
||||||
@@ -35,122 +39,118 @@ import org.springframework.shell.core.annotation.CliCommand;
|
|||||||
import org.springframework.shell.core.annotation.CliOption;
|
import org.springframework.shell.core.annotation.CliOption;
|
||||||
import org.springframework.stereotype.Component;
|
import org.springframework.stereotype.Component;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
@Component
|
@Component
|
||||||
public class SavepointsCommand implements CommandMarker {
|
public class SavepointsCommand implements CommandMarker {
|
||||||
@CliAvailabilityIndicator({"savepoints show"})
|
|
||||||
public boolean isShowAvailable() {
|
@CliAvailabilityIndicator({"savepoints show"})
|
||||||
return HoodieCLI.tableMetadata != null;
|
public boolean isShowAvailable() {
|
||||||
|
return HoodieCLI.tableMetadata != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliAvailabilityIndicator({"savepoints refresh"})
|
||||||
|
public boolean isRefreshAvailable() {
|
||||||
|
return HoodieCLI.tableMetadata != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@CliAvailabilityIndicator({"savepoint create"})
|
||||||
|
public boolean isCreateSavepointAvailable() {
|
||||||
|
return HoodieCLI.tableMetadata != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliAvailabilityIndicator({"savepoint rollback"})
|
||||||
|
public boolean isRollbackToSavepointAvailable() {
|
||||||
|
return HoodieCLI.tableMetadata != null && !HoodieCLI.tableMetadata.getActiveTimeline()
|
||||||
|
.getSavePointTimeline().filterCompletedInstants().empty();
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliCommand(value = "savepoints show", help = "Show the savepoints")
|
||||||
|
public String showSavepoints() throws IOException {
|
||||||
|
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||||
|
HoodieTimeline timeline = activeTimeline.getSavePointTimeline().filterCompletedInstants();
|
||||||
|
List<HoodieInstant> commits = timeline.getInstants().collect(Collectors.toList());
|
||||||
|
String[][] rows = new String[commits.size()][];
|
||||||
|
Collections.reverse(commits);
|
||||||
|
for (int i = 0; i < commits.size(); i++) {
|
||||||
|
HoodieInstant commit = commits.get(i);
|
||||||
|
rows[i] = new String[]{commit.getTimestamp()};
|
||||||
|
}
|
||||||
|
return HoodiePrintHelper.print(new String[]{"SavepointTime"}, rows);
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliCommand(value = "savepoint create", help = "Savepoint a commit")
|
||||||
|
public String savepoint(
|
||||||
|
@CliOption(key = {"commit"}, help = "Commit to savepoint")
|
||||||
|
final String commitTime,
|
||||||
|
@CliOption(key = {"user"}, help = "User who is creating the savepoint")
|
||||||
|
final String user,
|
||||||
|
@CliOption(key = {"comments"}, help = "Comments for creating the savepoint")
|
||||||
|
final String comments) throws Exception {
|
||||||
|
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||||
|
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||||
|
HoodieInstant
|
||||||
|
commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||||
|
|
||||||
|
if (!timeline.containsInstant(commitInstant)) {
|
||||||
|
return "Commit " + commitTime + " not found in Commits " + timeline;
|
||||||
}
|
}
|
||||||
|
|
||||||
@CliAvailabilityIndicator({"savepoints refresh"})
|
HoodieWriteClient client = createHoodieClient(null, HoodieCLI.tableMetadata.getBasePath());
|
||||||
public boolean isRefreshAvailable() {
|
if (client.savepoint(commitTime, user, comments)) {
|
||||||
return HoodieCLI.tableMetadata != null;
|
// Refresh the current
|
||||||
|
refreshMetaClient();
|
||||||
|
return String.format("The commit \"%s\" has been savepointed.", commitTime);
|
||||||
|
}
|
||||||
|
return String.format("Failed: Could not savepoint commit \"%s\".", commitTime);
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliCommand(value = "savepoint rollback", help = "Savepoint a commit")
|
||||||
|
public String rollbackToSavepoint(
|
||||||
|
@CliOption(key = {"savepoint"}, help = "Savepoint to rollback")
|
||||||
|
final String commitTime,
|
||||||
|
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path")
|
||||||
|
final String sparkPropertiesPath) throws Exception {
|
||||||
|
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||||
|
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||||
|
HoodieInstant
|
||||||
|
commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||||
|
|
||||||
|
if (!timeline.containsInstant(commitInstant)) {
|
||||||
|
return "Commit " + commitTime + " not found in Commits " + timeline;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
||||||
@CliAvailabilityIndicator({"savepoint create"})
|
sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK_TO_SAVEPOINT.toString(),
|
||||||
public boolean isCreateSavepointAvailable() {
|
commitTime,
|
||||||
return HoodieCLI.tableMetadata != null;
|
HoodieCLI.tableMetadata.getBasePath());
|
||||||
}
|
Process process = sparkLauncher.launch();
|
||||||
|
InputStreamConsumer.captureOutput(process);
|
||||||
@CliAvailabilityIndicator({"savepoint rollback"})
|
int exitCode = process.waitFor();
|
||||||
public boolean isRollbackToSavepointAvailable() {
|
// Refresh the current
|
||||||
return HoodieCLI.tableMetadata != null && !HoodieCLI.tableMetadata.getActiveTimeline().getSavePointTimeline().filterCompletedInstants().empty();
|
refreshMetaClient();
|
||||||
}
|
if (exitCode != 0) {
|
||||||
|
return "Savepoint " + commitTime + " failed to roll back";
|
||||||
@CliCommand(value = "savepoints show", help = "Show the savepoints")
|
|
||||||
public String showSavepoints() throws IOException {
|
|
||||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
|
||||||
HoodieTimeline timeline = activeTimeline.getSavePointTimeline().filterCompletedInstants();
|
|
||||||
List<HoodieInstant> commits = timeline.getInstants().collect(Collectors.toList());
|
|
||||||
String[][] rows = new String[commits.size()][];
|
|
||||||
Collections.reverse(commits);
|
|
||||||
for (int i = 0; i < commits.size(); i++) {
|
|
||||||
HoodieInstant commit = commits.get(i);
|
|
||||||
rows[i] = new String[] {commit.getTimestamp()};
|
|
||||||
}
|
|
||||||
return HoodiePrintHelper.print(new String[] {"SavepointTime"}, rows);
|
|
||||||
}
|
|
||||||
|
|
||||||
@CliCommand(value = "savepoint create", help = "Savepoint a commit")
|
|
||||||
public String savepoint(
|
|
||||||
@CliOption(key = {"commit"}, help = "Commit to savepoint")
|
|
||||||
final String commitTime,
|
|
||||||
@CliOption(key = {"user"}, help = "User who is creating the savepoint")
|
|
||||||
final String user,
|
|
||||||
@CliOption(key = {"comments"}, help = "Comments for creating the savepoint")
|
|
||||||
final String comments) throws Exception {
|
|
||||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
|
||||||
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
|
||||||
HoodieInstant
|
|
||||||
commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
|
||||||
|
|
||||||
if (!timeline.containsInstant(commitInstant)) {
|
|
||||||
return "Commit " + commitTime + " not found in Commits " + timeline;
|
|
||||||
}
|
|
||||||
|
|
||||||
HoodieWriteClient client = createHoodieClient(null, HoodieCLI.tableMetadata.getBasePath());
|
|
||||||
if (client.savepoint(commitTime, user, comments)) {
|
|
||||||
// Refresh the current
|
|
||||||
refreshMetaClient();
|
|
||||||
return String.format("The commit \"%s\" has been savepointed.", commitTime);
|
|
||||||
}
|
|
||||||
return String.format("Failed: Could not savepoint commit \"%s\".", commitTime);
|
|
||||||
}
|
|
||||||
|
|
||||||
@CliCommand(value = "savepoint rollback", help = "Savepoint a commit")
|
|
||||||
public String rollbackToSavepoint(
|
|
||||||
@CliOption(key = {"savepoint"}, help = "Savepoint to rollback")
|
|
||||||
final String commitTime,
|
|
||||||
@CliOption(key = {"sparkProperties"}, help = "Spark Properites File Path")
|
|
||||||
final String sparkPropertiesPath) throws Exception {
|
|
||||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
|
||||||
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
|
||||||
HoodieInstant
|
|
||||||
commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
|
||||||
|
|
||||||
if (!timeline.containsInstant(commitInstant)) {
|
|
||||||
return "Commit " + commitTime + " not found in Commits " + timeline;
|
|
||||||
}
|
|
||||||
|
|
||||||
SparkLauncher sparkLauncher = SparkUtil.initLauncher(sparkPropertiesPath);
|
|
||||||
sparkLauncher.addAppArgs(SparkMain.SparkCommand.ROLLBACK_TO_SAVEPOINT.toString(),
|
|
||||||
commitTime,
|
|
||||||
HoodieCLI.tableMetadata.getBasePath());
|
|
||||||
Process process = sparkLauncher.launch();
|
|
||||||
InputStreamConsumer.captureOutput(process);
|
|
||||||
int exitCode = process.waitFor();
|
|
||||||
// Refresh the current
|
|
||||||
refreshMetaClient();
|
|
||||||
if (exitCode != 0) {
|
|
||||||
return "Savepoint " + commitTime + " failed to roll back";
|
|
||||||
}
|
|
||||||
return "Savepoint " + commitTime + " rolled back";
|
|
||||||
}
|
}
|
||||||
|
return "Savepoint " + commitTime + " rolled back";
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
@CliCommand(value = "savepoints refresh", help = "Refresh the savepoints")
|
@CliCommand(value = "savepoints refresh", help = "Refresh the savepoints")
|
||||||
public String refreshMetaClient() throws IOException {
|
public String refreshMetaClient() throws IOException {
|
||||||
HoodieTableMetaClient metadata =
|
HoodieTableMetaClient metadata =
|
||||||
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
|
new HoodieTableMetaClient(HoodieCLI.fs, HoodieCLI.tableMetadata.getBasePath());
|
||||||
HoodieCLI.setTableMetadata(metadata);
|
HoodieCLI.setTableMetadata(metadata);
|
||||||
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
|
return "Metadata for table " + metadata.getTableConfig().getTableName() + " refreshed.";
|
||||||
}
|
}
|
||||||
|
|
||||||
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath)
|
|
||||||
throws Exception {
|
|
||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
|
||||||
.withIndexConfig(
|
|
||||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
|
||||||
.build();
|
|
||||||
return new HoodieWriteClient(jsc, config, false);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath)
|
||||||
|
throws Exception {
|
||||||
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
|
.withIndexConfig(
|
||||||
|
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||||
|
.build();
|
||||||
|
return new HoodieWriteClient(jsc, config, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -30,109 +30,110 @@ import org.apache.spark.sql.SQLContext;
|
|||||||
|
|
||||||
public class SparkMain {
|
public class SparkMain {
|
||||||
|
|
||||||
protected final static Logger LOG = Logger.getLogger(SparkMain.class);
|
protected final static Logger LOG = Logger.getLogger(SparkMain.class);
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Commands
|
* Commands
|
||||||
*/
|
*/
|
||||||
enum SparkCommand {
|
enum SparkCommand {
|
||||||
ROLLBACK,
|
ROLLBACK,
|
||||||
DEDUPLICATE,
|
DEDUPLICATE,
|
||||||
ROLLBACK_TO_SAVEPOINT,
|
ROLLBACK_TO_SAVEPOINT,
|
||||||
SAVEPOINT,
|
SAVEPOINT,
|
||||||
IMPORT
|
IMPORT
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void main(String[] args) throws Exception {
|
||||||
|
String command = args[0];
|
||||||
|
LOG.info("Invoking SparkMain:" + command);
|
||||||
|
|
||||||
|
SparkCommand cmd = SparkCommand.valueOf(command);
|
||||||
|
|
||||||
|
JavaSparkContext jsc = SparkUtil.initJavaSparkConf("hoodie-cli-" + command);
|
||||||
|
int returnCode = 0;
|
||||||
|
switch (cmd) {
|
||||||
|
case ROLLBACK:
|
||||||
|
assert (args.length == 3);
|
||||||
|
returnCode = rollback(jsc, args[1], args[2]);
|
||||||
|
break;
|
||||||
|
case DEDUPLICATE:
|
||||||
|
assert (args.length == 4);
|
||||||
|
returnCode = deduplicatePartitionPath(jsc, args[1], args[2], args[3]);
|
||||||
|
break;
|
||||||
|
case ROLLBACK_TO_SAVEPOINT:
|
||||||
|
assert (args.length == 3);
|
||||||
|
returnCode = rollbackToSavepoint(jsc, args[1], args[2]);
|
||||||
|
break;
|
||||||
|
case IMPORT:
|
||||||
|
assert (args.length == 11);
|
||||||
|
returnCode = dataImport(jsc, args[1], args[2], args[3], args[4], args[5], args[6],
|
||||||
|
Integer.parseInt(args[7]), args[8], SparkUtil.DEFUALT_SPARK_MASTER, args[9],
|
||||||
|
Integer.parseInt(args[10]));
|
||||||
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
System.exit(returnCode);
|
||||||
String command = args[0];
|
}
|
||||||
LOG.info("Invoking SparkMain:" + command);
|
|
||||||
|
|
||||||
SparkCommand cmd = SparkCommand.valueOf(command);
|
private static int dataImport(JavaSparkContext jsc, String srcPath, String targetPath,
|
||||||
|
String tableName, String tableType, String rowKey, String partitionKey, int parallelism,
|
||||||
|
String schemaFile, String sparkMaster, String sparkMemory, int retry) throws Exception {
|
||||||
|
HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config();
|
||||||
|
cfg.srcPath = srcPath;
|
||||||
|
cfg.targetPath = targetPath;
|
||||||
|
cfg.tableName = tableName;
|
||||||
|
cfg.tableType = tableType;
|
||||||
|
cfg.rowKey = rowKey;
|
||||||
|
cfg.partitionKey = partitionKey;
|
||||||
|
cfg.parallelism = parallelism;
|
||||||
|
cfg.schemaFile = schemaFile;
|
||||||
|
jsc.getConf().set("spark.executor.memory", sparkMemory);
|
||||||
|
return new HDFSParquetImporter(cfg).dataImport(jsc, retry);
|
||||||
|
}
|
||||||
|
|
||||||
JavaSparkContext jsc = SparkUtil.initJavaSparkConf("hoodie-cli-" + command);
|
private static int deduplicatePartitionPath(JavaSparkContext jsc,
|
||||||
int returnCode = 0;
|
String duplicatedPartitionPath,
|
||||||
switch(cmd) {
|
String repairedOutputPath,
|
||||||
case ROLLBACK:
|
String basePath)
|
||||||
assert (args.length == 3);
|
throws Exception {
|
||||||
returnCode = rollback(jsc, args[1], args[2]);
|
DedupeSparkJob job = new DedupeSparkJob(basePath,
|
||||||
break;
|
duplicatedPartitionPath, repairedOutputPath, new SQLContext(jsc), FSUtils.getFs());
|
||||||
case DEDUPLICATE:
|
job.fixDuplicates(true);
|
||||||
assert (args.length == 4);
|
return 0;
|
||||||
returnCode = deduplicatePartitionPath(jsc, args[1], args[2], args[3]);
|
}
|
||||||
break;
|
|
||||||
case ROLLBACK_TO_SAVEPOINT:
|
|
||||||
assert (args.length == 3);
|
|
||||||
returnCode = rollbackToSavepoint(jsc, args[1], args[2]);
|
|
||||||
break;
|
|
||||||
case IMPORT:
|
|
||||||
assert (args.length == 11);
|
|
||||||
returnCode = dataImport(jsc, args[1], args[2], args[3], args[4], args[5], args[6],
|
|
||||||
Integer.parseInt(args[7]), args[8], SparkUtil.DEFUALT_SPARK_MASTER, args[9],
|
|
||||||
Integer.parseInt(args[10]));
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
|
|
||||||
System.exit(returnCode);
|
private static int rollback(JavaSparkContext jsc, String commitTime, String basePath)
|
||||||
|
throws Exception {
|
||||||
|
HoodieWriteClient client = createHoodieClient(jsc, basePath);
|
||||||
|
if (client.rollback(commitTime)) {
|
||||||
|
LOG.info(String.format("The commit \"%s\" rolled back.", commitTime));
|
||||||
|
return 0;
|
||||||
|
} else {
|
||||||
|
LOG.info(String.format("The commit \"%s\" failed to roll back.", commitTime));
|
||||||
|
return -1;
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private static int dataImport(JavaSparkContext jsc, String srcPath, String targetPath,
|
private static int rollbackToSavepoint(JavaSparkContext jsc, String savepointTime,
|
||||||
String tableName, String tableType, String rowKey, String partitionKey, int parallelism,
|
String basePath)
|
||||||
String schemaFile, String sparkMaster, String sparkMemory, int retry) throws Exception {
|
throws Exception {
|
||||||
HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config();
|
HoodieWriteClient client = createHoodieClient(jsc, basePath);
|
||||||
cfg.srcPath = srcPath;
|
if (client.rollbackToSavepoint(savepointTime)) {
|
||||||
cfg.targetPath = targetPath;
|
LOG.info(String.format("The commit \"%s\" rolled back.", savepointTime));
|
||||||
cfg.tableName = tableName;
|
return 0;
|
||||||
cfg.tableType = tableType;
|
} else {
|
||||||
cfg.rowKey = rowKey;
|
LOG.info(String.format("The commit \"%s\" failed to roll back.", savepointTime));
|
||||||
cfg.partitionKey = partitionKey;
|
return -1;
|
||||||
cfg.parallelism = parallelism;
|
|
||||||
cfg.schemaFile = schemaFile;
|
|
||||||
jsc.getConf().set("spark.executor.memory", sparkMemory);
|
|
||||||
return new HDFSParquetImporter(cfg).dataImport(jsc, retry);
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private static int deduplicatePartitionPath(JavaSparkContext jsc,
|
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath)
|
||||||
String duplicatedPartitionPath,
|
throws Exception {
|
||||||
String repairedOutputPath,
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
String basePath)
|
.withIndexConfig(
|
||||||
throws Exception {
|
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||||
DedupeSparkJob job = new DedupeSparkJob(basePath,
|
.build();
|
||||||
duplicatedPartitionPath,repairedOutputPath,new SQLContext(jsc), FSUtils.getFs());
|
return new HoodieWriteClient(jsc, config);
|
||||||
job.fixDuplicates(true);
|
}
|
||||||
return 0;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static int rollback(JavaSparkContext jsc, String commitTime, String basePath)
|
|
||||||
throws Exception {
|
|
||||||
HoodieWriteClient client = createHoodieClient(jsc, basePath);
|
|
||||||
if (client.rollback(commitTime)) {
|
|
||||||
LOG.info(String.format("The commit \"%s\" rolled back.", commitTime));
|
|
||||||
return 0;
|
|
||||||
} else {
|
|
||||||
LOG.info(String.format("The commit \"%s\" failed to roll back.", commitTime));
|
|
||||||
return -1;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static int rollbackToSavepoint(JavaSparkContext jsc, String savepointTime, String basePath)
|
|
||||||
throws Exception {
|
|
||||||
HoodieWriteClient client = createHoodieClient(jsc, basePath);
|
|
||||||
if (client.rollbackToSavepoint(savepointTime)) {
|
|
||||||
LOG.info(String.format("The commit \"%s\" rolled back.", savepointTime));
|
|
||||||
return 0;
|
|
||||||
} else {
|
|
||||||
LOG.info(String.format("The commit \"%s\" failed to roll back.", savepointTime));
|
|
||||||
return -1;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath)
|
|
||||||
throws Exception {
|
|
||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
|
||||||
.withIndexConfig(
|
|
||||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
|
||||||
.build();
|
|
||||||
return new HoodieWriteClient(jsc, config);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -28,7 +28,10 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
|||||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
import com.uber.hoodie.common.util.NumericUtils;
|
import com.uber.hoodie.common.util.NumericUtils;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.text.DecimalFormat;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
@@ -38,106 +41,105 @@ import org.springframework.shell.core.annotation.CliCommand;
|
|||||||
import org.springframework.shell.core.annotation.CliOption;
|
import org.springframework.shell.core.annotation.CliOption;
|
||||||
import org.springframework.stereotype.Component;
|
import org.springframework.stereotype.Component;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.text.DecimalFormat;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
@Component
|
@Component
|
||||||
public class StatsCommand implements CommandMarker {
|
public class StatsCommand implements CommandMarker {
|
||||||
@CliAvailabilityIndicator({"stats wa"})
|
|
||||||
public boolean isWriteAmpAvailable() {
|
@CliAvailabilityIndicator({"stats wa"})
|
||||||
return HoodieCLI.tableMetadata != null;
|
public boolean isWriteAmpAvailable() {
|
||||||
|
return HoodieCLI.tableMetadata != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliCommand(value = "stats wa", help = "Write Amplification. Ratio of how many records were upserted to how many records were actually written")
|
||||||
|
public String writeAmplificationStats() throws IOException {
|
||||||
|
long totalRecordsUpserted = 0;
|
||||||
|
long totalRecordsWritten = 0;
|
||||||
|
|
||||||
|
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||||
|
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
||||||
|
|
||||||
|
String[][] rows = new String[new Long(timeline.countInstants()).intValue() + 1][];
|
||||||
|
int i = 0;
|
||||||
|
DecimalFormat df = new DecimalFormat("#.00");
|
||||||
|
for (HoodieInstant commitTime : timeline.getInstants().collect(
|
||||||
|
Collectors.toList())) {
|
||||||
|
String waf = "0";
|
||||||
|
HoodieCommitMetadata commit = HoodieCommitMetadata
|
||||||
|
.fromBytes(activeTimeline.getInstantDetails(commitTime).get());
|
||||||
|
if (commit.fetchTotalUpdateRecordsWritten() > 0) {
|
||||||
|
waf = df.format(
|
||||||
|
(float) commit.fetchTotalRecordsWritten() / commit
|
||||||
|
.fetchTotalUpdateRecordsWritten());
|
||||||
|
}
|
||||||
|
rows[i++] = new String[]{commitTime.getTimestamp(),
|
||||||
|
String.valueOf(commit.fetchTotalUpdateRecordsWritten()),
|
||||||
|
String.valueOf(commit.fetchTotalRecordsWritten()), waf};
|
||||||
|
totalRecordsUpserted += commit.fetchTotalUpdateRecordsWritten();
|
||||||
|
totalRecordsWritten += commit.fetchTotalRecordsWritten();
|
||||||
|
}
|
||||||
|
String waf = "0";
|
||||||
|
if (totalRecordsUpserted > 0) {
|
||||||
|
waf = df.format((float) totalRecordsWritten / totalRecordsUpserted);
|
||||||
|
}
|
||||||
|
rows[i] = new String[]{"Total", String.valueOf(totalRecordsUpserted),
|
||||||
|
String.valueOf(totalRecordsWritten), waf};
|
||||||
|
return HoodiePrintHelper.print(
|
||||||
|
new String[]{"CommitTime", "Total Upserted", "Total Written",
|
||||||
|
"Write Amplifiation Factor"}, rows);
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private String[] printFileSizeHistogram(String commitTime, Snapshot s) {
|
||||||
|
return new String[]{
|
||||||
|
commitTime,
|
||||||
|
NumericUtils.humanReadableByteCount(s.getMin()),
|
||||||
|
NumericUtils.humanReadableByteCount(s.getValue(0.1)),
|
||||||
|
NumericUtils.humanReadableByteCount(s.getMedian()),
|
||||||
|
NumericUtils.humanReadableByteCount(s.getMean()),
|
||||||
|
NumericUtils.humanReadableByteCount(s.get95thPercentile()),
|
||||||
|
NumericUtils.humanReadableByteCount(s.getMax()),
|
||||||
|
String.valueOf(s.size()),
|
||||||
|
NumericUtils.humanReadableByteCount(s.getStdDev())
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@CliCommand(value = "stats filesizes", help = "File Sizes. Display summary stats on sizes of files")
|
||||||
|
public String fileSizeStats(
|
||||||
|
@CliOption(key = {
|
||||||
|
"partitionPath"}, help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*")
|
||||||
|
final String globRegex) throws IOException {
|
||||||
|
|
||||||
|
FileSystem fs = HoodieCLI.fs;
|
||||||
|
String globPath = String.format("%s/%s/*",
|
||||||
|
HoodieCLI.tableMetadata.getBasePath(),
|
||||||
|
globRegex);
|
||||||
|
FileStatus[] statuses = fs.globStatus(new Path(globPath));
|
||||||
|
|
||||||
|
// max, min, #small files < 10MB, 50th, avg, 95th
|
||||||
|
final int MAX_FILES = 1000000;
|
||||||
|
Histogram globalHistogram = new Histogram(new UniformReservoir(MAX_FILES));
|
||||||
|
HashMap<String, Histogram> commitHistoMap = new HashMap<String, Histogram>();
|
||||||
|
for (FileStatus fileStatus : statuses) {
|
||||||
|
String commitTime = FSUtils.getCommitTime(fileStatus.getPath().getName());
|
||||||
|
long sz = fileStatus.getLen();
|
||||||
|
if (!commitHistoMap.containsKey(commitTime)) {
|
||||||
|
commitHistoMap.put(commitTime, new Histogram(new UniformReservoir(MAX_FILES)));
|
||||||
|
}
|
||||||
|
commitHistoMap.get(commitTime).update(sz);
|
||||||
|
globalHistogram.update(sz);
|
||||||
}
|
}
|
||||||
|
|
||||||
@CliCommand(value = "stats wa", help = "Write Amplification. Ratio of how many records were upserted to how many records were actually written")
|
String[][] rows = new String[commitHistoMap.size() + 1][];
|
||||||
public String writeAmplificationStats() throws IOException {
|
int ind = 0;
|
||||||
long totalRecordsUpserted = 0;
|
for (String commitTime : commitHistoMap.keySet()) {
|
||||||
long totalRecordsWritten = 0;
|
Snapshot s = commitHistoMap.get(commitTime).getSnapshot();
|
||||||
|
rows[ind++] = printFileSizeHistogram(commitTime, s);
|
||||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
|
||||||
HoodieTimeline timeline = activeTimeline.getCommitTimeline().filterCompletedInstants();
|
|
||||||
|
|
||||||
String[][] rows = new String[new Long(timeline.countInstants()).intValue() + 1][];
|
|
||||||
int i = 0;
|
|
||||||
DecimalFormat df = new DecimalFormat("#.00");
|
|
||||||
for (HoodieInstant commitTime : timeline.getInstants().collect(
|
|
||||||
Collectors.toList())) {
|
|
||||||
String waf = "0";
|
|
||||||
HoodieCommitMetadata commit = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(commitTime).get());
|
|
||||||
if (commit.fetchTotalUpdateRecordsWritten() > 0) {
|
|
||||||
waf = df.format(
|
|
||||||
(float) commit.fetchTotalRecordsWritten() / commit
|
|
||||||
.fetchTotalUpdateRecordsWritten());
|
|
||||||
}
|
|
||||||
rows[i++] = new String[] {commitTime.getTimestamp(),
|
|
||||||
String.valueOf(commit.fetchTotalUpdateRecordsWritten()),
|
|
||||||
String.valueOf(commit.fetchTotalRecordsWritten()), waf};
|
|
||||||
totalRecordsUpserted += commit.fetchTotalUpdateRecordsWritten();
|
|
||||||
totalRecordsWritten += commit.fetchTotalRecordsWritten();
|
|
||||||
}
|
|
||||||
String waf = "0";
|
|
||||||
if (totalRecordsUpserted > 0) {
|
|
||||||
waf = df.format((float) totalRecordsWritten / totalRecordsUpserted);
|
|
||||||
}
|
|
||||||
rows[i] = new String[] {"Total", String.valueOf(totalRecordsUpserted),
|
|
||||||
String.valueOf(totalRecordsWritten), waf};
|
|
||||||
return HoodiePrintHelper.print(
|
|
||||||
new String[] {"CommitTime", "Total Upserted", "Total Written",
|
|
||||||
"Write Amplifiation Factor"}, rows);
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
Snapshot s = globalHistogram.getSnapshot();
|
||||||
|
rows[ind++] = printFileSizeHistogram("ALL", s);
|
||||||
|
|
||||||
|
return HoodiePrintHelper.print(
|
||||||
private String[] printFileSizeHistogram(String commitTime, Snapshot s) {
|
new String[]{"CommitTime", "Min", "10th", "50th", "avg", "95th", "Max", "NumFiles",
|
||||||
return new String[]{
|
"StdDev"}, rows);
|
||||||
commitTime,
|
}
|
||||||
NumericUtils.humanReadableByteCount(s.getMin()),
|
|
||||||
NumericUtils.humanReadableByteCount(s.getValue(0.1)),
|
|
||||||
NumericUtils.humanReadableByteCount(s.getMedian()),
|
|
||||||
NumericUtils.humanReadableByteCount(s.getMean()),
|
|
||||||
NumericUtils.humanReadableByteCount(s.get95thPercentile()),
|
|
||||||
NumericUtils.humanReadableByteCount(s.getMax()),
|
|
||||||
String.valueOf(s.size()),
|
|
||||||
NumericUtils.humanReadableByteCount(s.getStdDev())
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
@CliCommand(value = "stats filesizes", help = "File Sizes. Display summary stats on sizes of files")
|
|
||||||
public String fileSizeStats(
|
|
||||||
@CliOption(key = {"partitionPath"}, help = "regex to select files, eg: 2016/08/02", unspecifiedDefaultValue = "*/*/*")
|
|
||||||
final String globRegex) throws IOException {
|
|
||||||
|
|
||||||
FileSystem fs = HoodieCLI.fs;
|
|
||||||
String globPath = String.format("%s/%s/*",
|
|
||||||
HoodieCLI.tableMetadata.getBasePath(),
|
|
||||||
globRegex);
|
|
||||||
FileStatus[] statuses = fs.globStatus(new Path(globPath));
|
|
||||||
|
|
||||||
// max, min, #small files < 10MB, 50th, avg, 95th
|
|
||||||
final int MAX_FILES = 1000000;
|
|
||||||
Histogram globalHistogram = new Histogram(new UniformReservoir(MAX_FILES));
|
|
||||||
HashMap<String, Histogram> commitHistoMap = new HashMap<String, Histogram>();
|
|
||||||
for (FileStatus fileStatus: statuses) {
|
|
||||||
String commitTime = FSUtils.getCommitTime(fileStatus.getPath().getName());
|
|
||||||
long sz = fileStatus.getLen();
|
|
||||||
if (!commitHistoMap.containsKey(commitTime)) {
|
|
||||||
commitHistoMap.put(commitTime, new Histogram(new UniformReservoir(MAX_FILES)));
|
|
||||||
}
|
|
||||||
commitHistoMap.get(commitTime).update(sz);
|
|
||||||
globalHistogram.update(sz);
|
|
||||||
}
|
|
||||||
|
|
||||||
String[][] rows = new String[commitHistoMap.size() + 1][];
|
|
||||||
int ind = 0;
|
|
||||||
for (String commitTime: commitHistoMap.keySet()) {
|
|
||||||
Snapshot s = commitHistoMap.get(commitTime).getSnapshot();
|
|
||||||
rows[ind++] = printFileSizeHistogram(commitTime, s);
|
|
||||||
}
|
|
||||||
Snapshot s = globalHistogram.getSnapshot();
|
|
||||||
rows[ind++] = printFileSizeHistogram("ALL", s);
|
|
||||||
|
|
||||||
return HoodiePrintHelper.print(
|
|
||||||
new String[] {"CommitTime", "Min", "10th", "50th", "avg", "95th", "Max", "NumFiles", "StdDev"}, rows);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -23,12 +23,13 @@ import org.springframework.stereotype.Component;
|
|||||||
|
|
||||||
@Component
|
@Component
|
||||||
public class UtilsCommand implements CommandMarker {
|
public class UtilsCommand implements CommandMarker {
|
||||||
@CliCommand(value = "utils loadClass", help = "Load a class" )
|
|
||||||
public String loadClass(
|
@CliCommand(value = "utils loadClass", help = "Load a class")
|
||||||
@CliOption(key = {"class"}, help = "Check mode" ) final String clazz
|
public String loadClass(
|
||||||
) throws Exception {
|
@CliOption(key = {"class"}, help = "Check mode") final String clazz
|
||||||
Class klass = Class.forName(clazz);
|
) throws Exception {
|
||||||
return klass.getProtectionDomain().getCodeSource().getLocation().toExternalForm();
|
Class klass = Class.forName(clazz);
|
||||||
}
|
return klass.getProtectionDomain().getCodeSource().getLocation().toExternalForm();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -20,21 +20,22 @@ import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
|||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
public class CommitUtil {
|
public class CommitUtil {
|
||||||
public static long countNewRecords(HoodieTableMetaClient target, List<String> commitsToCatchup)
|
|
||||||
throws IOException {
|
public static long countNewRecords(HoodieTableMetaClient target, List<String> commitsToCatchup)
|
||||||
long totalNew = 0;
|
throws IOException {
|
||||||
HoodieTimeline timeline = target.getActiveTimeline().reload().getCommitTimeline().filterCompletedInstants();
|
long totalNew = 0;
|
||||||
for(String commit:commitsToCatchup) {
|
HoodieTimeline timeline = target.getActiveTimeline().reload().getCommitTimeline()
|
||||||
HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes(timeline
|
.filterCompletedInstants();
|
||||||
.getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit))
|
for (String commit : commitsToCatchup) {
|
||||||
.get());
|
HoodieCommitMetadata c = HoodieCommitMetadata.fromBytes(timeline
|
||||||
totalNew += c.fetchTotalRecordsWritten() - c.fetchTotalUpdateRecordsWritten();
|
.getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit))
|
||||||
}
|
.get());
|
||||||
return totalNew;
|
totalNew += c.fetchTotalRecordsWritten() - c.fetchTotalUpdateRecordsWritten();
|
||||||
}
|
}
|
||||||
|
return totalNew;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,107 +17,112 @@
|
|||||||
package com.uber.hoodie.cli.utils;
|
package com.uber.hoodie.cli.utils;
|
||||||
|
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
import org.apache.commons.dbcp.BasicDataSource;
|
|
||||||
import org.joda.time.DateTime;
|
|
||||||
|
|
||||||
import javax.sql.DataSource;
|
|
||||||
import java.sql.Connection;
|
import java.sql.Connection;
|
||||||
import java.sql.ResultSet;
|
import java.sql.ResultSet;
|
||||||
import java.sql.SQLException;
|
import java.sql.SQLException;
|
||||||
import java.sql.Statement;
|
import java.sql.Statement;
|
||||||
|
import javax.sql.DataSource;
|
||||||
|
import org.apache.commons.dbcp.BasicDataSource;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
public class HiveUtil {
|
public class HiveUtil {
|
||||||
private static String driverName = "org.apache.hive.jdbc.HiveDriver";
|
|
||||||
|
|
||||||
static {
|
private static String driverName = "org.apache.hive.jdbc.HiveDriver";
|
||||||
try {
|
|
||||||
Class.forName(driverName);
|
static {
|
||||||
} catch (ClassNotFoundException e) {
|
try {
|
||||||
throw new IllegalStateException("Could not find " + driverName + " in classpath. ", e);
|
Class.forName(driverName);
|
||||||
}
|
} catch (ClassNotFoundException e) {
|
||||||
|
throw new IllegalStateException("Could not find " + driverName + " in classpath. ", e);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private static Connection connection;
|
private static Connection connection;
|
||||||
|
|
||||||
private static Connection getConnection(String jdbcUrl, String user, String pass) throws SQLException {
|
private static Connection getConnection(String jdbcUrl, String user, String pass)
|
||||||
DataSource ds = getDatasource(jdbcUrl, user, pass);
|
throws SQLException {
|
||||||
return ds.getConnection();
|
DataSource ds = getDatasource(jdbcUrl, user, pass);
|
||||||
|
return ds.getConnection();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static DataSource getDatasource(String jdbcUrl, String user, String pass) {
|
||||||
|
BasicDataSource ds = new BasicDataSource();
|
||||||
|
ds.setDriverClassName(driverName);
|
||||||
|
ds.setUrl(jdbcUrl);
|
||||||
|
ds.setUsername(user);
|
||||||
|
ds.setPassword(pass);
|
||||||
|
return ds;
|
||||||
|
}
|
||||||
|
|
||||||
|
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();
|
||||||
|
try {
|
||||||
|
//stmt.execute("set mapred.job.queue.name=<queue_name>");
|
||||||
|
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
|
||||||
|
.getTableConfig()
|
||||||
|
.getTableName());
|
||||||
|
long count = -1;
|
||||||
|
if (rs.next()) {
|
||||||
|
count = rs.getLong("cnt");
|
||||||
|
}
|
||||||
|
System.out
|
||||||
|
.println("Total records in " + source.getTableConfig().getTableName() + " is " + count);
|
||||||
|
return count;
|
||||||
|
} finally {
|
||||||
|
if (rs != null) {
|
||||||
|
rs.close();
|
||||||
|
}
|
||||||
|
if (stmt != null) {
|
||||||
|
stmt.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private static DataSource getDatasource(String jdbcUrl, String user, String pass) {
|
public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb,
|
||||||
BasicDataSource ds = new BasicDataSource();
|
int partitions, String user, String pass) throws SQLException {
|
||||||
ds.setDriverClassName(driverName);
|
DateTime dateTime = DateTime.now();
|
||||||
ds.setUrl(jdbcUrl);
|
String endDateStr =
|
||||||
ds.setUsername(user);
|
dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" +
|
||||||
ds.setPassword(pass);
|
String.format("%02d", dateTime.getDayOfMonth());
|
||||||
return ds;
|
dateTime = dateTime.minusDays(partitions);
|
||||||
}
|
String startDateStr =
|
||||||
|
dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" +
|
||||||
|
String.format("%02d", dateTime.getDayOfMonth());
|
||||||
|
System.out.println("Start date " + startDateStr + " and end date " + endDateStr);
|
||||||
|
return countRecords(jdbcUrl, source, srcDb, startDateStr, endDateStr, user, pass);
|
||||||
|
}
|
||||||
|
|
||||||
public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String dbName, String user, String pass) throws SQLException {
|
private static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb,
|
||||||
Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass);
|
String startDateStr,
|
||||||
ResultSet rs = null;
|
String endDateStr, String user, String pass) throws SQLException {
|
||||||
Statement stmt = conn.createStatement();
|
Connection conn = HiveUtil.getConnection(jdbcUrl, user, pass);
|
||||||
try {
|
ResultSet rs = null;
|
||||||
//stmt.execute("set mapred.job.queue.name=<queue_name>");
|
Statement stmt = conn.createStatement();
|
||||||
stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat" );
|
try {
|
||||||
stmt.execute("set hive.stats.autogather=false" );
|
//stmt.execute("set mapred.job.queue.name=<queue_name>");
|
||||||
rs = stmt.executeQuery(
|
stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat");
|
||||||
"select count(`_hoodie_commit_time`) as cnt from " + dbName + "." + source.getTableConfig()
|
stmt.execute("set hive.stats.autogather=false");
|
||||||
.getTableName());
|
rs = stmt.executeQuery(
|
||||||
long count = -1;
|
"select count(`_hoodie_commit_time`) as cnt from " + srcDb + "." + source.getTableConfig()
|
||||||
if(rs.next()) {
|
.getTableName() + " where datestr>'" + startDateStr + "' and datestr<='"
|
||||||
count = rs.getLong("cnt");
|
+ endDateStr + "'");
|
||||||
}
|
if (rs.next()) {
|
||||||
System.out.println("Total records in " + source.getTableConfig().getTableName() + " is " + count);
|
return rs.getLong("cnt");
|
||||||
return count;
|
}
|
||||||
} finally {
|
return -1;
|
||||||
if (rs != null) {
|
} finally {
|
||||||
rs.close();
|
if (rs != null) {
|
||||||
}
|
rs.close();
|
||||||
if (stmt != null) {
|
}
|
||||||
stmt.close();
|
if (stmt != null) {
|
||||||
}
|
stmt.close();
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
public static long countRecords(String jdbcUrl, HoodieTableMetaClient source, String srcDb,
|
|
||||||
int partitions, String user, String pass) throws SQLException {
|
|
||||||
DateTime dateTime = DateTime.now();
|
|
||||||
String endDateStr =
|
|
||||||
dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" +
|
|
||||||
String.format("%02d", dateTime.getDayOfMonth());
|
|
||||||
dateTime = dateTime.minusDays(partitions);
|
|
||||||
String startDateStr =
|
|
||||||
dateTime.getYear() + "-" + String.format("%02d", dateTime.getMonthOfYear()) + "-" +
|
|
||||||
String.format("%02d", dateTime.getDayOfMonth());
|
|
||||||
System.out.println("Start date " + startDateStr + " and end date " + endDateStr);
|
|
||||||
return countRecords(jdbcUrl, source, srcDb, startDateStr, endDateStr, user, pass);
|
|
||||||
}
|
|
||||||
|
|
||||||
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;
|
|
||||||
Statement stmt = conn.createStatement();
|
|
||||||
try {
|
|
||||||
//stmt.execute("set mapred.job.queue.name=<queue_name>");
|
|
||||||
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.getTableConfig()
|
|
||||||
.getTableName() + " where datestr>'" + startDateStr + "' and datestr<='"
|
|
||||||
+ endDateStr + "'");
|
|
||||||
if(rs.next()) {
|
|
||||||
return rs.getLong("cnt");
|
|
||||||
}
|
|
||||||
return -1;
|
|
||||||
} finally {
|
|
||||||
if (rs != null) {
|
|
||||||
rs.close();
|
|
||||||
}
|
|
||||||
if (stmt != null) {
|
|
||||||
stmt.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -23,34 +23,37 @@ import java.io.InputStreamReader;
|
|||||||
import java.util.logging.Logger;
|
import java.util.logging.Logger;
|
||||||
|
|
||||||
public class InputStreamConsumer extends Thread {
|
public class InputStreamConsumer extends Thread {
|
||||||
protected final static Logger LOG = Logger.getLogger(InputStreamConsumer.class.getName());
|
|
||||||
private InputStream is;
|
|
||||||
public InputStreamConsumer(InputStream is) {
|
|
||||||
this.is = is;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
protected final static Logger LOG = Logger.getLogger(InputStreamConsumer.class.getName());
|
||||||
public void run() {
|
private InputStream is;
|
||||||
try {
|
|
||||||
InputStreamReader isr = new InputStreamReader(is);
|
|
||||||
BufferedReader br = new BufferedReader(isr);
|
|
||||||
String line;
|
|
||||||
while ( (line = br.readLine()) != null)
|
|
||||||
LOG.info(line);
|
|
||||||
} catch (IOException ioe) {
|
|
||||||
LOG.severe(ioe.toString());
|
|
||||||
ioe.printStackTrace();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void captureOutput(Process p) {
|
public InputStreamConsumer(InputStream is) {
|
||||||
InputStreamConsumer stdout;
|
this.is = is;
|
||||||
InputStreamConsumer errout;
|
}
|
||||||
errout = new InputStreamConsumer(p.getErrorStream());
|
|
||||||
stdout = new InputStreamConsumer(p.getInputStream());
|
@Override
|
||||||
errout.start();
|
public void run() {
|
||||||
stdout.start();
|
try {
|
||||||
|
InputStreamReader isr = new InputStreamReader(is);
|
||||||
|
BufferedReader br = new BufferedReader(isr);
|
||||||
|
String line;
|
||||||
|
while ((line = br.readLine()) != null) {
|
||||||
|
LOG.info(line);
|
||||||
|
}
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
LOG.severe(ioe.toString());
|
||||||
|
ioe.printStackTrace();
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void captureOutput(Process p) {
|
||||||
|
InputStreamConsumer stdout;
|
||||||
|
InputStreamConsumer errout;
|
||||||
|
errout = new InputStreamConsumer(p.getErrorStream());
|
||||||
|
stdout = new InputStreamConsumer(p.getInputStream());
|
||||||
|
errout.start();
|
||||||
|
stdout.start();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,59 +18,54 @@ package com.uber.hoodie.cli.utils;
|
|||||||
|
|
||||||
import com.uber.hoodie.HoodieWriteClient;
|
import com.uber.hoodie.HoodieWriteClient;
|
||||||
import com.uber.hoodie.cli.commands.SparkMain;
|
import com.uber.hoodie.cli.commands.SparkMain;
|
||||||
|
import java.io.File;
|
||||||
|
import java.net.URISyntaxException;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.apache.spark.launcher.SparkLauncher;
|
import org.apache.spark.launcher.SparkLauncher;
|
||||||
|
|
||||||
import java.io.File;
|
|
||||||
import java.net.URISyntaxException;
|
|
||||||
|
|
||||||
public class SparkUtil {
|
public class SparkUtil {
|
||||||
|
|
||||||
public static Logger logger = Logger.getLogger(SparkUtil.class);
|
public static Logger logger = Logger.getLogger(SparkUtil.class);
|
||||||
public static final String DEFUALT_SPARK_MASTER = "yarn-client";
|
public static final String DEFUALT_SPARK_MASTER = "yarn-client";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
* TODO: Need to fix a bunch of hardcoded stuff here eg: history server, spark distro
|
||||||
* TODO: Need to fix a bunch of hardcoded stuff here eg: history server, spark distro
|
*/
|
||||||
*
|
public static SparkLauncher initLauncher(String propertiesFile) throws URISyntaxException {
|
||||||
* @return
|
String currentJar = new File(
|
||||||
* @throws URISyntaxException
|
SparkUtil.class.getProtectionDomain().getCodeSource().getLocation().toURI().getPath())
|
||||||
*/
|
.getAbsolutePath();
|
||||||
public static SparkLauncher initLauncher(String propertiesFile) throws URISyntaxException {
|
SparkLauncher sparkLauncher =
|
||||||
String currentJar = new File(
|
new SparkLauncher().setAppResource(currentJar)
|
||||||
SparkUtil.class.getProtectionDomain().getCodeSource().getLocation().toURI().getPath())
|
.setMainClass(SparkMain.class.getName())
|
||||||
.getAbsolutePath();
|
.setPropertiesFile(propertiesFile);
|
||||||
SparkLauncher sparkLauncher =
|
File libDirectory = new File(new File(currentJar).getParent(), "lib");
|
||||||
new SparkLauncher().setAppResource(currentJar)
|
for (String library : libDirectory.list()) {
|
||||||
.setMainClass(SparkMain.class.getName())
|
sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath());
|
||||||
.setPropertiesFile(propertiesFile);
|
|
||||||
File libDirectory = new File(new File(currentJar).getParent(), "lib");
|
|
||||||
for (String library : libDirectory.list()) {
|
|
||||||
sparkLauncher.addJar(new File(libDirectory, library).getAbsolutePath());
|
|
||||||
}
|
|
||||||
return sparkLauncher;
|
|
||||||
}
|
}
|
||||||
|
return sparkLauncher;
|
||||||
|
}
|
||||||
|
|
||||||
public static JavaSparkContext initJavaSparkConf(String name) {
|
public static JavaSparkContext initJavaSparkConf(String name) {
|
||||||
SparkConf sparkConf = new SparkConf().setAppName(name);
|
SparkConf sparkConf = new SparkConf().setAppName(name);
|
||||||
sparkConf.setMaster(DEFUALT_SPARK_MASTER);
|
sparkConf.setMaster(DEFUALT_SPARK_MASTER);
|
||||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||||
sparkConf.set("spark.driver.maxResultSize", "2g");
|
sparkConf.set("spark.driver.maxResultSize", "2g");
|
||||||
sparkConf.set("spark.eventLog.overwrite", "true");
|
sparkConf.set("spark.eventLog.overwrite", "true");
|
||||||
sparkConf.set("spark.eventLog.enabled", "true");
|
sparkConf.set("spark.eventLog.enabled", "true");
|
||||||
|
|
||||||
// Configure hadoop conf
|
// Configure hadoop conf
|
||||||
sparkConf.set("spark.hadoop.mapred.output.compress", "true");
|
sparkConf.set("spark.hadoop.mapred.output.compress", "true");
|
||||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true");
|
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true");
|
||||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "org.apache.hadoop.io.compress.GzipCodec");
|
sparkConf.set("spark.hadoop.mapred.output.compression.codec",
|
||||||
sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK");
|
"org.apache.hadoop.io.compress.GzipCodec");
|
||||||
|
sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK");
|
||||||
|
|
||||||
sparkConf = HoodieWriteClient.registerClasses(sparkConf);
|
sparkConf = HoodieWriteClient.registerClasses(sparkConf);
|
||||||
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
|
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
|
||||||
jsc.hadoopConfiguration().setBoolean("parquet.enable.summary-metadata", false);
|
jsc.hadoopConfiguration().setBoolean("parquet.enable.summary-metadata", false);
|
||||||
return jsc;
|
return jsc;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,11 +16,11 @@
|
|||||||
-->
|
-->
|
||||||
|
|
||||||
<beans xmlns="http://www.springframework.org/schema/beans"
|
<beans xmlns="http://www.springframework.org/schema/beans"
|
||||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||||
xmlns:context="http://www.springframework.org/schema/context"
|
xmlns:context="http://www.springframework.org/schema/context"
|
||||||
xsi:schemaLocation="http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
|
xsi:schemaLocation="http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
|
||||||
http://www.springframework.org/schema/context http://www.springframework.org/schema/context/spring-context.xsd">
|
http://www.springframework.org/schema/context http://www.springframework.org/schema/context/spring-context.xsd">
|
||||||
|
|
||||||
<context:component-scan base-package="com.uber.hoodie.cli" />
|
<context:component-scan base-package="com.uber.hoodie.cli"/>
|
||||||
|
|
||||||
</beans>
|
</beans>
|
||||||
|
|||||||
@@ -34,11 +34,11 @@ import scala.collection.mutable._
|
|||||||
/**
|
/**
|
||||||
* Spark job to de-duplicate data present in a partition path
|
* Spark job to de-duplicate data present in a partition path
|
||||||
*/
|
*/
|
||||||
class DedupeSparkJob (basePath: String,
|
class DedupeSparkJob(basePath: String,
|
||||||
duplicatedPartitionPath: String,
|
duplicatedPartitionPath: String,
|
||||||
repairOutputPath: String,
|
repairOutputPath: String,
|
||||||
sqlContext: SQLContext,
|
sqlContext: SQLContext,
|
||||||
fs: FileSystem) {
|
fs: FileSystem) {
|
||||||
|
|
||||||
|
|
||||||
val sparkHelper = new SparkHelper(sqlContext, fs)
|
val sparkHelper = new SparkHelper(sqlContext, fs)
|
||||||
@@ -50,8 +50,9 @@ class DedupeSparkJob (basePath: String,
|
|||||||
* @param tblName
|
* @param tblName
|
||||||
* @return
|
* @return
|
||||||
*/
|
*/
|
||||||
def getDupeKeyDF(tblName: String) : DataFrame = {
|
def getDupeKeyDF(tblName: String): DataFrame = {
|
||||||
val dupeSql = s"""
|
val dupeSql =
|
||||||
|
s"""
|
||||||
select `${HoodieRecord.RECORD_KEY_METADATA_FIELD}` as dupe_key,
|
select `${HoodieRecord.RECORD_KEY_METADATA_FIELD}` as dupe_key,
|
||||||
count(*) as dupe_cnt
|
count(*) as dupe_cnt
|
||||||
from ${tblName}
|
from ${tblName}
|
||||||
@@ -69,7 +70,7 @@ class DedupeSparkJob (basePath: String,
|
|||||||
*
|
*
|
||||||
* @return
|
* @return
|
||||||
*/
|
*/
|
||||||
private def planDuplicateFix() : HashMap[String, HashSet[String]] = {
|
private def planDuplicateFix(): HashMap[String, HashSet[String]] = {
|
||||||
|
|
||||||
val tmpTableName = s"htbl_${System.currentTimeMillis()}"
|
val tmpTableName = s"htbl_${System.currentTimeMillis()}"
|
||||||
val dedupeTblName = s"${tmpTableName}_dupeKeys"
|
val dedupeTblName = s"${tmpTableName}_dupeKeys"
|
||||||
@@ -78,17 +79,18 @@ class DedupeSparkJob (basePath: String,
|
|||||||
|
|
||||||
val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"${basePath}/${duplicatedPartitionPath}"))
|
val allFiles = fs.listStatus(new org.apache.hadoop.fs.Path(s"${basePath}/${duplicatedPartitionPath}"))
|
||||||
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
|
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
|
||||||
val latestFiles:java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]())
|
val latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]())
|
||||||
val filteredStatuses = latestFiles.map(f => f.getPath)
|
val filteredStatuses = latestFiles.map(f => f.getPath)
|
||||||
LOG.info(s" List of files under partition: ${} => ${filteredStatuses.mkString(" ")}")
|
LOG.info(s" List of files under partition: ${} => ${filteredStatuses.mkString(" ")}")
|
||||||
|
|
||||||
val df = sqlContext.parquetFile(filteredStatuses:_*)
|
val df = sqlContext.parquetFile(filteredStatuses: _*)
|
||||||
df.registerTempTable(tmpTableName)
|
df.registerTempTable(tmpTableName)
|
||||||
val dupeKeyDF = getDupeKeyDF(tmpTableName)
|
val dupeKeyDF = getDupeKeyDF(tmpTableName)
|
||||||
dupeKeyDF.registerTempTable(dedupeTblName)
|
dupeKeyDF.registerTempTable(dedupeTblName)
|
||||||
|
|
||||||
// Obtain necessary satellite information for duplicate rows
|
// Obtain necessary satellite information for duplicate rows
|
||||||
val dupeDataSql = s"""
|
val dupeDataSql =
|
||||||
|
s"""
|
||||||
SELECT `_hoodie_record_key`, `_hoodie_partition_path`, `_hoodie_file_name`, `_hoodie_commit_time`
|
SELECT `_hoodie_record_key`, `_hoodie_partition_path`, `_hoodie_file_name`, `_hoodie_commit_time`
|
||||||
FROM ${tmpTableName} h
|
FROM ${tmpTableName} h
|
||||||
JOIN ${dedupeTblName} d
|
JOIN ${dedupeTblName} d
|
||||||
@@ -111,9 +113,9 @@ class DedupeSparkJob (basePath: String,
|
|||||||
|
|
||||||
rows.foreach(r => {
|
rows.foreach(r => {
|
||||||
val c = r(3).asInstanceOf[String].toLong
|
val c = r(3).asInstanceOf[String].toLong
|
||||||
if (c != maxCommit){
|
if (c != maxCommit) {
|
||||||
val f = r(2).asInstanceOf[String].split("_")(0)
|
val f = r(2).asInstanceOf[String].split("_")(0)
|
||||||
if (!fileToDeleteKeyMap.contains(f)){
|
if (!fileToDeleteKeyMap.contains(f)) {
|
||||||
fileToDeleteKeyMap(f) = HashSet[String]()
|
fileToDeleteKeyMap(f) = HashSet[String]()
|
||||||
}
|
}
|
||||||
fileToDeleteKeyMap(f).add(key)
|
fileToDeleteKeyMap(f).add(key)
|
||||||
@@ -130,28 +132,30 @@ class DedupeSparkJob (basePath: String,
|
|||||||
val allFiles = fs.listStatus(new Path(s"${basePath}/${duplicatedPartitionPath}"))
|
val allFiles = fs.listStatus(new Path(s"${basePath}/${duplicatedPartitionPath}"))
|
||||||
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
|
val fsView = new HoodieTableFileSystemView(metadata, metadata.getActiveTimeline.getCommitTimeline.filterCompletedInstants(), allFiles)
|
||||||
|
|
||||||
val latestFiles:java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]())
|
val latestFiles: java.util.List[HoodieDataFile] = fsView.getLatestDataFiles().collect(Collectors.toList[HoodieDataFile]())
|
||||||
|
|
||||||
val fileNameToPathMap = latestFiles.map(f => (f.getFileId, new Path(f.getPath))).toMap
|
val fileNameToPathMap = latestFiles.map(f => (f.getFileId, new Path(f.getPath))).toMap
|
||||||
val dupeFixPlan = planDuplicateFix()
|
val dupeFixPlan = planDuplicateFix()
|
||||||
|
|
||||||
// 1. Copy all latest files into the temp fix path
|
// 1. Copy all latest files into the temp fix path
|
||||||
fileNameToPathMap.foreach{ case(fileName, filePath) => {
|
fileNameToPathMap.foreach { case (fileName, filePath) => {
|
||||||
val badSuffix = if (dupeFixPlan.contains(fileName)) ".bad" else ""
|
val badSuffix = if (dupeFixPlan.contains(fileName)) ".bad" else ""
|
||||||
val dstPath = new Path(s"${repairOutputPath}/${filePath.getName}${badSuffix}")
|
val dstPath = new Path(s"${repairOutputPath}/${filePath.getName}${badSuffix}")
|
||||||
LOG.info(s"Copying from ${filePath} to ${dstPath}")
|
LOG.info(s"Copying from ${filePath} to ${dstPath}")
|
||||||
FileUtil.copy(fs, filePath, fs, dstPath, false, true, fs.getConf)
|
FileUtil.copy(fs, filePath, fs, dstPath, false, true, fs.getConf)
|
||||||
}}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// 2. Remove duplicates from the bad files
|
// 2. Remove duplicates from the bad files
|
||||||
dupeFixPlan.foreach{case(fileName, keysToSkip) => {
|
dupeFixPlan.foreach { case (fileName, keysToSkip) => {
|
||||||
val commitTime = FSUtils.getCommitTime(fileNameToPathMap(fileName).getName)
|
val commitTime = FSUtils.getCommitTime(fileNameToPathMap(fileName).getName)
|
||||||
val badFilePath = new Path(s"${repairOutputPath}/${fileNameToPathMap(fileName).getName}.bad")
|
val badFilePath = new Path(s"${repairOutputPath}/${fileNameToPathMap(fileName).getName}.bad")
|
||||||
val newFilePath = new Path(s"${repairOutputPath}/${fileNameToPathMap(fileName).getName}")
|
val newFilePath = new Path(s"${repairOutputPath}/${fileNameToPathMap(fileName).getName}")
|
||||||
LOG.info(" Skipping and writing new file for : " + fileName)
|
LOG.info(" Skipping and writing new file for : " + fileName)
|
||||||
SparkHelpers.skipKeysAndWriteNewFile(commitTime, fs, badFilePath, newFilePath, dupeFixPlan(fileName))
|
SparkHelpers.skipKeysAndWriteNewFile(commitTime, fs, badFilePath, newFilePath, dupeFixPlan(fileName))
|
||||||
fs.delete(badFilePath, false)
|
fs.delete(badFilePath, false)
|
||||||
}}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// 3. Check that there are no duplicates anymore.
|
// 3. Check that there are no duplicates anymore.
|
||||||
val df = sqlContext.read.parquet(s"${repairOutputPath}/*.parquet")
|
val df = sqlContext.read.parquet(s"${repairOutputPath}/*.parquet")
|
||||||
@@ -186,6 +190,7 @@ class DedupeSparkJob (basePath: String,
|
|||||||
LOG.info(s"[FOR REAL!!!] Copying from ${srcPath} to ${dstPath}")
|
LOG.info(s"[FOR REAL!!!] Copying from ${srcPath} to ${dstPath}")
|
||||||
FileUtil.copy(fs, srcPath, fs, dstPath, false, true, fs.getConf)
|
FileUtil.copy(fs, srcPath, fs, dstPath, false, true, fs.getConf)
|
||||||
}
|
}
|
||||||
}}
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,9 +17,9 @@
|
|||||||
package com.uber.hoodie.cli
|
package com.uber.hoodie.cli
|
||||||
|
|
||||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport
|
import com.uber.hoodie.avro.HoodieAvroWriteSupport
|
||||||
import com.uber.hoodie.common.{BloomFilter, HoodieJsonPayload}
|
|
||||||
import com.uber.hoodie.common.model.HoodieRecord
|
import com.uber.hoodie.common.model.HoodieRecord
|
||||||
import com.uber.hoodie.common.util.ParquetUtils
|
import com.uber.hoodie.common.util.ParquetUtils
|
||||||
|
import com.uber.hoodie.common.{BloomFilter, HoodieJsonPayload}
|
||||||
import com.uber.hoodie.config.{HoodieIndexConfig, HoodieStorageConfig}
|
import com.uber.hoodie.config.{HoodieIndexConfig, HoodieStorageConfig}
|
||||||
import com.uber.hoodie.io.storage.{HoodieParquetConfig, HoodieParquetWriter}
|
import com.uber.hoodie.io.storage.{HoodieParquetConfig, HoodieParquetWriter}
|
||||||
import org.apache.avro.Schema
|
import org.apache.avro.Schema
|
||||||
@@ -107,7 +107,7 @@ class SparkHelper(sqlContext: SQLContext, fs: FileSystem) {
|
|||||||
* @param file
|
* @param file
|
||||||
* @param sqlContext
|
* @param sqlContext
|
||||||
*/
|
*/
|
||||||
def getKeyCount(file: String, sqlContext: org.apache.spark.sql.SQLContext) ={
|
def getKeyCount(file: String, sqlContext: org.apache.spark.sql.SQLContext) = {
|
||||||
println(getRowKeyDF(file).collect().size)
|
println(getRowKeyDF(file).collect().size)
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -122,7 +122,7 @@ class SparkHelper(sqlContext: SQLContext, fs: FileSystem) {
|
|||||||
* @param file
|
* @param file
|
||||||
* @return
|
* @return
|
||||||
*/
|
*/
|
||||||
def fileKeysAgainstBF(conf: Configuration, sqlContext: SQLContext, file: String) : Boolean = {
|
def fileKeysAgainstBF(conf: Configuration, sqlContext: SQLContext, file: String): Boolean = {
|
||||||
val bfStr = SparkHelpers.getBloomFilter(file, conf)
|
val bfStr = SparkHelpers.getBloomFilter(file, conf)
|
||||||
val bf = new com.uber.hoodie.common.BloomFilter(bfStr)
|
val bf = new com.uber.hoodie.common.BloomFilter(bfStr)
|
||||||
val foundCount = sqlContext.parquetFile(file)
|
val foundCount = sqlContext.parquetFile(file)
|
||||||
@@ -134,7 +134,7 @@ class SparkHelper(sqlContext: SQLContext, fs: FileSystem) {
|
|||||||
totalCount == foundCount
|
totalCount == foundCount
|
||||||
}
|
}
|
||||||
|
|
||||||
def getDistinctKeyDF(paths: List[String]) : DataFrame = {
|
def getDistinctKeyDF(paths: List[String]): DataFrame = {
|
||||||
sqlContext.read.parquet(paths:_*).select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`").distinct()
|
sqlContext.read.parquet(paths: _*).select(s"`${HoodieRecord.RECORD_KEY_METADATA_FIELD}`").distinct()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -15,180 +15,182 @@
|
|||||||
~ limitations under the License.
|
~ limitations under the License.
|
||||||
-->
|
-->
|
||||||
|
|
||||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
||||||
<parent>
|
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||||
<artifactId>hoodie</artifactId>
|
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||||
<groupId>com.uber.hoodie</groupId>
|
<parent>
|
||||||
<version>0.4.1-SNAPSHOT</version>
|
<artifactId>hoodie</artifactId>
|
||||||
</parent>
|
<groupId>com.uber.hoodie</groupId>
|
||||||
<modelVersion>4.0.0</modelVersion>
|
<version>0.4.1-SNAPSHOT</version>
|
||||||
|
</parent>
|
||||||
|
<modelVersion>4.0.0</modelVersion>
|
||||||
|
|
||||||
<artifactId>hoodie-client</artifactId>
|
<artifactId>hoodie-client</artifactId>
|
||||||
<packaging>jar</packaging>
|
<packaging>jar</packaging>
|
||||||
<build>
|
<build>
|
||||||
<plugins>
|
<plugins>
|
||||||
<plugin>
|
<plugin>
|
||||||
<groupId>org.jacoco</groupId>
|
<groupId>org.jacoco</groupId>
|
||||||
<artifactId>jacoco-maven-plugin</artifactId>
|
<artifactId>jacoco-maven-plugin</artifactId>
|
||||||
</plugin>
|
</plugin>
|
||||||
<plugin>
|
<plugin>
|
||||||
<groupId>org.apache.maven.plugins</groupId>
|
<groupId>org.apache.maven.plugins</groupId>
|
||||||
<artifactId>maven-jar-plugin</artifactId>
|
<artifactId>maven-jar-plugin</artifactId>
|
||||||
<executions>
|
<executions>
|
||||||
<execution>
|
<execution>
|
||||||
<goals>
|
<goals>
|
||||||
<goal>test-jar</goal>
|
<goal>test-jar</goal>
|
||||||
</goals>
|
</goals>
|
||||||
</execution>
|
</execution>
|
||||||
</executions>
|
</executions>
|
||||||
</plugin>
|
</plugin>
|
||||||
<plugin>
|
<plugin>
|
||||||
<groupId>org.apache.rat</groupId>
|
<groupId>org.apache.rat</groupId>
|
||||||
<artifactId>apache-rat-plugin</artifactId>
|
<artifactId>apache-rat-plugin</artifactId>
|
||||||
</plugin>
|
</plugin>
|
||||||
</plugins>
|
</plugins>
|
||||||
|
|
||||||
<resources>
|
<resources>
|
||||||
<resource>
|
<resource>
|
||||||
<directory>src/main/resources</directory>
|
<directory>src/main/resources</directory>
|
||||||
</resource>
|
</resource>
|
||||||
<resource>
|
<resource>
|
||||||
<directory>src/test/resources</directory>
|
<directory>src/test/resources</directory>
|
||||||
</resource>
|
</resource>
|
||||||
</resources>
|
</resources>
|
||||||
</build>
|
</build>
|
||||||
|
|
||||||
<dependencies>
|
<dependencies>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>com.uber.hoodie</groupId>
|
<groupId>com.uber.hoodie</groupId>
|
||||||
<artifactId>hoodie-common</artifactId>
|
<artifactId>hoodie-common</artifactId>
|
||||||
<version>${project.version}</version>
|
<version>${project.version}</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hadoop</groupId>
|
<groupId>org.apache.hadoop</groupId>
|
||||||
<artifactId>hadoop-hdfs</artifactId>
|
<artifactId>hadoop-hdfs</artifactId>
|
||||||
<classifier>tests</classifier>
|
<classifier>tests</classifier>
|
||||||
<!-- Need these exclusions to make sure JavaSparkContext can be setup. https://issues.apache.org/jira/browse/SPARK-1693 -->
|
<!-- Need these exclusions to make sure JavaSparkContext can be setup. https://issues.apache.org/jira/browse/SPARK-1693 -->
|
||||||
<exclusions>
|
<exclusions>
|
||||||
<exclusion>
|
<exclusion>
|
||||||
<groupId>org.mortbay.jetty</groupId>
|
<groupId>org.mortbay.jetty</groupId>
|
||||||
<artifactId>*</artifactId>
|
<artifactId>*</artifactId>
|
||||||
</exclusion>
|
</exclusion>
|
||||||
<exclusion>
|
<exclusion>
|
||||||
<groupId>javax.servlet.jsp</groupId>
|
<groupId>javax.servlet.jsp</groupId>
|
||||||
<artifactId>*</artifactId>
|
<artifactId>*</artifactId>
|
||||||
</exclusion>
|
</exclusion>
|
||||||
<exclusion>
|
<exclusion>
|
||||||
<groupId>javax.servlet</groupId>
|
<groupId>javax.servlet</groupId>
|
||||||
<artifactId>*</artifactId>
|
<artifactId>*</artifactId>
|
||||||
</exclusion>
|
</exclusion>
|
||||||
</exclusions>
|
</exclusions>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hadoop</groupId>
|
<groupId>org.apache.hadoop</groupId>
|
||||||
<artifactId>hadoop-common</artifactId>
|
<artifactId>hadoop-common</artifactId>
|
||||||
<classifier>tests</classifier>
|
<classifier>tests</classifier>
|
||||||
<exclusions>
|
<exclusions>
|
||||||
<exclusion>
|
<exclusion>
|
||||||
<groupId>org.mortbay.jetty</groupId>
|
<groupId>org.mortbay.jetty</groupId>
|
||||||
<artifactId>*</artifactId>
|
<artifactId>*</artifactId>
|
||||||
</exclusion>
|
</exclusion>
|
||||||
<exclusion>
|
<exclusion>
|
||||||
<groupId>javax.servlet.jsp</groupId>
|
<groupId>javax.servlet.jsp</groupId>
|
||||||
<artifactId>*</artifactId>
|
<artifactId>*</artifactId>
|
||||||
</exclusion>
|
</exclusion>
|
||||||
<exclusion>
|
<exclusion>
|
||||||
<groupId>javax.servlet</groupId>
|
<groupId>javax.servlet</groupId>
|
||||||
<artifactId>*</artifactId>
|
<artifactId>*</artifactId>
|
||||||
</exclusion>
|
</exclusion>
|
||||||
</exclusions>
|
</exclusions>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>com.uber.hoodie</groupId>
|
<groupId>com.uber.hoodie</groupId>
|
||||||
<artifactId>hoodie-common</artifactId>
|
<artifactId>hoodie-common</artifactId>
|
||||||
<version>${project.version}</version>
|
<version>${project.version}</version>
|
||||||
<type>test-jar</type>
|
<type>test-jar</type>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>io.dropwizard.metrics</groupId>
|
<groupId>io.dropwizard.metrics</groupId>
|
||||||
<artifactId>metrics-graphite</artifactId>
|
<artifactId>metrics-graphite</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>io.dropwizard.metrics</groupId>
|
<groupId>io.dropwizard.metrics</groupId>
|
||||||
<artifactId>metrics-core</artifactId>
|
<artifactId>metrics-core</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>com.beust</groupId>
|
<groupId>com.beust</groupId>
|
||||||
<artifactId>jcommander</artifactId>
|
<artifactId>jcommander</artifactId>
|
||||||
<version>1.48</version>
|
<version>1.48</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<!-- Parent dependencies -->
|
<!-- Parent dependencies -->
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>log4j</groupId>
|
<groupId>log4j</groupId>
|
||||||
<artifactId>log4j</artifactId>
|
<artifactId>log4j</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hadoop</groupId>
|
<groupId>org.apache.hadoop</groupId>
|
||||||
<artifactId>hadoop-client</artifactId>
|
<artifactId>hadoop-client</artifactId>
|
||||||
<exclusions>
|
<exclusions>
|
||||||
<exclusion>
|
<exclusion>
|
||||||
<groupId>javax.servlet</groupId>
|
<groupId>javax.servlet</groupId>
|
||||||
<artifactId>*</artifactId>
|
<artifactId>*</artifactId>
|
||||||
</exclusion>
|
</exclusion>
|
||||||
</exclusions>
|
</exclusions>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.parquet</groupId>
|
<groupId>org.apache.parquet</groupId>
|
||||||
<artifactId>parquet-avro</artifactId>
|
<artifactId>parquet-avro</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.parquet</groupId>
|
<groupId>org.apache.parquet</groupId>
|
||||||
<artifactId>parquet-hadoop</artifactId>
|
<artifactId>parquet-hadoop</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>com.google.guava</groupId>
|
<groupId>com.google.guava</groupId>
|
||||||
<artifactId>guava</artifactId>
|
<artifactId>guava</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.spark</groupId>
|
<groupId>org.apache.spark</groupId>
|
||||||
<artifactId>spark-core_2.11</artifactId>
|
<artifactId>spark-core_2.11</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.spark</groupId>
|
<groupId>org.apache.spark</groupId>
|
||||||
<artifactId>spark-sql_2.11</artifactId>
|
<artifactId>spark-sql_2.11</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hbase</groupId>
|
<groupId>org.apache.hbase</groupId>
|
||||||
<artifactId>hbase-client</artifactId>
|
<artifactId>hbase-client</artifactId>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.mockito</groupId>
|
<groupId>org.mockito</groupId>
|
||||||
<artifactId>mockito-all</artifactId>
|
<artifactId>mockito-all</artifactId>
|
||||||
<version>1.10.19</version>
|
<version>1.10.19</version>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>com.uber.hoodie</groupId>
|
<groupId>com.uber.hoodie</groupId>
|
||||||
<artifactId>hoodie-hadoop-mr</artifactId>
|
<artifactId>hoodie-hadoop-mr</artifactId>
|
||||||
<version>${project.version}</version>
|
<version>${project.version}</version>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>org.apache.hive</groupId>
|
<groupId>org.apache.hive</groupId>
|
||||||
<artifactId>hive-exec</artifactId>
|
<artifactId>hive-exec</artifactId>
|
||||||
<scope>test</scope>
|
<scope>test</scope>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
</dependencies>
|
</dependencies>
|
||||||
|
|
||||||
</project>
|
</project>
|
||||||
|
|||||||
@@ -17,25 +17,19 @@
|
|||||||
package com.uber.hoodie;
|
package com.uber.hoodie;
|
||||||
|
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
|
|
||||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
|
||||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
|
||||||
import com.uber.hoodie.common.model.HoodieKey;
|
import com.uber.hoodie.common.model.HoodieKey;
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
|
||||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
|
||||||
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
|
|
||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.exception.HoodieException;
|
|
||||||
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
|
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
|
||||||
|
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Set;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
@@ -46,136 +40,126 @@ import org.apache.spark.sql.Dataset;
|
|||||||
import org.apache.spark.sql.Row;
|
import org.apache.spark.sql.Row;
|
||||||
import org.apache.spark.sql.SQLContext;
|
import org.apache.spark.sql.SQLContext;
|
||||||
import org.apache.spark.sql.types.StructType;
|
import org.apache.spark.sql.types.StructType;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.io.Serializable;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Provides an RDD based API for accessing/filtering Hoodie tables, based on keys.
|
* Provides an RDD based API for accessing/filtering Hoodie tables, based on keys.
|
||||||
*
|
|
||||||
*/
|
*/
|
||||||
public class HoodieReadClient implements Serializable {
|
public class HoodieReadClient implements Serializable {
|
||||||
|
|
||||||
private static Logger logger = LogManager.getLogger(HoodieReadClient.class);
|
private static Logger logger = LogManager.getLogger(HoodieReadClient.class);
|
||||||
|
|
||||||
private transient final JavaSparkContext jsc;
|
private transient final JavaSparkContext jsc;
|
||||||
|
|
||||||
private transient final FileSystem fs;
|
private transient final FileSystem fs;
|
||||||
/**
|
/**
|
||||||
* TODO: We need to persist the index type into hoodie.properties and be able to access the
|
* TODO: We need to persist the index type into hoodie.properties and be able to access the index
|
||||||
* index just with a simple basepath pointing to the dataset. Until, then just always assume a
|
* just with a simple basepath pointing to the dataset. Until, then just always assume a
|
||||||
* BloomIndex
|
* BloomIndex
|
||||||
*/
|
*/
|
||||||
private transient final HoodieBloomIndex index;
|
private transient final HoodieBloomIndex index;
|
||||||
private final HoodieTimeline commitTimeline;
|
private final HoodieTimeline commitTimeline;
|
||||||
private HoodieTable hoodieTable;
|
private HoodieTable hoodieTable;
|
||||||
private transient Optional<SQLContext> sqlContextOpt;
|
private transient Optional<SQLContext> sqlContextOpt;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param basePath path to Hoodie dataset
|
* @param basePath path to Hoodie dataset
|
||||||
*/
|
*/
|
||||||
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
|
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
|
||||||
this.jsc = jsc;
|
this.jsc = jsc;
|
||||||
this.fs = FSUtils.getFs();
|
this.fs = FSUtils.getFs();
|
||||||
// Create a Hoodie table which encapsulated the commits and files visible
|
// Create a Hoodie table which encapsulated the commits and files visible
|
||||||
this.hoodieTable = HoodieTable
|
this.hoodieTable = HoodieTable
|
||||||
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
|
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
|
||||||
this.commitTimeline = hoodieTable.getCompletedCompactionCommitTimeline();
|
this.commitTimeline = hoodieTable.getCompletedCompactionCommitTimeline();
|
||||||
this.index =
|
this.index =
|
||||||
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
|
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
|
||||||
this.sqlContextOpt = Optional.absent();
|
this.sqlContextOpt = Optional.absent();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
*
|
||||||
|
* @param jsc
|
||||||
|
* @param basePath
|
||||||
|
* @param sqlContext
|
||||||
|
*/
|
||||||
|
public HoodieReadClient(JavaSparkContext jsc, String basePath, SQLContext sqlContext) {
|
||||||
|
this(jsc, basePath);
|
||||||
|
this.sqlContextOpt = Optional.of(sqlContext);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Adds support for accessing Hoodie built tables from SparkSQL, as you normally would.
|
||||||
|
*
|
||||||
|
* @return SparkConf object to be used to construct the SparkContext by caller
|
||||||
|
*/
|
||||||
|
public static SparkConf addHoodieSupport(SparkConf conf) {
|
||||||
|
conf.set("spark.sql.hive.convertMetastoreParquet", "false");
|
||||||
|
return conf;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void assertSqlContext() {
|
||||||
|
if (!sqlContextOpt.isPresent()) {
|
||||||
|
throw new IllegalStateException(
|
||||||
|
"SQLContext must be set, when performing dataframe operations");
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
* Given a bunch of hoodie keys, fetches all the individual records out as a data frame
|
||||||
* @param jsc
|
*
|
||||||
* @param basePath
|
* @return a dataframe
|
||||||
* @param sqlContext
|
*/
|
||||||
*/
|
public Dataset<Row> read(JavaRDD<HoodieKey> hoodieKeys, int parallelism)
|
||||||
public HoodieReadClient(JavaSparkContext jsc, String basePath, SQLContext sqlContext) {
|
throws Exception {
|
||||||
this(jsc, basePath);
|
|
||||||
this.sqlContextOpt = Optional.of(sqlContext);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
assertSqlContext();
|
||||||
* Adds support for accessing Hoodie built tables from SparkSQL, as you normally would.
|
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD =
|
||||||
*
|
index.fetchRecordLocation(hoodieKeys, hoodieTable);
|
||||||
* @return SparkConf object to be used to construct the SparkContext by caller
|
List<String> paths = keyToFileRDD
|
||||||
*/
|
.filter(keyFileTuple -> keyFileTuple._2().isPresent())
|
||||||
public static SparkConf addHoodieSupport(SparkConf conf) {
|
.map(keyFileTuple -> keyFileTuple._2().get())
|
||||||
conf.set("spark.sql.hive.convertMetastoreParquet", "false");
|
.collect();
|
||||||
return conf;
|
|
||||||
}
|
|
||||||
|
|
||||||
private void assertSqlContext() {
|
// record locations might be same for multiple keys, so need a unique list
|
||||||
if (!sqlContextOpt.isPresent()) {
|
Set<String> uniquePaths = new HashSet<>(paths);
|
||||||
throw new IllegalStateException("SQLContext must be set, when performing dataframe operations");
|
Dataset<Row> originalDF = sqlContextOpt.get().read()
|
||||||
}
|
.parquet(uniquePaths.toArray(new String[uniquePaths.size()]));
|
||||||
}
|
StructType schema = originalDF.schema();
|
||||||
|
JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD()
|
||||||
|
.mapToPair(row -> {
|
||||||
|
HoodieKey key = new HoodieKey(
|
||||||
|
row.<String>getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
|
||||||
|
row.<String>getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
|
||||||
|
return new Tuple2<>(key, row);
|
||||||
|
});
|
||||||
|
|
||||||
/**
|
// Now, we need to further filter out, for only rows that match the supplied hoodie keys
|
||||||
* Given a bunch of hoodie keys, fetches all the individual records out as a data frame
|
JavaRDD<Row> rowRDD = keyRowRDD.join(keyToFileRDD, parallelism)
|
||||||
*
|
.map(tuple -> tuple._2()._1());
|
||||||
* @return a dataframe
|
|
||||||
*/
|
|
||||||
public Dataset<Row> read(JavaRDD<HoodieKey> hoodieKeys, int parallelism)
|
|
||||||
throws Exception {
|
|
||||||
|
|
||||||
assertSqlContext();
|
return sqlContextOpt.get().createDataFrame(rowRDD, schema);
|
||||||
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD =
|
}
|
||||||
index.fetchRecordLocation(hoodieKeys, hoodieTable);
|
|
||||||
List<String> paths = keyToFileRDD
|
|
||||||
.filter(keyFileTuple -> keyFileTuple._2().isPresent())
|
|
||||||
.map(keyFileTuple -> keyFileTuple._2().get())
|
|
||||||
.collect();
|
|
||||||
|
|
||||||
// record locations might be same for multiple keys, so need a unique list
|
/**
|
||||||
Set<String> uniquePaths = new HashSet<>(paths);
|
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Optional[FullFilePath]]
|
||||||
Dataset<Row> originalDF = sqlContextOpt.get().read()
|
* If the optional FullFilePath value is not present, then the key is not found. If the
|
||||||
.parquet(uniquePaths.toArray(new String[uniquePaths.size()]));
|
* FullFilePath value is present, it is the path component (without scheme) of the URI underlying
|
||||||
StructType schema = originalDF.schema();
|
* file
|
||||||
JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD()
|
*/
|
||||||
.mapToPair(row -> {
|
public JavaPairRDD<HoodieKey, Optional<String>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
|
||||||
HoodieKey key = new HoodieKey(
|
return index.fetchRecordLocation(hoodieKeys, hoodieTable);
|
||||||
row.<String>getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
|
}
|
||||||
row.<String>getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
|
|
||||||
return new Tuple2<>(key, row);
|
|
||||||
});
|
|
||||||
|
|
||||||
// Now, we need to further filter out, for only rows that match the supplied hoodie keys
|
/**
|
||||||
JavaRDD<Row> rowRDD = keyRowRDD.join(keyToFileRDD, parallelism)
|
* Filter out HoodieRecords that already exists in the output folder. This is useful in
|
||||||
.map(tuple -> tuple._2()._1());
|
* deduplication.
|
||||||
|
*
|
||||||
return sqlContextOpt.get().createDataFrame(rowRDD, schema);
|
* @param hoodieRecords Input RDD of Hoodie records.
|
||||||
}
|
* @return A subset of hoodieRecords RDD, with existing records filtered out.
|
||||||
|
*/
|
||||||
/**
|
public JavaRDD<HoodieRecord> filterExists(JavaRDD<HoodieRecord> hoodieRecords) {
|
||||||
* Checks if the given [Keys] exists in the hoodie table and returns [Key,
|
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, hoodieTable);
|
||||||
* Optional[FullFilePath]] If the optional FullFilePath value is not present, then the key is
|
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
|
||||||
* not found. If the FullFilePath value is present, it is the path component (without scheme) of
|
}
|
||||||
* the URI underlying file
|
|
||||||
*/
|
|
||||||
public JavaPairRDD<HoodieKey, Optional<String>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
|
|
||||||
return index.fetchRecordLocation(hoodieKeys, hoodieTable);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Filter out HoodieRecords that already exists in the output folder. This is useful in
|
|
||||||
* deduplication.
|
|
||||||
*
|
|
||||||
* @param hoodieRecords Input RDD of Hoodie records.
|
|
||||||
* @return A subset of hoodieRecords RDD, with existing records filtered out.
|
|
||||||
*/
|
|
||||||
public JavaRDD<HoodieRecord> filterExists(JavaRDD<HoodieRecord> hoodieRecords) {
|
|
||||||
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, hoodieTable);
|
|
||||||
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -50,10 +50,21 @@ import com.uber.hoodie.func.BulkInsertMapFunction;
|
|||||||
import com.uber.hoodie.index.HoodieIndex;
|
import com.uber.hoodie.index.HoodieIndex;
|
||||||
import com.uber.hoodie.io.HoodieCommitArchiveLog;
|
import com.uber.hoodie.io.HoodieCommitArchiveLog;
|
||||||
import com.uber.hoodie.metrics.HoodieMetrics;
|
import com.uber.hoodie.metrics.HoodieMetrics;
|
||||||
import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner;
|
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner;
|
||||||
import com.uber.hoodie.table.WorkloadProfile;
|
import com.uber.hoodie.table.WorkloadProfile;
|
||||||
import com.uber.hoodie.table.WorkloadStat;
|
import com.uber.hoodie.table.WorkloadStat;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
|
import java.text.ParseException;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Date;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
@@ -66,25 +77,12 @@ import org.apache.spark.storage.StorageLevel;
|
|||||||
import scala.Option;
|
import scala.Option;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.io.Serializable;
|
|
||||||
import java.nio.charset.StandardCharsets;
|
|
||||||
import java.text.ParseException;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.Date;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then
|
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient
|
||||||
* perform efficient mutations on a HDFS dataset [upsert()]
|
* 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 {
|
public class HoodieWriteClient<T extends HoodieRecordPayload> implements Serializable {
|
||||||
|
|
||||||
@@ -102,7 +100,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
* @param clientConfig
|
* @param clientConfig
|
||||||
* @throws Exception
|
* @throws Exception
|
||||||
*/
|
*/
|
||||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) throws Exception {
|
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig)
|
||||||
|
throws Exception {
|
||||||
this(jsc, clientConfig, false);
|
this(jsc, clientConfig, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -111,7 +110,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
* @param clientConfig
|
* @param clientConfig
|
||||||
* @param rollbackInFlight
|
* @param rollbackInFlight
|
||||||
*/
|
*/
|
||||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackInFlight) {
|
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
|
||||||
|
boolean rollbackInFlight) {
|
||||||
this.fs = FSUtils.getFs();
|
this.fs = FSUtils.getFs();
|
||||||
this.jsc = jsc;
|
this.jsc = jsc;
|
||||||
this.config = clientConfig;
|
this.config = clientConfig;
|
||||||
@@ -121,7 +121,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
|
|
||||||
if (rollbackInFlight) {
|
if (rollbackInFlight) {
|
||||||
rollbackInflightCommits();
|
rollbackInflightCommits();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -163,17 +163,17 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
throw (HoodieUpsertException) e;
|
throw (HoodieUpsertException) e;
|
||||||
}
|
}
|
||||||
throw new HoodieUpsertException("Failed to upsert for commit time " + commitTime, e);
|
throw new HoodieUpsertException("Failed to upsert for commit time " + commitTime, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Inserts the given HoodieRecords, into the table. This API is intended to be used for normal
|
* Inserts the given HoodieRecords, into the table. This API is intended to be used for normal
|
||||||
* writes.
|
* writes.
|
||||||
*
|
*
|
||||||
* This implementation skips the index check and is able to leverage benefits such as
|
* This implementation skips the index check and is able to leverage benefits such as small file
|
||||||
* small file handling/blocking alignment, as with upsert(), by profiling the workload
|
* handling/blocking alignment, as with upsert(), by profiling the workload
|
||||||
*
|
*
|
||||||
* @param records HoodieRecords to insert
|
* @param records HoodieRecords to insert
|
||||||
* @param commitTime Commit Time handle
|
* @param commitTime Commit Time handle
|
||||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||||
*/
|
*/
|
||||||
@@ -194,7 +194,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
throw e;
|
throw e;
|
||||||
}
|
}
|
||||||
throw new HoodieInsertException("Failed to insert for commit time " + commitTime, e);
|
throw new HoodieInsertException("Failed to insert for commit time " + commitTime, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -206,11 +206,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
* attempts to control the numbers of files with less memory compared to the {@link
|
* attempts to control the numbers of files with less memory compared to the {@link
|
||||||
* HoodieWriteClient#insert(JavaRDD, String)}
|
* HoodieWriteClient#insert(JavaRDD, String)}
|
||||||
*
|
*
|
||||||
* @param records HoodieRecords to insert
|
* @param records HoodieRecords to insert
|
||||||
* @param commitTime Commit Time handle
|
* @param commitTime Commit Time handle
|
||||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||||
*/
|
*/
|
||||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
|
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records,
|
||||||
|
final String commitTime) {
|
||||||
return bulkInsert(records, commitTime, Option.empty());
|
return bulkInsert(records, commitTime, Option.empty());
|
||||||
}
|
}
|
||||||
|
|
||||||
@@ -221,16 +222,18 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
*
|
*
|
||||||
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and
|
* 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
|
* attempts to control the numbers of files with less memory compared to the {@link
|
||||||
* HoodieWriteClient#insert(JavaRDD, String)}. Optionally it allows users to specify their own partitioner. If
|
* HoodieWriteClient#insert(JavaRDD, String)}. Optionally it allows users to specify their own
|
||||||
* specified then it will be used for repartitioning records. See {@link UserDefinedBulkInsertPartitioner}.
|
* partitioner. If specified then it will be used for repartitioning records. See {@link
|
||||||
|
* UserDefinedBulkInsertPartitioner}.
|
||||||
*
|
*
|
||||||
* @param records HoodieRecords to insert
|
* @param records HoodieRecords to insert
|
||||||
* @param commitTime Commit Time handle
|
* @param commitTime Commit Time handle
|
||||||
* @param bulkInsertPartitioner If specified then it will be used to partition input records before they are
|
* @param bulkInsertPartitioner If specified then it will be used to partition input records
|
||||||
* inserted into hoodie.
|
* before they are inserted into hoodie.
|
||||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||||
*/
|
*/
|
||||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String commitTime,
|
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records,
|
||||||
|
final String commitTime,
|
||||||
Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
|
Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
|
||||||
writeContext = metrics.getCommitCtx();
|
writeContext = metrics.getCommitCtx();
|
||||||
// Create a Hoodie table which encapsulated the commits and files visible
|
// Create a Hoodie table which encapsulated the commits and files visible
|
||||||
@@ -240,7 +243,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
try {
|
try {
|
||||||
// De-dupe/merge if needed
|
// De-dupe/merge if needed
|
||||||
JavaRDD<HoodieRecord<T>> dedupedRecords =
|
JavaRDD<HoodieRecord<T>> dedupedRecords =
|
||||||
combineOnCondition(config.shouldCombineBeforeInsert(), records, config.getInsertShuffleParallelism());
|
combineOnCondition(config.shouldCombineBeforeInsert(), records,
|
||||||
|
config.getInsertShuffleParallelism());
|
||||||
|
|
||||||
final JavaRDD<HoodieRecord<T>> repartitionedRecords;
|
final JavaRDD<HoodieRecord<T>> repartitionedRecords;
|
||||||
if (bulkInsertPartitioner.isDefined()) {
|
if (bulkInsertPartitioner.isDefined()) {
|
||||||
@@ -259,20 +263,22 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
}, true, config.getBulkInsertShuffleParallelism());
|
}, true, config.getBulkInsertShuffleParallelism());
|
||||||
}
|
}
|
||||||
JavaRDD<WriteStatus> writeStatusRDD = repartitionedRecords
|
JavaRDD<WriteStatus> writeStatusRDD = repartitionedRecords
|
||||||
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(commitTime, config, table), true)
|
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(commitTime, config, table),
|
||||||
.flatMap(writeStatuses -> writeStatuses.iterator());
|
true)
|
||||||
|
.flatMap(writeStatuses -> writeStatuses.iterator());
|
||||||
|
|
||||||
return updateIndexAndCommitIfNeeded(writeStatusRDD, table, commitTime);
|
return updateIndexAndCommitIfNeeded(writeStatusRDD, table, commitTime);
|
||||||
} catch (Throwable e) {
|
} catch (Throwable e) {
|
||||||
if (e instanceof HoodieInsertException) {
|
if (e instanceof HoodieInsertException) {
|
||||||
throw e;
|
throw e;
|
||||||
}
|
}
|
||||||
throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime, e);
|
throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime,
|
||||||
}
|
e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void commitOnAutoCommit(String commitTime, JavaRDD<WriteStatus> resultRDD) {
|
private void commitOnAutoCommit(String commitTime, JavaRDD<WriteStatus> resultRDD) {
|
||||||
if(config.shouldAutoCommit()) {
|
if (config.shouldAutoCommit()) {
|
||||||
logger.info("Auto commit enabled: Committing " + commitTime);
|
logger.info("Auto commit enabled: Committing " + commitTime);
|
||||||
boolean commitResult = commit(commitTime, resultRDD);
|
boolean commitResult = commit(commitTime, resultRDD);
|
||||||
if (!commitResult) {
|
if (!commitResult) {
|
||||||
@@ -280,30 +286,28 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
logger.info("Auto commit disabled for " + commitTime);
|
logger.info("Auto commit disabled for " + commitTime);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private JavaRDD<HoodieRecord<T>> combineOnCondition(boolean condition,
|
private JavaRDD<HoodieRecord<T>> combineOnCondition(boolean condition,
|
||||||
JavaRDD<HoodieRecord<T>> records,
|
JavaRDD<HoodieRecord<T>> records,
|
||||||
int parallelism) {
|
int parallelism) {
|
||||||
if(condition) {
|
if (condition) {
|
||||||
return deduplicateRecords(records, parallelism);
|
return deduplicateRecords(records, parallelism);
|
||||||
}
|
}
|
||||||
return records;
|
return records;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
* Save the workload profile in an intermediate file (here re-using commit files) This is useful
|
||||||
* Save the workload profile in an intermediate file (here re-using commit files)
|
* when performing rollback for MOR datasets. Only updates are recorded in the workload profile
|
||||||
* This is useful when performing rollback for MOR datasets. Only updates are recorded
|
* metadata since updates to log blocks are unknown across batches Inserts (which are new parquet
|
||||||
* in the workload profile metadata since updates to log blocks are unknown across batches
|
* files) are rolled back based on commit time. // TODO : Create a new WorkloadProfile metadata
|
||||||
* Inserts (which are new parquet files) are rolled back based on commit time.
|
* file instead of using HoodieCommitMetadata
|
||||||
* // TODO : Create a new WorkloadProfile metadata file instead of using HoodieCommitMetadata
|
|
||||||
* @param profile
|
|
||||||
* @param commitTime
|
|
||||||
* @throws HoodieCommitException
|
|
||||||
*/
|
*/
|
||||||
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, HoodieTable<T> table, String commitTime) throws HoodieCommitException {
|
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile,
|
||||||
|
HoodieTable<T> table,
|
||||||
|
String commitTime) throws HoodieCommitException {
|
||||||
try {
|
try {
|
||||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||||
profile.getPartitionPaths().stream().forEach(path -> {
|
profile.getPartitionPaths().stream().forEach(path -> {
|
||||||
@@ -319,16 +323,17 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||||
Optional<HoodieInstant> instant = activeTimeline.filterInflights().lastInstant();
|
Optional<HoodieInstant> instant = activeTimeline.filterInflights().lastInstant();
|
||||||
activeTimeline.saveToInflight(instant.get(),
|
activeTimeline.saveToInflight(instant.get(),
|
||||||
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||||
} catch(IOException io) {
|
} catch (IOException io) {
|
||||||
throw new HoodieCommitException("Failed to commit " + commitTime + " unable to save inflight metadata ", io);
|
throw new HoodieCommitException(
|
||||||
}
|
"Failed to commit " + commitTime + " unable to save inflight metadata ", io);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private JavaRDD<WriteStatus> upsertRecordsInternal(JavaRDD<HoodieRecord<T>> preppedRecords,
|
private JavaRDD<WriteStatus> upsertRecordsInternal(JavaRDD<HoodieRecord<T>> preppedRecords,
|
||||||
String commitTime,
|
String commitTime,
|
||||||
HoodieTable<T> hoodieTable,
|
HoodieTable<T> hoodieTable,
|
||||||
final boolean isUpsert) {
|
final boolean isUpsert) {
|
||||||
|
|
||||||
// Cache the tagged records, so we don't end up computing both
|
// Cache the tagged records, so we don't end up computing both
|
||||||
preppedRecords.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
preppedRecords.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
||||||
@@ -344,29 +349,31 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
final Partitioner partitioner = getPartitioner(hoodieTable, isUpsert, profile);
|
final Partitioner partitioner = getPartitioner(hoodieTable, isUpsert, profile);
|
||||||
JavaRDD<HoodieRecord<T>> partitionedRecords = partition(preppedRecords, partitioner);
|
JavaRDD<HoodieRecord<T>> partitionedRecords = partition(preppedRecords, partitioner);
|
||||||
JavaRDD<WriteStatus> writeStatusRDD = partitionedRecords
|
JavaRDD<WriteStatus> writeStatusRDD = partitionedRecords
|
||||||
.mapPartitionsWithIndex((partition, recordItr) -> {
|
.mapPartitionsWithIndex((partition, recordItr) -> {
|
||||||
if (isUpsert) {
|
if (isUpsert) {
|
||||||
return hoodieTable
|
return hoodieTable
|
||||||
.handleUpsertPartition(commitTime, partition, recordItr, partitioner);
|
.handleUpsertPartition(commitTime, partition, recordItr, partitioner);
|
||||||
} else {
|
} else {
|
||||||
return hoodieTable
|
return hoodieTable
|
||||||
.handleInsertPartition(commitTime, partition, recordItr, partitioner);
|
.handleInsertPartition(commitTime, partition, recordItr, partitioner);
|
||||||
}
|
}
|
||||||
}, true)
|
}, true)
|
||||||
.flatMap(writeStatuses -> writeStatuses.iterator());
|
.flatMap(writeStatuses -> writeStatuses.iterator());
|
||||||
|
|
||||||
return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime);
|
return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime);
|
||||||
}
|
}
|
||||||
|
|
||||||
private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, WorkloadProfile profile) {
|
private Partitioner getPartitioner(HoodieTable table, boolean isUpsert,
|
||||||
|
WorkloadProfile profile) {
|
||||||
if (isUpsert) {
|
if (isUpsert) {
|
||||||
return table.getUpsertPartitioner(profile);
|
return table.getUpsertPartitioner(profile);
|
||||||
} else {
|
} else {
|
||||||
return table.getInsertPartitioner(profile);
|
return table.getInsertPartitioner(profile);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private JavaRDD<WriteStatus> updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD, HoodieTable<T> table, String commitTime) {
|
private JavaRDD<WriteStatus> updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD,
|
||||||
|
HoodieTable<T> table, String commitTime) {
|
||||||
// Update the index back
|
// Update the index back
|
||||||
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, table);
|
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, table);
|
||||||
// Trigger the insert and collect statuses
|
// Trigger the insert and collect statuses
|
||||||
@@ -375,12 +382,15 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
return statuses;
|
return statuses;
|
||||||
}
|
}
|
||||||
|
|
||||||
private JavaRDD<HoodieRecord<T>> partition(JavaRDD<HoodieRecord<T>> dedupedRecords, Partitioner partitioner) {
|
private JavaRDD<HoodieRecord<T>> partition(JavaRDD<HoodieRecord<T>> dedupedRecords,
|
||||||
|
Partitioner partitioner) {
|
||||||
return dedupedRecords
|
return dedupedRecords
|
||||||
.mapToPair(record ->
|
.mapToPair(record ->
|
||||||
new Tuple2<>(new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), record))
|
new Tuple2<>(
|
||||||
.partitionBy(partitioner)
|
new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())),
|
||||||
.map(tuple -> tuple._2());
|
record))
|
||||||
|
.partitionBy(partitioner)
|
||||||
|
.map(tuple -> tuple._2());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -394,8 +404,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
* Commit changes performed at the given commitTime marker
|
* Commit changes performed at the given commitTime marker
|
||||||
*/
|
*/
|
||||||
public boolean commit(String commitTime,
|
public boolean commit(String commitTime,
|
||||||
JavaRDD<WriteStatus> writeStatuses,
|
JavaRDD<WriteStatus> writeStatuses,
|
||||||
Optional<HashMap<String, String>> extraMetadata) {
|
Optional<HashMap<String, String>> extraMetadata) {
|
||||||
|
|
||||||
logger.info("Commiting " + commitTime);
|
logger.info("Commiting " + commitTime);
|
||||||
// Create a Hoodie table which encapsulated the commits and files visible
|
// Create a Hoodie table which encapsulated the commits and files visible
|
||||||
@@ -405,9 +415,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||||
|
|
||||||
List<Tuple2<String, HoodieWriteStat>> stats = writeStatuses
|
List<Tuple2<String, HoodieWriteStat>> stats = writeStatuses
|
||||||
.mapToPair((PairFunction<WriteStatus, String, HoodieWriteStat>) writeStatus ->
|
.mapToPair((PairFunction<WriteStatus, String, HoodieWriteStat>) writeStatus ->
|
||||||
new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat()))
|
new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat()))
|
||||||
.collect();
|
.collect();
|
||||||
|
|
||||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||||
for (Tuple2<String, HoodieWriteStat> stat : stats) {
|
for (Tuple2<String, HoodieWriteStat> stat : stats) {
|
||||||
@@ -438,7 +448,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
|
|
||||||
// We cannot have unbounded commit files. Archive commits if we have to archive
|
// We cannot have unbounded commit files. Archive commits if we have to archive
|
||||||
archiveLog.archiveIfRequired();
|
archiveLog.archiveIfRequired();
|
||||||
if(config.isAutoClean()) {
|
if (config.isAutoClean()) {
|
||||||
// Call clean to cleanup if there is anything to cleanup after the commit,
|
// Call clean to cleanup if there is anything to cleanup after the commit,
|
||||||
logger.info("Auto cleaning is enabled. Running cleaner now");
|
logger.info("Auto cleaning is enabled. Running cleaner now");
|
||||||
clean(commitTime);
|
clean(commitTime);
|
||||||
@@ -465,12 +475,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime
|
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime will
|
||||||
* will be referenced in the savepoint and will never be cleaned. The savepointed commit
|
* be referenced in the savepoint and will never be cleaned. The savepointed commit will never be
|
||||||
* will never be rolledback or archived.
|
* rolledback or archived.
|
||||||
*
|
*
|
||||||
* This gives an option to rollback the state to the savepoint anytime.
|
* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be
|
||||||
* Savepoint needs to be manually created and deleted.
|
* manually created and deleted.
|
||||||
*
|
*
|
||||||
* Savepoint should be on a commit that could not have been cleaned.
|
* Savepoint should be on a commit that could not have been cleaned.
|
||||||
*
|
*
|
||||||
@@ -491,12 +501,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime
|
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime will
|
||||||
* will be referenced in the savepoint and will never be cleaned. The savepointed commit
|
* be referenced in the savepoint and will never be cleaned. The savepointed commit will never be
|
||||||
* will never be rolledback or archived.
|
* rolledback or archived.
|
||||||
*
|
*
|
||||||
* This gives an option to rollback the state to the savepoint anytime.
|
* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be
|
||||||
* Savepoint needs to be manually created and deleted.
|
* manually created and deleted.
|
||||||
*
|
*
|
||||||
* Savepoint should be on a commit that could not have been cleaned.
|
* Savepoint should be on a commit that could not have been cleaned.
|
||||||
*
|
*
|
||||||
@@ -510,9 +520,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||||
Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
|
Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
|
||||||
|
|
||||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION,
|
||||||
if(!table.getCompletedCommitTimeline().containsInstant(commitInstant)) {
|
commitTime);
|
||||||
throw new HoodieSavepointException("Could not savepoint non-existing commit " + commitInstant);
|
if (!table.getCompletedCommitTimeline().containsInstant(commitInstant)) {
|
||||||
|
throw new HoodieSavepointException(
|
||||||
|
"Could not savepoint non-existing commit " + commitInstant);
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
@@ -534,7 +546,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
+ lastCommitRetained);
|
+ lastCommitRetained);
|
||||||
|
|
||||||
Map<String, List<String>> latestFilesMap = jsc.parallelize(
|
Map<String, List<String>> latestFilesMap = jsc.parallelize(
|
||||||
FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()))
|
FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath(),
|
||||||
|
config.shouldAssumeDatePartitioning()))
|
||||||
.mapToPair((PairFunction<String, String, List<String>>) partitionPath -> {
|
.mapToPair((PairFunction<String, String, List<String>>) partitionPath -> {
|
||||||
// Scan all partitions files with this commit time
|
// Scan all partitions files with this commit time
|
||||||
logger.info("Collecting latest files in partition path " + partitionPath);
|
logger.info("Collecting latest files in partition path " + partitionPath);
|
||||||
@@ -555,12 +568,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
return true;
|
return true;
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new HoodieSavepointException("Failed to savepoint " + commitTime, e);
|
throw new HoodieSavepointException("Failed to savepoint " + commitTime, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Delete a savepoint that was created. Once the savepoint is deleted, the commit can be rolledback
|
* Delete a savepoint that was created. Once the savepoint is deleted, the commit can be
|
||||||
* and cleaner may clean up data files.
|
* rolledback and cleaner may clean up data files.
|
||||||
*
|
*
|
||||||
* @param savepointTime - delete the savepoint
|
* @param savepointTime - delete the savepoint
|
||||||
* @return true if the savepoint was deleted successfully
|
* @return true if the savepoint was deleted successfully
|
||||||
@@ -586,9 +599,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Rollback the state to the savepoint.
|
* Rollback the state to the savepoint. WARNING: This rollsback recent commits and deleted data
|
||||||
* WARNING: This rollsback recent commits and deleted data files. Queries accessing the files
|
* files. Queries accessing the files will mostly fail. This should be done during a downtime.
|
||||||
* will mostly fail. This should be done during a downtime.
|
|
||||||
*
|
*
|
||||||
* @param savepointTime - savepoint time to rollback to
|
* @param savepointTime - savepoint time to rollback to
|
||||||
* @return true if the savepoint was rollecback to successfully
|
* @return true if the savepoint was rollecback to successfully
|
||||||
@@ -616,7 +628,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
|
|
||||||
// Make sure the rollback was successful
|
// Make sure the rollback was successful
|
||||||
Optional<HoodieInstant> lastInstant =
|
Optional<HoodieInstant> lastInstant =
|
||||||
activeTimeline.reload().getCommitsAndCompactionsTimeline().filterCompletedInstants().lastInstant();
|
activeTimeline.reload().getCommitsAndCompactionsTimeline().filterCompletedInstants()
|
||||||
|
.lastInstant();
|
||||||
Preconditions.checkArgument(lastInstant.isPresent());
|
Preconditions.checkArgument(lastInstant.isPresent());
|
||||||
Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime),
|
Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime),
|
||||||
savepointTime + "is not the last commit after rolling back " + commitsToRollback
|
savepointTime + "is not the last commit after rolling back " + commitsToRollback
|
||||||
@@ -625,12 +638,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Rollback the (inflight/committed) record changes with the given commit time.
|
* Rollback the (inflight/committed) record changes with the given commit time. Three steps: (1)
|
||||||
* Three steps:
|
* Atomically unpublish this commit (2) clean indexing data, (3) clean new generated parquet
|
||||||
* (1) Atomically unpublish this commit
|
* files. (4) Finally delete .commit or .inflight file,
|
||||||
* (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 {
|
public boolean rollback(final String commitTime) throws HoodieRollbackException {
|
||||||
rollback(Lists.newArrayList(commitTime));
|
rollback(Lists.newArrayList(commitTime));
|
||||||
@@ -638,7 +648,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
}
|
}
|
||||||
|
|
||||||
private void rollback(List<String> commits) {
|
private void rollback(List<String> commits) {
|
||||||
if(commits.isEmpty()) {
|
if (commits.isEmpty()) {
|
||||||
logger.info("List of commits to rollback is empty");
|
logger.info("List of commits to rollback is empty");
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
@@ -702,7 +712,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
Optional<Long> durationInMs = Optional.empty();
|
Optional<Long> durationInMs = Optional.empty();
|
||||||
if (context != null) {
|
if (context != null) {
|
||||||
durationInMs = Optional.of(metrics.getDurationInMs(context.stop()));
|
durationInMs = Optional.of(metrics.getDurationInMs(context.stop()));
|
||||||
Long numFilesDeleted = stats.stream().mapToLong(stat -> stat.getSuccessDeleteFiles().size()).sum();
|
Long numFilesDeleted = stats.stream()
|
||||||
|
.mapToLong(stat -> stat.getSuccessDeleteFiles().size())
|
||||||
|
.sum();
|
||||||
metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted);
|
metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted);
|
||||||
}
|
}
|
||||||
HoodieRollbackMetadata rollbackMetadata =
|
HoodieRollbackMetadata rollbackMetadata =
|
||||||
@@ -722,7 +734,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new HoodieRollbackException("Failed to rollback " +
|
throw new HoodieRollbackException("Failed to rollback " +
|
||||||
config.getBasePath() + " commits " + commits, e);
|
config.getBasePath() + " commits " + commits, e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -733,9 +745,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Clean up any stale/old files/data lying around (either on file storage or index storage)
|
* Clean up any stale/old files/data lying around (either on file storage or index storage) based
|
||||||
* based on the configurations and CleaningPolicy used. (typically files that no longer can be used
|
* on the configurations and CleaningPolicy used. (typically files that no longer can be used by a
|
||||||
* by a running query can be cleaned)
|
* running query can be cleaned)
|
||||||
*/
|
*/
|
||||||
public void clean() throws HoodieIOException {
|
public void clean() throws HoodieIOException {
|
||||||
String startCleanTime = HoodieActiveTimeline.createNewCommitTime();
|
String startCleanTime = HoodieActiveTimeline.createNewCommitTime();
|
||||||
@@ -743,11 +755,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Clean up any stale/old files/data lying around (either on file storage or index storage)
|
* Clean up any stale/old files/data lying around (either on file storage or index storage) based
|
||||||
* based on the configurations and CleaningPolicy used. (typically files that no longer can be used
|
* on the configurations and CleaningPolicy used. (typically files that no longer can be used by a
|
||||||
* by a running query can be cleaned)
|
* running query can be cleaned)
|
||||||
*/
|
*/
|
||||||
private void clean(String startCleanTime) throws HoodieIOException {
|
private void clean(String startCleanTime) throws HoodieIOException {
|
||||||
try {
|
try {
|
||||||
logger.info("Cleaner started");
|
logger.info("Cleaner started");
|
||||||
final Timer.Context context = metrics.getCleanCtx();
|
final Timer.Context context = metrics.getCleanCtx();
|
||||||
@@ -788,7 +800,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
}
|
}
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new HoodieIOException("Failed to clean up after commit", e);
|
throw new HoodieIOException("Failed to clean up after commit", e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -811,30 +823,30 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
|||||||
}
|
}
|
||||||
|
|
||||||
public static SparkConf registerClasses(SparkConf conf) {
|
public static SparkConf registerClasses(SparkConf conf) {
|
||||||
conf.registerKryoClasses(new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
|
conf.registerKryoClasses(
|
||||||
|
new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
|
||||||
return conf;
|
return conf;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deduplicate Hoodie records, using the given deduplication funciton.
|
* Deduplicate Hoodie records, using the given deduplication funciton.
|
||||||
*/
|
*/
|
||||||
private JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records, int parallelism) {
|
private JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records,
|
||||||
|
int parallelism) {
|
||||||
return records
|
return records
|
||||||
.mapToPair(record -> new Tuple2<>(record.getKey(), record))
|
.mapToPair(record -> new Tuple2<>(record.getKey(), record))
|
||||||
.reduceByKey((rec1, rec2) -> {
|
.reduceByKey((rec1, rec2) -> {
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
T reducedData = (T) rec1.getData().preCombine(rec2.getData());
|
T reducedData = (T) rec1.getData().preCombine(rec2.getData());
|
||||||
// we cannot allow the user to change the key or partitionPath, since that will affect everything
|
// we cannot allow the user to change the key or partitionPath, since that will affect everything
|
||||||
// so pick it from one of the records.
|
// so pick it from one of the records.
|
||||||
return new HoodieRecord<T>(rec1.getKey(), reducedData);
|
return new HoodieRecord<T>(rec1.getKey(), reducedData);
|
||||||
}, parallelism)
|
}, parallelism)
|
||||||
.map(recordTuple -> recordTuple._2());
|
.map(recordTuple -> recordTuple._2());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cleanup all inflight commits
|
* Cleanup all inflight commits
|
||||||
*
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
*/
|
||||||
private void rollbackInflightCommits() {
|
private void rollbackInflightCommits() {
|
||||||
HoodieTable<T> table = HoodieTable
|
HoodieTable<T> table = HoodieTable
|
||||||
|
|||||||
@@ -19,7 +19,6 @@ package com.uber.hoodie;
|
|||||||
import com.uber.hoodie.common.model.HoodieKey;
|
import com.uber.hoodie.common.model.HoodieKey;
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
@@ -32,124 +31,130 @@ import java.util.Optional;
|
|||||||
*/
|
*/
|
||||||
public class WriteStatus implements Serializable {
|
public class WriteStatus implements Serializable {
|
||||||
|
|
||||||
private final HashMap<HoodieKey, Throwable> errors = new HashMap<>();
|
private final HashMap<HoodieKey, Throwable> errors = new HashMap<>();
|
||||||
|
|
||||||
private final List<HoodieRecord> writtenRecords = new ArrayList<>();
|
private final List<HoodieRecord> writtenRecords = new ArrayList<>();
|
||||||
|
|
||||||
private final List<HoodieRecord> failedRecords = new ArrayList<>();
|
private final List<HoodieRecord> failedRecords = new ArrayList<>();
|
||||||
|
|
||||||
private Throwable globalError = null;
|
private Throwable globalError = null;
|
||||||
|
|
||||||
private String fileId = null;
|
private String fileId = null;
|
||||||
|
|
||||||
private String partitionPath = null;
|
private String partitionPath = null;
|
||||||
|
|
||||||
private HoodieWriteStat stat = null;
|
private HoodieWriteStat stat = null;
|
||||||
|
|
||||||
private long totalRecords = 0;
|
private long totalRecords = 0;
|
||||||
private long totalErrorRecords = 0;
|
private long totalErrorRecords = 0;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Mark write as success, optionally using given parameters for the purpose of calculating
|
* Mark write as success, optionally using given parameters for the purpose of calculating some
|
||||||
* some aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
|
* aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
|
||||||
* objects are collected in Spark Driver.
|
* objects are collected in Spark Driver.
|
||||||
*
|
*
|
||||||
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies it.
|
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies
|
||||||
* @param optionalRecordMetadata optional metadata related to data contained in {@link HoodieRecord} before deflation.
|
* it.
|
||||||
*/
|
* @param optionalRecordMetadata optional metadata related to data contained in {@link
|
||||||
public void markSuccess(HoodieRecord record,
|
* HoodieRecord} before deflation.
|
||||||
Optional<Map<String, String>> optionalRecordMetadata) {
|
*/
|
||||||
writtenRecords.add(record);
|
public void markSuccess(HoodieRecord record,
|
||||||
totalRecords++;
|
Optional<Map<String, String>> optionalRecordMetadata) {
|
||||||
}
|
writtenRecords.add(record);
|
||||||
|
totalRecords++;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Mark write as failed, optionally using given parameters for the purpose of calculating
|
* Mark write as failed, optionally using given parameters for the purpose of calculating some
|
||||||
* some aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
|
* aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
|
||||||
* objects are collected in Spark Driver.
|
* objects are collected in Spark Driver.
|
||||||
*
|
*
|
||||||
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies it.
|
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies
|
||||||
* @param optionalRecordMetadata optional metadata related to data contained in {@link HoodieRecord} before deflation.
|
* it.
|
||||||
*/
|
* @param optionalRecordMetadata optional metadata related to data contained in {@link
|
||||||
public void markFailure(HoodieRecord record, Throwable t,
|
* HoodieRecord} before deflation.
|
||||||
Optional<Map<String, String>> optionalRecordMetadata) {
|
*/
|
||||||
failedRecords.add(record);
|
public void markFailure(HoodieRecord record, Throwable t,
|
||||||
errors.put(record.getKey(), t);
|
Optional<Map<String, String>> optionalRecordMetadata) {
|
||||||
totalRecords++;
|
failedRecords.add(record);
|
||||||
totalErrorRecords++;
|
errors.put(record.getKey(), t);
|
||||||
}
|
totalRecords++;
|
||||||
|
totalErrorRecords++;
|
||||||
|
}
|
||||||
|
|
||||||
public String getFileId() {
|
public String getFileId() {
|
||||||
return fileId;
|
return fileId;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setFileId(String fileId) {
|
public void setFileId(String fileId) {
|
||||||
this.fileId = fileId;
|
this.fileId = fileId;
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean hasErrors() {
|
public boolean hasErrors() {
|
||||||
return totalErrorRecords > 0;
|
return totalErrorRecords > 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean isErrored(HoodieKey key) {
|
public boolean isErrored(HoodieKey key) {
|
||||||
return errors.containsKey(key);
|
return errors.containsKey(key);
|
||||||
}
|
}
|
||||||
|
|
||||||
public HashMap<HoodieKey, Throwable> getErrors() {
|
public HashMap<HoodieKey, Throwable> getErrors() {
|
||||||
return errors;
|
return errors;
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean hasGlobalError() {
|
public boolean hasGlobalError() {
|
||||||
return globalError != null;
|
return globalError != null;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setGlobalError(Throwable t) {
|
public void setGlobalError(Throwable t) {
|
||||||
this.globalError = t;
|
this.globalError = t;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Throwable getGlobalError() {
|
public Throwable getGlobalError() {
|
||||||
return this.globalError;
|
return this.globalError;
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<HoodieRecord> getWrittenRecords() {
|
public List<HoodieRecord> getWrittenRecords() {
|
||||||
return writtenRecords;
|
return writtenRecords;
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<HoodieRecord> getFailedRecords() {
|
public List<HoodieRecord> getFailedRecords() {
|
||||||
return failedRecords;
|
return failedRecords;
|
||||||
}
|
}
|
||||||
|
|
||||||
public HoodieWriteStat getStat() {
|
public HoodieWriteStat getStat() {
|
||||||
return stat;
|
return stat;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setStat(HoodieWriteStat stat) {
|
public void setStat(HoodieWriteStat stat) {
|
||||||
this.stat = stat;
|
this.stat = stat;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getPartitionPath() {
|
public String getPartitionPath() {
|
||||||
return partitionPath;
|
return partitionPath;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setPartitionPath(String partitionPath) {
|
public void setPartitionPath(String partitionPath) {
|
||||||
this.partitionPath = partitionPath;
|
this.partitionPath = partitionPath;
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getTotalRecords() {
|
public long getTotalRecords() {
|
||||||
return totalRecords;
|
return totalRecords;
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getTotalErrorRecords() { return totalErrorRecords; }
|
public long getTotalErrorRecords() {
|
||||||
|
return totalErrorRecords;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
final StringBuilder sb = new StringBuilder("WriteStatus {");
|
final StringBuilder sb = new StringBuilder("WriteStatus {");
|
||||||
sb.append("fileId=").append(fileId);
|
sb.append("fileId=").append(fileId);
|
||||||
sb.append(", globalError='").append(globalError).append('\'');
|
sb.append(", globalError='").append(globalError).append('\'');
|
||||||
sb.append(", hasErrors='").append(hasErrors()).append('\'');
|
sb.append(", hasErrors='").append(hasErrors()).append('\'');
|
||||||
sb.append(", errorCount='").append(totalErrorRecords).append('\'');
|
sb.append(", errorCount='").append(totalErrorRecords).append('\'');
|
||||||
sb.append(", errorPct='").append((100.0 * totalErrorRecords) / totalRecords).append('\'');
|
sb.append(", errorPct='").append((100.0 * totalErrorRecords) / totalRecords).append('\'');
|
||||||
sb.append('}');
|
sb.append('}');
|
||||||
return sb.toString();
|
return sb.toString();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,33 +17,35 @@
|
|||||||
package com.uber.hoodie.config;
|
package com.uber.hoodie.config;
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Default Way to load Hoodie config through a java.util.Properties
|
* Default Way to load Hoodie config through a java.util.Properties
|
||||||
*/
|
*/
|
||||||
public class DefaultHoodieConfig implements Serializable {
|
public class DefaultHoodieConfig implements Serializable {
|
||||||
protected final Properties props;
|
|
||||||
public DefaultHoodieConfig(Properties props) {
|
|
||||||
this.props = props;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Properties getProps() {
|
protected final Properties props;
|
||||||
return props;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void setDefaultOnCondition(Properties props, boolean condition, String propName,
|
public DefaultHoodieConfig(Properties props) {
|
||||||
String defaultValue) {
|
this.props = props;
|
||||||
if (condition) {
|
}
|
||||||
props.setProperty(propName, defaultValue);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void setDefaultOnCondition(Properties props, boolean condition, DefaultHoodieConfig config) {
|
public Properties getProps() {
|
||||||
if (condition) {
|
return props;
|
||||||
props.putAll(config.getProps());
|
}
|
||||||
}
|
|
||||||
|
public static void setDefaultOnCondition(Properties props, boolean condition, String propName,
|
||||||
|
String defaultValue) {
|
||||||
|
if (condition) {
|
||||||
|
props.setProperty(propName, defaultValue);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void setDefaultOnCondition(Properties props, boolean condition,
|
||||||
|
DefaultHoodieConfig config) {
|
||||||
|
if (condition) {
|
||||||
|
props.putAll(config.getProps());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -19,231 +19,239 @@ package com.uber.hoodie.config;
|
|||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.uber.hoodie.common.model.HoodieAvroPayload;
|
import com.uber.hoodie.common.model.HoodieAvroPayload;
|
||||||
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
|
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
|
||||||
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
|
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
|
||||||
import com.uber.hoodie.io.compact.strategy.LogFileSizeBasedCompactionStrategy;
|
import com.uber.hoodie.io.compact.strategy.LogFileSizeBasedCompactionStrategy;
|
||||||
|
|
||||||
import javax.annotation.concurrent.Immutable;
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileReader;
|
import java.io.FileReader;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
|
import javax.annotation.concurrent.Immutable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Compaction related config
|
* Compaction related config
|
||||||
*/
|
*/
|
||||||
@Immutable
|
@Immutable
|
||||||
public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||||
public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy";
|
|
||||||
private static final String DEFAULT_CLEANER_POLICY =
|
|
||||||
HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name();
|
|
||||||
|
|
||||||
public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic";
|
public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy";
|
||||||
private static final String DEFAULT_AUTO_CLEAN = "true";
|
private static final String DEFAULT_CLEANER_POLICY =
|
||||||
|
HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name();
|
||||||
|
|
||||||
// Turn on inline compaction - after fw delta commits a inline compaction will be run
|
public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic";
|
||||||
public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
|
private static final String DEFAULT_AUTO_CLEAN = "true";
|
||||||
private static final String DEFAULT_INLINE_COMPACT = "true";
|
|
||||||
|
|
||||||
// Run a compaction every N delta commits
|
// Turn on inline compaction - after fw delta commits a inline compaction will be run
|
||||||
public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits";
|
public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
|
||||||
private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "10";
|
private static final String DEFAULT_INLINE_COMPACT = "true";
|
||||||
|
|
||||||
public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP =
|
// Run a compaction every N delta commits
|
||||||
"hoodie.cleaner.fileversions.retained";
|
public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits";
|
||||||
private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3";
|
private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "10";
|
||||||
|
|
||||||
public static final String CLEANER_COMMITS_RETAINED_PROP = "hoodie.cleaner.commits.retained";
|
public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP =
|
||||||
private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "24";
|
"hoodie.cleaner.fileversions.retained";
|
||||||
|
private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3";
|
||||||
|
|
||||||
public static final String MAX_COMMITS_TO_KEEP = "hoodie.keep.max.commits";
|
public static final String CLEANER_COMMITS_RETAINED_PROP = "hoodie.cleaner.commits.retained";
|
||||||
private static final String DEFAULT_MAX_COMMITS_TO_KEEP = String.valueOf(128);
|
private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "24";
|
||||||
public static final String MIN_COMMITS_TO_KEEP = "hoodie.keep.min.commits";
|
|
||||||
private static final String DEFAULT_MIN_COMMITS_TO_KEEP = String.valueOf(96);
|
public static final String MAX_COMMITS_TO_KEEP = "hoodie.keep.max.commits";
|
||||||
// Upsert uses this file size to compact new data onto existing files..
|
private static final String DEFAULT_MAX_COMMITS_TO_KEEP = String.valueOf(128);
|
||||||
public static final String PARQUET_SMALL_FILE_LIMIT_BYTES = "hoodie.parquet.small.file.limit";
|
public static final String MIN_COMMITS_TO_KEEP = "hoodie.keep.min.commits";
|
||||||
// Turned off by default
|
private static final String DEFAULT_MIN_COMMITS_TO_KEEP = String.valueOf(96);
|
||||||
public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = String.valueOf(0);
|
// Upsert uses this file size to compact new data onto existing files..
|
||||||
|
public static final String PARQUET_SMALL_FILE_LIMIT_BYTES = "hoodie.parquet.small.file.limit";
|
||||||
|
// Turned off by default
|
||||||
|
public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = String.valueOf(0);
|
||||||
|
|
||||||
|
|
||||||
/** Configs related to specific table types **/
|
/**
|
||||||
// Number of inserts, that will be put each partition/bucket for writing
|
* Configs related to specific table types
|
||||||
public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = "hoodie.copyonwrite.insert.split.size";
|
**/
|
||||||
// The rationale to pick the insert parallelism is the following. Writing out 100MB files,
|
// Number of inserts, that will be put each partition/bucket for writing
|
||||||
// with atleast 1kb records, means 100K records per file. we just overprovision to 500K
|
public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = "hoodie.copyonwrite.insert.split.size";
|
||||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = String.valueOf(500000);
|
// The rationale to pick the insert parallelism is the following. Writing out 100MB files,
|
||||||
|
// with atleast 1kb records, means 100K records per file. we just overprovision to 500K
|
||||||
|
public static final String DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = String.valueOf(500000);
|
||||||
|
|
||||||
// Config to control whether we control insert split sizes automatically based on average record sizes
|
// Config to control whether we control insert split sizes automatically based on average record sizes
|
||||||
public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = "hoodie.copyonwrite.insert.auto.split";
|
public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = "hoodie.copyonwrite.insert.auto.split";
|
||||||
// its off by default
|
// its off by default
|
||||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = String.valueOf(false);
|
public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = String.valueOf(false);
|
||||||
|
|
||||||
|
|
||||||
// This value is used as a guessimate for the record size, if we can't determine this from previous commits
|
// This value is used as a guessimate for the record size, if we can't determine this from previous commits
|
||||||
public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = "hoodie.copyonwrite.record.size.estimate";
|
public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = "hoodie.copyonwrite.record.size.estimate";
|
||||||
// Used to determine how much more can be packed into a small file, before it exceeds the size limit.
|
// Used to determine how much more can be packed into a small file, before it exceeds the size limit.
|
||||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = String.valueOf(1024);
|
public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = String
|
||||||
|
.valueOf(1024);
|
||||||
|
|
||||||
public static final String CLEANER_PARALLELISM = "hoodie.cleaner.parallelism";
|
public static final String CLEANER_PARALLELISM = "hoodie.cleaner.parallelism";
|
||||||
public static final String DEFAULT_CLEANER_PARALLELISM = String.valueOf(200);
|
public static final String DEFAULT_CLEANER_PARALLELISM = String.valueOf(200);
|
||||||
|
|
||||||
public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = "hoodie.compaction.target.io";
|
public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = "hoodie.compaction.target.io";
|
||||||
// 500GB of target IO per compaction (both read and write)
|
// 500GB of target IO per compaction (both read and write)
|
||||||
public static final String DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB = String.valueOf(500 * 1024);
|
public static final String DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB = String.valueOf(500 * 1024);
|
||||||
|
|
||||||
public static final String COMPACTION_STRATEGY_PROP = "hoodie.compaction.strategy";
|
public static final String COMPACTION_STRATEGY_PROP = "hoodie.compaction.strategy";
|
||||||
// 200GB of target IO per compaction
|
// 200GB of target IO per compaction
|
||||||
public static final String DEFAULT_COMPACTION_STRATEGY = LogFileSizeBasedCompactionStrategy.class.getName();
|
public static final String DEFAULT_COMPACTION_STRATEGY = LogFileSizeBasedCompactionStrategy.class
|
||||||
|
.getName();
|
||||||
|
|
||||||
// used to merge records written to log file
|
// used to merge records written to log file
|
||||||
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
|
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
|
||||||
public static final String PAYLOAD_CLASS = "hoodie.compaction.payload.class";
|
public static final String PAYLOAD_CLASS = "hoodie.compaction.payload.class";
|
||||||
|
|
||||||
private HoodieCompactionConfig(Properties props) {
|
private HoodieCompactionConfig(Properties props) {
|
||||||
super(props);
|
super(props);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static HoodieCompactionConfig.Builder newBuilder() {
|
||||||
|
return new Builder();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Builder {
|
||||||
|
|
||||||
|
private final Properties props = new Properties();
|
||||||
|
|
||||||
|
public Builder fromFile(File propertiesFile) throws IOException {
|
||||||
|
FileReader reader = new FileReader(propertiesFile);
|
||||||
|
try {
|
||||||
|
this.props.load(reader);
|
||||||
|
return this;
|
||||||
|
} finally {
|
||||||
|
reader.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static HoodieCompactionConfig.Builder newBuilder() {
|
public Builder fromProperties(Properties props) {
|
||||||
return new Builder();
|
this.props.putAll(props);
|
||||||
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class Builder {
|
|
||||||
private final Properties props = new Properties();
|
|
||||||
|
|
||||||
public Builder fromFile(File propertiesFile) throws IOException {
|
|
||||||
FileReader reader = new FileReader(propertiesFile);
|
|
||||||
try {
|
|
||||||
this.props.load(reader);
|
|
||||||
return this;
|
|
||||||
} finally {
|
|
||||||
reader.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder fromProperties(Properties props) {
|
|
||||||
this.props.putAll(props);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
public Builder withAutoClean(Boolean autoClean) {
|
|
||||||
props.setProperty(AUTO_CLEAN_PROP, String.valueOf(autoClean));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withInlineCompaction(Boolean inlineCompaction) {
|
|
||||||
props.setProperty(INLINE_COMPACT_PROP, String.valueOf(inlineCompaction));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder inlineCompactionEvery(int deltaCommits) {
|
|
||||||
props.setProperty(INLINE_COMPACT_PROP, String.valueOf(deltaCommits));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withCleanerPolicy(HoodieCleaningPolicy policy) {
|
|
||||||
props.setProperty(CLEANER_POLICY_PROP, policy.name());
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder retainFileVersions(int fileVersionsRetained) {
|
|
||||||
props.setProperty(CLEANER_FILE_VERSIONS_RETAINED_PROP,
|
|
||||||
String.valueOf(fileVersionsRetained));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder retainCommits(int commitsRetained) {
|
|
||||||
props.setProperty(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder archiveCommitsWith(int minToKeep, int maxToKeep) {
|
|
||||||
props.setProperty(MIN_COMMITS_TO_KEEP, String.valueOf(minToKeep));
|
|
||||||
props.setProperty(MAX_COMMITS_TO_KEEP, String.valueOf(maxToKeep));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder compactionSmallFileSize(long smallFileLimitBytes) {
|
|
||||||
props.setProperty(PARQUET_SMALL_FILE_LIMIT_BYTES, String.valueOf(smallFileLimitBytes));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder insertSplitSize(int insertSplitSize) {
|
|
||||||
props.setProperty(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, String.valueOf(insertSplitSize));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder autoTuneInsertSplits(boolean autoTuneInsertSplits) {
|
|
||||||
props.setProperty(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, String.valueOf(autoTuneInsertSplits));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder approxRecordSize(int recordSizeEstimate) {
|
|
||||||
props.setProperty(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, String.valueOf(recordSizeEstimate));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withCleanerParallelism(int cleanerParallelism) {
|
|
||||||
props.setProperty(CLEANER_PARALLELISM, String.valueOf(cleanerParallelism));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withCompactionStrategy(CompactionStrategy compactionStrategy) {
|
|
||||||
props.setProperty(COMPACTION_STRATEGY_PROP, compactionStrategy.getClass().getName());
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withPayloadClass(String payloadClassName) {
|
|
||||||
props.setProperty(PAYLOAD_CLASS, payloadClassName);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) {
|
|
||||||
props.setProperty(TARGET_IO_PER_COMPACTION_IN_MB_PROP, String.valueOf(targetIOPerCompactionInMB));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public HoodieCompactionConfig build() {
|
|
||||||
HoodieCompactionConfig config = new HoodieCompactionConfig(props);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP),
|
|
||||||
AUTO_CLEAN_PROP, DEFAULT_AUTO_CLEAN);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP),
|
|
||||||
INLINE_COMPACT_PROP, DEFAULT_INLINE_COMPACT);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP),
|
|
||||||
INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_POLICY_PROP),
|
|
||||||
CLEANER_POLICY_PROP, DEFAULT_CLEANER_POLICY);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_FILE_VERSIONS_RETAINED_PROP),
|
|
||||||
CLEANER_FILE_VERSIONS_RETAINED_PROP, DEFAULT_CLEANER_FILE_VERSIONS_RETAINED);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP),
|
|
||||||
CLEANER_COMMITS_RETAINED_PROP, DEFAULT_CLEANER_COMMITS_RETAINED);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP),
|
|
||||||
MAX_COMMITS_TO_KEEP, DEFAULT_MAX_COMMITS_TO_KEEP);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP),
|
|
||||||
MIN_COMMITS_TO_KEEP, DEFAULT_MIN_COMMITS_TO_KEEP);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_SMALL_FILE_LIMIT_BYTES),
|
|
||||||
PARQUET_SMALL_FILE_LIMIT_BYTES, DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE),
|
|
||||||
COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS),
|
|
||||||
COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE),
|
|
||||||
COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM),
|
|
||||||
CLEANER_PARALLELISM, DEFAULT_CLEANER_PARALLELISM);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP),
|
|
||||||
COMPACTION_STRATEGY_PROP, DEFAULT_COMPACTION_STRATEGY);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS),
|
|
||||||
PAYLOAD_CLASS, DEFAULT_PAYLOAD_CLASS);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(TARGET_IO_PER_COMPACTION_IN_MB_PROP),
|
|
||||||
TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB);
|
|
||||||
|
|
||||||
HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP));
|
|
||||||
Preconditions.checkArgument(
|
|
||||||
Integer.parseInt(props.getProperty(MAX_COMMITS_TO_KEEP)) > Integer
|
|
||||||
.parseInt(props.getProperty(MIN_COMMITS_TO_KEEP)));
|
|
||||||
return config;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
public Builder withAutoClean(Boolean autoClean) {
|
||||||
|
props.setProperty(AUTO_CLEAN_PROP, String.valueOf(autoClean));
|
||||||
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder withInlineCompaction(Boolean inlineCompaction) {
|
||||||
|
props.setProperty(INLINE_COMPACT_PROP, String.valueOf(inlineCompaction));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder inlineCompactionEvery(int deltaCommits) {
|
||||||
|
props.setProperty(INLINE_COMPACT_PROP, String.valueOf(deltaCommits));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withCleanerPolicy(HoodieCleaningPolicy policy) {
|
||||||
|
props.setProperty(CLEANER_POLICY_PROP, policy.name());
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder retainFileVersions(int fileVersionsRetained) {
|
||||||
|
props.setProperty(CLEANER_FILE_VERSIONS_RETAINED_PROP,
|
||||||
|
String.valueOf(fileVersionsRetained));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder retainCommits(int commitsRetained) {
|
||||||
|
props.setProperty(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder archiveCommitsWith(int minToKeep, int maxToKeep) {
|
||||||
|
props.setProperty(MIN_COMMITS_TO_KEEP, String.valueOf(minToKeep));
|
||||||
|
props.setProperty(MAX_COMMITS_TO_KEEP, String.valueOf(maxToKeep));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder compactionSmallFileSize(long smallFileLimitBytes) {
|
||||||
|
props.setProperty(PARQUET_SMALL_FILE_LIMIT_BYTES, String.valueOf(smallFileLimitBytes));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder insertSplitSize(int insertSplitSize) {
|
||||||
|
props.setProperty(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, String.valueOf(insertSplitSize));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder autoTuneInsertSplits(boolean autoTuneInsertSplits) {
|
||||||
|
props.setProperty(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS,
|
||||||
|
String.valueOf(autoTuneInsertSplits));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder approxRecordSize(int recordSizeEstimate) {
|
||||||
|
props.setProperty(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE,
|
||||||
|
String.valueOf(recordSizeEstimate));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withCleanerParallelism(int cleanerParallelism) {
|
||||||
|
props.setProperty(CLEANER_PARALLELISM, String.valueOf(cleanerParallelism));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withCompactionStrategy(CompactionStrategy compactionStrategy) {
|
||||||
|
props.setProperty(COMPACTION_STRATEGY_PROP, compactionStrategy.getClass().getName());
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withPayloadClass(String payloadClassName) {
|
||||||
|
props.setProperty(PAYLOAD_CLASS, payloadClassName);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) {
|
||||||
|
props.setProperty(TARGET_IO_PER_COMPACTION_IN_MB_PROP,
|
||||||
|
String.valueOf(targetIOPerCompactionInMB));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieCompactionConfig build() {
|
||||||
|
HoodieCompactionConfig config = new HoodieCompactionConfig(props);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP),
|
||||||
|
AUTO_CLEAN_PROP, DEFAULT_AUTO_CLEAN);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP),
|
||||||
|
INLINE_COMPACT_PROP, DEFAULT_INLINE_COMPACT);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP),
|
||||||
|
INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(CLEANER_POLICY_PROP),
|
||||||
|
CLEANER_POLICY_PROP, DEFAULT_CLEANER_POLICY);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(CLEANER_FILE_VERSIONS_RETAINED_PROP),
|
||||||
|
CLEANER_FILE_VERSIONS_RETAINED_PROP, DEFAULT_CLEANER_FILE_VERSIONS_RETAINED);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP),
|
||||||
|
CLEANER_COMMITS_RETAINED_PROP, DEFAULT_CLEANER_COMMITS_RETAINED);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP),
|
||||||
|
MAX_COMMITS_TO_KEEP, DEFAULT_MAX_COMMITS_TO_KEEP);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP),
|
||||||
|
MIN_COMMITS_TO_KEEP, DEFAULT_MIN_COMMITS_TO_KEEP);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(PARQUET_SMALL_FILE_LIMIT_BYTES),
|
||||||
|
PARQUET_SMALL_FILE_LIMIT_BYTES, DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE),
|
||||||
|
COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS),
|
||||||
|
COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE),
|
||||||
|
COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE,
|
||||||
|
DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM),
|
||||||
|
CLEANER_PARALLELISM, DEFAULT_CLEANER_PARALLELISM);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP),
|
||||||
|
COMPACTION_STRATEGY_PROP, DEFAULT_COMPACTION_STRATEGY);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS),
|
||||||
|
PAYLOAD_CLASS, DEFAULT_PAYLOAD_CLASS);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(TARGET_IO_PER_COMPACTION_IN_MB_PROP),
|
||||||
|
TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB);
|
||||||
|
|
||||||
|
HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP));
|
||||||
|
Preconditions.checkArgument(
|
||||||
|
Integer.parseInt(props.getProperty(MAX_COMMITS_TO_KEEP)) > Integer
|
||||||
|
.parseInt(props.getProperty(MIN_COMMITS_TO_KEEP)));
|
||||||
|
return config;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,14 +16,12 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.config;
|
package com.uber.hoodie.config;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
|
||||||
import com.uber.hoodie.index.HoodieIndex;
|
import com.uber.hoodie.index.HoodieIndex;
|
||||||
|
|
||||||
import javax.annotation.concurrent.Immutable;
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileReader;
|
import java.io.FileReader;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
|
import javax.annotation.concurrent.Immutable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Indexing related config
|
* Indexing related config
|
||||||
@@ -31,123 +29,124 @@ import java.util.Properties;
|
|||||||
@Immutable
|
@Immutable
|
||||||
public class HoodieIndexConfig extends DefaultHoodieConfig {
|
public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||||
|
|
||||||
public static final String INDEX_TYPE_PROP = "hoodie.index.type";
|
public static final String INDEX_TYPE_PROP = "hoodie.index.type";
|
||||||
public static final String DEFAULT_INDEX_TYPE = HoodieIndex.IndexType.BLOOM.name();
|
public static final String DEFAULT_INDEX_TYPE = HoodieIndex.IndexType.BLOOM.name();
|
||||||
|
|
||||||
// ***** Bloom Index configs *****
|
// ***** Bloom Index configs *****
|
||||||
public static final String BLOOM_FILTER_NUM_ENTRIES = "hoodie.index.bloom.num_entries";
|
public static final String BLOOM_FILTER_NUM_ENTRIES = "hoodie.index.bloom.num_entries";
|
||||||
public static final String DEFAULT_BLOOM_FILTER_NUM_ENTRIES = "60000";
|
public static final String DEFAULT_BLOOM_FILTER_NUM_ENTRIES = "60000";
|
||||||
public static final String BLOOM_FILTER_FPP = "hoodie.index.bloom.fpp";
|
public static final String BLOOM_FILTER_FPP = "hoodie.index.bloom.fpp";
|
||||||
public static final String DEFAULT_BLOOM_FILTER_FPP = "0.000000001";
|
public static final String DEFAULT_BLOOM_FILTER_FPP = "0.000000001";
|
||||||
public static final String BLOOM_INDEX_PARALLELISM_PROP = "hoodie.bloom.index.parallelism";
|
public static final String BLOOM_INDEX_PARALLELISM_PROP = "hoodie.bloom.index.parallelism";
|
||||||
// Disable explicit bloom index parallelism setting by default - hoodie auto computes
|
// Disable explicit bloom index parallelism setting by default - hoodie auto computes
|
||||||
public static final String DEFAULT_BLOOM_INDEX_PARALLELISM = "0";
|
public static final String DEFAULT_BLOOM_INDEX_PARALLELISM = "0";
|
||||||
public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP = "hoodie.bloom.index.prune.by.ranges";
|
public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP = "hoodie.bloom.index.prune.by.ranges";
|
||||||
public static final String DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES = "true";
|
public static final String DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES = "true";
|
||||||
public static final String BLOOM_INDEX_USE_CACHING_PROP = "hoodie.bloom.index.use.caching";
|
public static final String BLOOM_INDEX_USE_CACHING_PROP = "hoodie.bloom.index.use.caching";
|
||||||
public static final String DEFAULT_BLOOM_INDEX_USE_CACHING = "true";
|
public static final String DEFAULT_BLOOM_INDEX_USE_CACHING = "true";
|
||||||
|
|
||||||
// ***** HBase Index Configs *****
|
// ***** HBase Index Configs *****
|
||||||
public final static String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum";
|
public final static String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum";
|
||||||
public final static String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport";
|
public final static String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport";
|
||||||
public final static String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table";
|
public final static String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table";
|
||||||
|
|
||||||
// ***** Bucketed Index Configs *****
|
// ***** Bucketed Index Configs *****
|
||||||
public final static String BUCKETED_INDEX_NUM_BUCKETS_PROP = "hoodie.index.bucketed.numbuckets";
|
public final static String BUCKETED_INDEX_NUM_BUCKETS_PROP = "hoodie.index.bucketed.numbuckets";
|
||||||
|
|
||||||
private HoodieIndexConfig(Properties props) {
|
private HoodieIndexConfig(Properties props) {
|
||||||
super(props);
|
super(props);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static HoodieIndexConfig.Builder newBuilder() {
|
||||||
|
return new Builder();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Builder {
|
||||||
|
|
||||||
|
private final Properties props = new Properties();
|
||||||
|
|
||||||
|
public Builder fromFile(File propertiesFile) throws IOException {
|
||||||
|
FileReader reader = new FileReader(propertiesFile);
|
||||||
|
try {
|
||||||
|
this.props.load(reader);
|
||||||
|
return this;
|
||||||
|
} finally {
|
||||||
|
reader.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static HoodieIndexConfig.Builder newBuilder() {
|
public Builder fromProperties(Properties props) {
|
||||||
return new Builder();
|
this.props.putAll(props);
|
||||||
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class Builder {
|
public Builder withIndexType(HoodieIndex.IndexType indexType) {
|
||||||
private final Properties props = new Properties();
|
props.setProperty(INDEX_TYPE_PROP, indexType.name());
|
||||||
|
return this;
|
||||||
public Builder fromFile(File propertiesFile) throws IOException {
|
|
||||||
FileReader reader = new FileReader(propertiesFile);
|
|
||||||
try {
|
|
||||||
this.props.load(reader);
|
|
||||||
return this;
|
|
||||||
} finally {
|
|
||||||
reader.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder fromProperties(Properties props) {
|
|
||||||
this.props.putAll(props);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withIndexType(HoodieIndex.IndexType indexType) {
|
|
||||||
props.setProperty(INDEX_TYPE_PROP, indexType.name());
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder bloomFilterNumEntries(int numEntries) {
|
|
||||||
props.setProperty(BLOOM_FILTER_NUM_ENTRIES, String.valueOf(numEntries));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder bloomFilterFPP(double fpp) {
|
|
||||||
props.setProperty(BLOOM_FILTER_FPP, String.valueOf(fpp));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder hbaseZkQuorum(String zkString) {
|
|
||||||
props.setProperty(HBASE_ZKQUORUM_PROP, zkString);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder hbaseZkPort(int port) {
|
|
||||||
props.setProperty(HBASE_ZKPORT_PROP, String.valueOf(port));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder hbaseTableName(String tableName) {
|
|
||||||
props.setProperty(HBASE_TABLENAME_PROP, tableName);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder bloomIndexParallelism(int parallelism) {
|
|
||||||
props.setProperty(BLOOM_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder bloomIndexPruneByRanges(boolean pruneRanges) {
|
|
||||||
props.setProperty(BLOOM_INDEX_PRUNE_BY_RANGES_PROP, String.valueOf(pruneRanges));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder bloomIndexUseCaching(boolean useCaching) {
|
|
||||||
props.setProperty(BLOOM_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder numBucketsPerPartition(int numBuckets) {
|
|
||||||
props.setProperty(BUCKETED_INDEX_NUM_BUCKETS_PROP, String.valueOf(numBuckets));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public HoodieIndexConfig build() {
|
|
||||||
HoodieIndexConfig config = new HoodieIndexConfig(props);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP),
|
|
||||||
INDEX_TYPE_PROP, DEFAULT_INDEX_TYPE);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_NUM_ENTRIES),
|
|
||||||
BLOOM_FILTER_NUM_ENTRIES, DEFAULT_BLOOM_FILTER_NUM_ENTRIES);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP),
|
|
||||||
BLOOM_FILTER_FPP, DEFAULT_BLOOM_FILTER_FPP);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PARALLELISM_PROP),
|
|
||||||
BLOOM_INDEX_PARALLELISM_PROP, DEFAULT_BLOOM_INDEX_PARALLELISM);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PRUNE_BY_RANGES_PROP),
|
|
||||||
BLOOM_INDEX_PRUNE_BY_RANGES_PROP, DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_USE_CACHING_PROP),
|
|
||||||
BLOOM_INDEX_USE_CACHING_PROP, DEFAULT_BLOOM_INDEX_USE_CACHING);
|
|
||||||
// Throws IllegalArgumentException if the value set is not a known Hoodie Index Type
|
|
||||||
HoodieIndex.IndexType.valueOf(props.getProperty(INDEX_TYPE_PROP));
|
|
||||||
return config;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder bloomFilterNumEntries(int numEntries) {
|
||||||
|
props.setProperty(BLOOM_FILTER_NUM_ENTRIES, String.valueOf(numEntries));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder bloomFilterFPP(double fpp) {
|
||||||
|
props.setProperty(BLOOM_FILTER_FPP, String.valueOf(fpp));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder hbaseZkQuorum(String zkString) {
|
||||||
|
props.setProperty(HBASE_ZKQUORUM_PROP, zkString);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder hbaseZkPort(int port) {
|
||||||
|
props.setProperty(HBASE_ZKPORT_PROP, String.valueOf(port));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder hbaseTableName(String tableName) {
|
||||||
|
props.setProperty(HBASE_TABLENAME_PROP, tableName);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder bloomIndexParallelism(int parallelism) {
|
||||||
|
props.setProperty(BLOOM_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder bloomIndexPruneByRanges(boolean pruneRanges) {
|
||||||
|
props.setProperty(BLOOM_INDEX_PRUNE_BY_RANGES_PROP, String.valueOf(pruneRanges));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder bloomIndexUseCaching(boolean useCaching) {
|
||||||
|
props.setProperty(BLOOM_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder numBucketsPerPartition(int numBuckets) {
|
||||||
|
props.setProperty(BUCKETED_INDEX_NUM_BUCKETS_PROP, String.valueOf(numBuckets));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieIndexConfig build() {
|
||||||
|
HoodieIndexConfig config = new HoodieIndexConfig(props);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP),
|
||||||
|
INDEX_TYPE_PROP, DEFAULT_INDEX_TYPE);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_NUM_ENTRIES),
|
||||||
|
BLOOM_FILTER_NUM_ENTRIES, DEFAULT_BLOOM_FILTER_NUM_ENTRIES);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP),
|
||||||
|
BLOOM_FILTER_FPP, DEFAULT_BLOOM_FILTER_FPP);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PARALLELISM_PROP),
|
||||||
|
BLOOM_INDEX_PARALLELISM_PROP, DEFAULT_BLOOM_INDEX_PARALLELISM);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PRUNE_BY_RANGES_PROP),
|
||||||
|
BLOOM_INDEX_PRUNE_BY_RANGES_PROP, DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_USE_CACHING_PROP),
|
||||||
|
BLOOM_INDEX_USE_CACHING_PROP, DEFAULT_BLOOM_INDEX_USE_CACHING);
|
||||||
|
// Throws IllegalArgumentException if the value set is not a known Hoodie Index Type
|
||||||
|
HoodieIndex.IndexType.valueOf(props.getProperty(INDEX_TYPE_PROP));
|
||||||
|
return config;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,12 +17,11 @@
|
|||||||
package com.uber.hoodie.config;
|
package com.uber.hoodie.config;
|
||||||
|
|
||||||
import com.uber.hoodie.metrics.MetricsReporterType;
|
import com.uber.hoodie.metrics.MetricsReporterType;
|
||||||
|
|
||||||
import javax.annotation.concurrent.Immutable;
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileReader;
|
import java.io.FileReader;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
|
import javax.annotation.concurrent.Immutable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Fetch the configurations used by the Metrics system.
|
* Fetch the configurations used by the Metrics system.
|
||||||
@@ -30,89 +29,90 @@ import java.util.Properties;
|
|||||||
@Immutable
|
@Immutable
|
||||||
public class HoodieMetricsConfig extends DefaultHoodieConfig {
|
public class HoodieMetricsConfig extends DefaultHoodieConfig {
|
||||||
|
|
||||||
public final static String METRIC_PREFIX = "hoodie.metrics";
|
public final static String METRIC_PREFIX = "hoodie.metrics";
|
||||||
public final static String METRICS_ON = METRIC_PREFIX + ".on";
|
public final static String METRICS_ON = METRIC_PREFIX + ".on";
|
||||||
public final static boolean DEFAULT_METRICS_ON = false;
|
public final static boolean DEFAULT_METRICS_ON = false;
|
||||||
public final static String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type";
|
public final static String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type";
|
||||||
public final static MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE =
|
public final static MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE =
|
||||||
MetricsReporterType.GRAPHITE;
|
MetricsReporterType.GRAPHITE;
|
||||||
|
|
||||||
// Graphite
|
// Graphite
|
||||||
public final static String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite";
|
public final static String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite";
|
||||||
public final static String GRAPHITE_SERVER_HOST = GRAPHITE_PREFIX + ".host";
|
public final static String GRAPHITE_SERVER_HOST = GRAPHITE_PREFIX + ".host";
|
||||||
public final static String DEFAULT_GRAPHITE_SERVER_HOST = "localhost";
|
public final static String DEFAULT_GRAPHITE_SERVER_HOST = "localhost";
|
||||||
|
|
||||||
public final static String GRAPHITE_SERVER_PORT = GRAPHITE_PREFIX + ".port";
|
public final static String GRAPHITE_SERVER_PORT = GRAPHITE_PREFIX + ".port";
|
||||||
public final static int DEFAULT_GRAPHITE_SERVER_PORT = 4756;
|
public final static int DEFAULT_GRAPHITE_SERVER_PORT = 4756;
|
||||||
|
|
||||||
public final static String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix";
|
public final static String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix";
|
||||||
|
|
||||||
private HoodieMetricsConfig(Properties props) {
|
private HoodieMetricsConfig(Properties props) {
|
||||||
super(props);
|
super(props);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static HoodieMetricsConfig.Builder newBuilder() {
|
||||||
|
return new Builder();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Builder {
|
||||||
|
|
||||||
|
private final Properties props = new Properties();
|
||||||
|
|
||||||
|
public Builder fromFile(File propertiesFile) throws IOException {
|
||||||
|
FileReader reader = new FileReader(propertiesFile);
|
||||||
|
try {
|
||||||
|
this.props.load(reader);
|
||||||
|
return this;
|
||||||
|
} finally {
|
||||||
|
reader.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static HoodieMetricsConfig.Builder newBuilder() {
|
public Builder fromProperties(Properties props) {
|
||||||
return new Builder();
|
this.props.putAll(props);
|
||||||
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class Builder {
|
|
||||||
private final Properties props = new Properties();
|
|
||||||
|
|
||||||
public Builder fromFile(File propertiesFile) throws IOException {
|
public Builder on(boolean metricsOn) {
|
||||||
FileReader reader = new FileReader(propertiesFile);
|
props.setProperty(METRICS_ON, String.valueOf(metricsOn));
|
||||||
try {
|
return this;
|
||||||
this.props.load(reader);
|
|
||||||
return this;
|
|
||||||
} finally {
|
|
||||||
reader.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder fromProperties(Properties props) {
|
|
||||||
this.props.putAll(props);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
public Builder on(boolean metricsOn) {
|
|
||||||
props.setProperty(METRICS_ON, String.valueOf(metricsOn));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withReporterType(String reporterType) {
|
|
||||||
props.setProperty(METRICS_REPORTER_TYPE, reporterType);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder toGraphiteHost(String host) {
|
|
||||||
props.setProperty(GRAPHITE_SERVER_HOST, host);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder onGraphitePort(int port) {
|
|
||||||
props.setProperty(GRAPHITE_SERVER_PORT, String.valueOf(port));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder usePrefix(String prefix) {
|
|
||||||
props.setProperty(GRAPHITE_METRIC_PREFIX, prefix);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public HoodieMetricsConfig build() {
|
|
||||||
HoodieMetricsConfig config = new HoodieMetricsConfig(props);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON,
|
|
||||||
String.valueOf(DEFAULT_METRICS_ON));
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE),
|
|
||||||
METRICS_REPORTER_TYPE, DEFAULT_METRICS_REPORTER_TYPE.name());
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST),
|
|
||||||
GRAPHITE_SERVER_HOST, DEFAULT_GRAPHITE_SERVER_HOST);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT),
|
|
||||||
GRAPHITE_SERVER_PORT, String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT),
|
|
||||||
GRAPHITE_SERVER_PORT, String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
|
|
||||||
return config;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder withReporterType(String reporterType) {
|
||||||
|
props.setProperty(METRICS_REPORTER_TYPE, reporterType);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder toGraphiteHost(String host) {
|
||||||
|
props.setProperty(GRAPHITE_SERVER_HOST, host);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder onGraphitePort(int port) {
|
||||||
|
props.setProperty(GRAPHITE_SERVER_PORT, String.valueOf(port));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder usePrefix(String prefix) {
|
||||||
|
props.setProperty(GRAPHITE_METRIC_PREFIX, prefix);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieMetricsConfig build() {
|
||||||
|
HoodieMetricsConfig config = new HoodieMetricsConfig(props);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON,
|
||||||
|
String.valueOf(DEFAULT_METRICS_ON));
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE),
|
||||||
|
METRICS_REPORTER_TYPE, DEFAULT_METRICS_REPORTER_TYPE.name());
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST),
|
||||||
|
GRAPHITE_SERVER_HOST, DEFAULT_GRAPHITE_SERVER_HOST);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT),
|
||||||
|
GRAPHITE_SERVER_PORT, String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT),
|
||||||
|
GRAPHITE_SERVER_PORT, String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
|
||||||
|
return config;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,75 +16,77 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.config;
|
package com.uber.hoodie.config;
|
||||||
|
|
||||||
import javax.annotation.concurrent.Immutable;
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileReader;
|
import java.io.FileReader;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
|
import javax.annotation.concurrent.Immutable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Storage related config
|
* Storage related config
|
||||||
*/
|
*/
|
||||||
@Immutable
|
@Immutable
|
||||||
public class HoodieStorageConfig extends DefaultHoodieConfig {
|
public class HoodieStorageConfig extends DefaultHoodieConfig {
|
||||||
public static final String PARQUET_FILE_MAX_BYTES = "hoodie.parquet.max.file.size";
|
|
||||||
public static final String DEFAULT_PARQUET_FILE_MAX_BYTES = String.valueOf(120 * 1024 * 1024);
|
|
||||||
public static final String PARQUET_BLOCK_SIZE_BYTES = "hoodie.parquet.block.size";
|
|
||||||
public static final String DEFAULT_PARQUET_BLOCK_SIZE_BYTES = DEFAULT_PARQUET_FILE_MAX_BYTES;
|
|
||||||
public static final String PARQUET_PAGE_SIZE_BYTES = "hoodie.parquet.page.size";
|
|
||||||
public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);
|
|
||||||
|
|
||||||
private HoodieStorageConfig(Properties props) {
|
public static final String PARQUET_FILE_MAX_BYTES = "hoodie.parquet.max.file.size";
|
||||||
super(props);
|
public static final String DEFAULT_PARQUET_FILE_MAX_BYTES = String.valueOf(120 * 1024 * 1024);
|
||||||
|
public static final String PARQUET_BLOCK_SIZE_BYTES = "hoodie.parquet.block.size";
|
||||||
|
public static final String DEFAULT_PARQUET_BLOCK_SIZE_BYTES = DEFAULT_PARQUET_FILE_MAX_BYTES;
|
||||||
|
public static final String PARQUET_PAGE_SIZE_BYTES = "hoodie.parquet.page.size";
|
||||||
|
public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);
|
||||||
|
|
||||||
|
private HoodieStorageConfig(Properties props) {
|
||||||
|
super(props);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static HoodieStorageConfig.Builder newBuilder() {
|
||||||
|
return new Builder();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class Builder {
|
||||||
|
|
||||||
|
private final Properties props = new Properties();
|
||||||
|
|
||||||
|
public Builder fromFile(File propertiesFile) throws IOException {
|
||||||
|
FileReader reader = new FileReader(propertiesFile);
|
||||||
|
try {
|
||||||
|
this.props.load(reader);
|
||||||
|
return this;
|
||||||
|
} finally {
|
||||||
|
reader.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static HoodieStorageConfig.Builder newBuilder() {
|
public Builder fromProperties(Properties props) {
|
||||||
return new Builder();
|
this.props.putAll(props);
|
||||||
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class Builder {
|
public Builder limitFileSize(int maxFileSize) {
|
||||||
private final Properties props = new Properties();
|
props.setProperty(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
|
||||||
|
return this;
|
||||||
public Builder fromFile(File propertiesFile) throws IOException {
|
|
||||||
FileReader reader = new FileReader(propertiesFile);
|
|
||||||
try {
|
|
||||||
this.props.load(reader);
|
|
||||||
return this;
|
|
||||||
} finally {
|
|
||||||
reader.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder fromProperties(Properties props) {
|
|
||||||
this.props.putAll(props);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder limitFileSize(int maxFileSize) {
|
|
||||||
props.setProperty(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder parquetBlockSize(int blockSize) {
|
|
||||||
props.setProperty(PARQUET_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder parquetPageSize(int pageSize) {
|
|
||||||
props.setProperty(PARQUET_PAGE_SIZE_BYTES, String.valueOf(pageSize));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public HoodieStorageConfig build() {
|
|
||||||
HoodieStorageConfig config = new HoodieStorageConfig(props);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES),
|
|
||||||
PARQUET_FILE_MAX_BYTES, DEFAULT_PARQUET_FILE_MAX_BYTES);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_BLOCK_SIZE_BYTES),
|
|
||||||
PARQUET_BLOCK_SIZE_BYTES, DEFAULT_PARQUET_BLOCK_SIZE_BYTES);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_PAGE_SIZE_BYTES),
|
|
||||||
PARQUET_PAGE_SIZE_BYTES, DEFAULT_PARQUET_PAGE_SIZE_BYTES);
|
|
||||||
return config;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder parquetBlockSize(int blockSize) {
|
||||||
|
props.setProperty(PARQUET_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder parquetPageSize(int pageSize) {
|
||||||
|
props.setProperty(PARQUET_PAGE_SIZE_BYTES, String.valueOf(pageSize));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieStorageConfig build() {
|
||||||
|
HoodieStorageConfig config = new HoodieStorageConfig(props);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES),
|
||||||
|
PARQUET_FILE_MAX_BYTES, DEFAULT_PARQUET_FILE_MAX_BYTES);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(PARQUET_BLOCK_SIZE_BYTES),
|
||||||
|
PARQUET_BLOCK_SIZE_BYTES, DEFAULT_PARQUET_BLOCK_SIZE_BYTES);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(PARQUET_PAGE_SIZE_BYTES),
|
||||||
|
PARQUET_PAGE_SIZE_BYTES, DEFAULT_PARQUET_PAGE_SIZE_BYTES);
|
||||||
|
return config;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -24,395 +24,401 @@ import com.uber.hoodie.common.util.ReflectionUtils;
|
|||||||
import com.uber.hoodie.index.HoodieIndex;
|
import com.uber.hoodie.index.HoodieIndex;
|
||||||
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
|
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
|
||||||
import com.uber.hoodie.metrics.MetricsReporterType;
|
import com.uber.hoodie.metrics.MetricsReporterType;
|
||||||
import org.apache.spark.storage.StorageLevel;
|
|
||||||
|
|
||||||
import javax.annotation.concurrent.Immutable;
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.FileReader;
|
import java.io.FileReader;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
|
import javax.annotation.concurrent.Immutable;
|
||||||
|
import org.apache.spark.storage.StorageLevel;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Class storing configs for the {@link com.uber.hoodie.HoodieWriteClient}
|
* Class storing configs for the {@link com.uber.hoodie.HoodieWriteClient}
|
||||||
*/
|
*/
|
||||||
@Immutable
|
@Immutable
|
||||||
public class HoodieWriteConfig extends DefaultHoodieConfig {
|
public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||||
private static final String BASE_PATH_PROP = "hoodie.base.path";
|
|
||||||
private static final String AVRO_SCHEMA = "hoodie.avro.schema";
|
|
||||||
public static final String TABLE_NAME = "hoodie.table.name";
|
|
||||||
private static final String DEFAULT_PARALLELISM = "200";
|
|
||||||
private static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism";
|
|
||||||
private static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism";
|
|
||||||
private static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism";
|
|
||||||
private static final String COMBINE_BEFORE_INSERT_PROP = "hoodie.combine.before.insert";
|
|
||||||
private static final String DEFAULT_COMBINE_BEFORE_INSERT = "false";
|
|
||||||
private static final String COMBINE_BEFORE_UPSERT_PROP = "hoodie.combine.before.upsert";
|
|
||||||
private static final String DEFAULT_COMBINE_BEFORE_UPSERT = "true";
|
|
||||||
private static final String WRITE_STATUS_STORAGE_LEVEL = "hoodie.write.status.storage.level";
|
|
||||||
private static final String DEFAULT_WRITE_STATUS_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
|
|
||||||
private static final String HOODIE_AUTO_COMMIT_PROP = "hoodie.auto.commit";
|
|
||||||
private static final String DEFAULT_HOODIE_AUTO_COMMIT = "true";
|
|
||||||
private static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = "hoodie.assume.date.partitioning";
|
|
||||||
private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false";
|
|
||||||
private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class";
|
|
||||||
private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName();
|
|
||||||
|
|
||||||
private HoodieWriteConfig(Properties props) {
|
private static final String BASE_PATH_PROP = "hoodie.base.path";
|
||||||
super(props);
|
private static final String AVRO_SCHEMA = "hoodie.avro.schema";
|
||||||
}
|
public static final String TABLE_NAME = "hoodie.table.name";
|
||||||
|
private static final String DEFAULT_PARALLELISM = "200";
|
||||||
|
private static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism";
|
||||||
|
private static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism";
|
||||||
|
private static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism";
|
||||||
|
private static final String COMBINE_BEFORE_INSERT_PROP = "hoodie.combine.before.insert";
|
||||||
|
private static final String DEFAULT_COMBINE_BEFORE_INSERT = "false";
|
||||||
|
private static final String COMBINE_BEFORE_UPSERT_PROP = "hoodie.combine.before.upsert";
|
||||||
|
private static final String DEFAULT_COMBINE_BEFORE_UPSERT = "true";
|
||||||
|
private static final String WRITE_STATUS_STORAGE_LEVEL = "hoodie.write.status.storage.level";
|
||||||
|
private static final String DEFAULT_WRITE_STATUS_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
|
||||||
|
private static final String HOODIE_AUTO_COMMIT_PROP = "hoodie.auto.commit";
|
||||||
|
private static final String DEFAULT_HOODIE_AUTO_COMMIT = "true";
|
||||||
|
private static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = "hoodie.assume.date.partitioning";
|
||||||
|
private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false";
|
||||||
|
private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class";
|
||||||
|
private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName();
|
||||||
|
|
||||||
/**
|
private HoodieWriteConfig(Properties props) {
|
||||||
* base properties
|
super(props);
|
||||||
**/
|
}
|
||||||
public String getBasePath() {
|
|
||||||
return props.getProperty(BASE_PATH_PROP);
|
|
||||||
}
|
|
||||||
|
|
||||||
public String getSchema() {
|
/**
|
||||||
return props.getProperty(AVRO_SCHEMA);
|
* base properties
|
||||||
}
|
**/
|
||||||
|
public String getBasePath() {
|
||||||
|
return props.getProperty(BASE_PATH_PROP);
|
||||||
|
}
|
||||||
|
|
||||||
public String getTableName() {
|
public String getSchema() {
|
||||||
return props.getProperty(TABLE_NAME);
|
return props.getProperty(AVRO_SCHEMA);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Boolean shouldAutoCommit() {
|
public String getTableName() {
|
||||||
return Boolean.parseBoolean(props.getProperty(HOODIE_AUTO_COMMIT_PROP));
|
return props.getProperty(TABLE_NAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Boolean shouldAssumeDatePartitioning() {
|
public Boolean shouldAutoCommit() {
|
||||||
return Boolean.parseBoolean(props.getProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP));
|
return Boolean.parseBoolean(props.getProperty(HOODIE_AUTO_COMMIT_PROP));
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getBulkInsertShuffleParallelism() {
|
public Boolean shouldAssumeDatePartitioning() {
|
||||||
return Integer.parseInt(props.getProperty(BULKINSERT_PARALLELISM));
|
return Boolean.parseBoolean(props.getProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP));
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getInsertShuffleParallelism() {
|
public int getBulkInsertShuffleParallelism() {
|
||||||
return Integer.parseInt(props.getProperty(INSERT_PARALLELISM));
|
return Integer.parseInt(props.getProperty(BULKINSERT_PARALLELISM));
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getUpsertShuffleParallelism() {
|
public int getInsertShuffleParallelism() {
|
||||||
return Integer.parseInt(props.getProperty(UPSERT_PARALLELISM));
|
return Integer.parseInt(props.getProperty(INSERT_PARALLELISM));
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean shouldCombineBeforeInsert() {
|
public int getUpsertShuffleParallelism() {
|
||||||
return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_INSERT_PROP));
|
return Integer.parseInt(props.getProperty(UPSERT_PARALLELISM));
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean shouldCombineBeforeUpsert() {
|
public boolean shouldCombineBeforeInsert() {
|
||||||
return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_UPSERT_PROP));
|
return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_INSERT_PROP));
|
||||||
}
|
}
|
||||||
|
|
||||||
public StorageLevel getWriteStatusStorageLevel() {
|
public boolean shouldCombineBeforeUpsert() {
|
||||||
return StorageLevel.fromString(props.getProperty(WRITE_STATUS_STORAGE_LEVEL));
|
return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_UPSERT_PROP));
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getWriteStatusClassName() {
|
public StorageLevel getWriteStatusStorageLevel() {
|
||||||
return props.getProperty(HOODIE_WRITE_STATUS_CLASS_PROP);
|
return StorageLevel.fromString(props.getProperty(WRITE_STATUS_STORAGE_LEVEL));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
public String getWriteStatusClassName() {
|
||||||
* compaction properties
|
return props.getProperty(HOODIE_WRITE_STATUS_CLASS_PROP);
|
||||||
**/
|
}
|
||||||
public HoodieCleaningPolicy getCleanerPolicy() {
|
|
||||||
return HoodieCleaningPolicy
|
|
||||||
.valueOf(props.getProperty(HoodieCompactionConfig.CLEANER_POLICY_PROP));
|
|
||||||
}
|
|
||||||
|
|
||||||
public int getCleanerFileVersionsRetained() {
|
/**
|
||||||
return Integer.parseInt(
|
* compaction properties
|
||||||
props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP));
|
**/
|
||||||
}
|
public HoodieCleaningPolicy getCleanerPolicy() {
|
||||||
|
return HoodieCleaningPolicy
|
||||||
|
.valueOf(props.getProperty(HoodieCompactionConfig.CLEANER_POLICY_PROP));
|
||||||
|
}
|
||||||
|
|
||||||
public int getCleanerCommitsRetained() {
|
public int getCleanerFileVersionsRetained() {
|
||||||
return Integer
|
return Integer.parseInt(
|
||||||
.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
|
props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP));
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getMaxCommitsToKeep() {
|
public int getCleanerCommitsRetained() {
|
||||||
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP));
|
return Integer
|
||||||
}
|
.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
|
||||||
|
}
|
||||||
|
|
||||||
public int getMinCommitsToKeep() {
|
public int getMaxCommitsToKeep() {
|
||||||
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP));
|
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP));
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getParquetSmallFileLimit() {
|
public int getMinCommitsToKeep() {
|
||||||
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT_BYTES));
|
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP));
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getCopyOnWriteInsertSplitSize() {
|
public int getParquetSmallFileLimit() {
|
||||||
return Integer.parseInt(
|
return Integer
|
||||||
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE));
|
.parseInt(props.getProperty(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT_BYTES));
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getCopyOnWriteRecordSizeEstimate() {
|
public int getCopyOnWriteInsertSplitSize() {
|
||||||
return Integer.parseInt(
|
return Integer.parseInt(
|
||||||
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE));
|
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE));
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean shouldAutoTuneInsertSplits() {
|
public int getCopyOnWriteRecordSizeEstimate() {
|
||||||
return Boolean.parseBoolean(
|
return Integer.parseInt(
|
||||||
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS));
|
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE));
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getCleanerParallelism() {
|
public boolean shouldAutoTuneInsertSplits() {
|
||||||
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_PARALLELISM));
|
return Boolean.parseBoolean(
|
||||||
}
|
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS));
|
||||||
|
}
|
||||||
|
|
||||||
public boolean isAutoClean() {
|
public int getCleanerParallelism() {
|
||||||
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.AUTO_CLEAN_PROP));
|
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_PARALLELISM));
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean isInlineCompaction() {
|
public boolean isAutoClean() {
|
||||||
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_PROP));
|
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.AUTO_CLEAN_PROP));
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getInlineCompactDeltaCommitMax() {
|
public boolean isInlineCompaction() {
|
||||||
return Integer.parseInt(
|
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_PROP));
|
||||||
props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP));
|
}
|
||||||
}
|
|
||||||
|
|
||||||
public CompactionStrategy getCompactionStrategy() {
|
public int getInlineCompactDeltaCommitMax() {
|
||||||
return ReflectionUtils.loadClass(props.getProperty(HoodieCompactionConfig.COMPACTION_STRATEGY_PROP));
|
return Integer.parseInt(
|
||||||
}
|
props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP));
|
||||||
|
}
|
||||||
|
|
||||||
public Long getTargetIOPerCompactionInMB() {
|
public CompactionStrategy getCompactionStrategy() {
|
||||||
return Long.parseLong(props.getProperty(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB_PROP));
|
return ReflectionUtils
|
||||||
}
|
.loadClass(props.getProperty(HoodieCompactionConfig.COMPACTION_STRATEGY_PROP));
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
public Long getTargetIOPerCompactionInMB() {
|
||||||
* index properties
|
return Long
|
||||||
**/
|
.parseLong(props.getProperty(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB_PROP));
|
||||||
public HoodieIndex.IndexType getIndexType() {
|
}
|
||||||
return HoodieIndex.IndexType.valueOf(props.getProperty(HoodieIndexConfig.INDEX_TYPE_PROP));
|
|
||||||
}
|
|
||||||
|
|
||||||
public int getBloomFilterNumEntries() {
|
/**
|
||||||
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES));
|
* index properties
|
||||||
}
|
**/
|
||||||
|
public HoodieIndex.IndexType getIndexType() {
|
||||||
|
return HoodieIndex.IndexType.valueOf(props.getProperty(HoodieIndexConfig.INDEX_TYPE_PROP));
|
||||||
|
}
|
||||||
|
|
||||||
public double getBloomFilterFPP() {
|
public int getBloomFilterNumEntries() {
|
||||||
return Double.parseDouble(props.getProperty(HoodieIndexConfig.BLOOM_FILTER_FPP));
|
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES));
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getHbaseZkQuorum() {
|
public double getBloomFilterFPP() {
|
||||||
return props.getProperty(HoodieIndexConfig.HBASE_ZKQUORUM_PROP);
|
return Double.parseDouble(props.getProperty(HoodieIndexConfig.BLOOM_FILTER_FPP));
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getHbaseZkPort() {
|
public String getHbaseZkQuorum() {
|
||||||
return Integer.parseInt(props.getProperty(HoodieIndexConfig.HBASE_ZKPORT_PROP));
|
return props.getProperty(HoodieIndexConfig.HBASE_ZKQUORUM_PROP);
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getHbaseTableName() {
|
public int getHbaseZkPort() {
|
||||||
return props.getProperty(HoodieIndexConfig.HBASE_TABLENAME_PROP);
|
return Integer.parseInt(props.getProperty(HoodieIndexConfig.HBASE_ZKPORT_PROP));
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getBloomIndexParallelism() {
|
public String getHbaseTableName() {
|
||||||
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PARALLELISM_PROP));
|
return props.getProperty(HoodieIndexConfig.HBASE_TABLENAME_PROP);
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean getBloomIndexPruneByRanges() {
|
public int getBloomIndexParallelism() {
|
||||||
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PRUNE_BY_RANGES_PROP));
|
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PARALLELISM_PROP));
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean getBloomIndexUseCaching() {
|
public boolean getBloomIndexPruneByRanges() {
|
||||||
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_USE_CACHING_PROP));
|
return Boolean
|
||||||
}
|
.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PRUNE_BY_RANGES_PROP));
|
||||||
|
}
|
||||||
|
|
||||||
public int getNumBucketsPerPartition() {
|
public boolean getBloomIndexUseCaching() {
|
||||||
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BUCKETED_INDEX_NUM_BUCKETS_PROP));
|
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_USE_CACHING_PROP));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
public int getNumBucketsPerPartition() {
|
||||||
* storage properties
|
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BUCKETED_INDEX_NUM_BUCKETS_PROP));
|
||||||
**/
|
}
|
||||||
public int getParquetMaxFileSize() {
|
|
||||||
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES));
|
|
||||||
}
|
|
||||||
|
|
||||||
public int getParquetBlockSize() {
|
/**
|
||||||
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_BLOCK_SIZE_BYTES));
|
* storage properties
|
||||||
}
|
**/
|
||||||
|
public int getParquetMaxFileSize() {
|
||||||
|
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES));
|
||||||
|
}
|
||||||
|
|
||||||
public int getParquetPageSize() {
|
public int getParquetBlockSize() {
|
||||||
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_PAGE_SIZE_BYTES));
|
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_BLOCK_SIZE_BYTES));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
public int getParquetPageSize() {
|
||||||
* metrics properties
|
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_PAGE_SIZE_BYTES));
|
||||||
**/
|
}
|
||||||
public boolean isMetricsOn() {
|
|
||||||
return Boolean.parseBoolean(props.getProperty(HoodieMetricsConfig.METRICS_ON));
|
|
||||||
}
|
|
||||||
|
|
||||||
public MetricsReporterType getMetricsReporterType() {
|
/**
|
||||||
return MetricsReporterType
|
* metrics properties
|
||||||
.valueOf(props.getProperty(HoodieMetricsConfig.METRICS_REPORTER_TYPE));
|
**/
|
||||||
}
|
public boolean isMetricsOn() {
|
||||||
|
return Boolean.parseBoolean(props.getProperty(HoodieMetricsConfig.METRICS_ON));
|
||||||
|
}
|
||||||
|
|
||||||
public String getGraphiteServerHost() {
|
public MetricsReporterType getMetricsReporterType() {
|
||||||
return props.getProperty(HoodieMetricsConfig.GRAPHITE_SERVER_HOST);
|
return MetricsReporterType
|
||||||
}
|
.valueOf(props.getProperty(HoodieMetricsConfig.METRICS_REPORTER_TYPE));
|
||||||
|
}
|
||||||
|
|
||||||
public int getGraphiteServerPort() {
|
public String getGraphiteServerHost() {
|
||||||
return Integer.parseInt(props.getProperty(HoodieMetricsConfig.GRAPHITE_SERVER_PORT));
|
return props.getProperty(HoodieMetricsConfig.GRAPHITE_SERVER_HOST);
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getGraphiteMetricPrefix() {
|
public int getGraphiteServerPort() {
|
||||||
return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX);
|
return Integer.parseInt(props.getProperty(HoodieMetricsConfig.GRAPHITE_SERVER_PORT));
|
||||||
}
|
}
|
||||||
|
|
||||||
public static HoodieWriteConfig.Builder newBuilder() {
|
public String getGraphiteMetricPrefix() {
|
||||||
return new Builder();
|
return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static HoodieWriteConfig.Builder newBuilder() {
|
||||||
|
return new Builder();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
public static class Builder {
|
public static class Builder {
|
||||||
private final Properties props = new Properties();
|
|
||||||
private boolean isIndexConfigSet = false;
|
|
||||||
private boolean isStorageConfigSet = false;
|
|
||||||
private boolean isCompactionConfigSet = false;
|
|
||||||
private boolean isMetricsConfigSet = false;
|
|
||||||
private boolean isAutoCommit = true;
|
|
||||||
|
|
||||||
public Builder fromFile(File propertiesFile) throws IOException {
|
private final Properties props = new Properties();
|
||||||
FileReader reader = new FileReader(propertiesFile);
|
private boolean isIndexConfigSet = false;
|
||||||
try {
|
private boolean isStorageConfigSet = false;
|
||||||
this.props.load(reader);
|
private boolean isCompactionConfigSet = false;
|
||||||
return this;
|
private boolean isMetricsConfigSet = false;
|
||||||
} finally {
|
private boolean isAutoCommit = true;
|
||||||
reader.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder fromInputStream(InputStream inputStream) throws IOException {
|
public Builder fromFile(File propertiesFile) throws IOException {
|
||||||
try {
|
FileReader reader = new FileReader(propertiesFile);
|
||||||
this.props.load(inputStream);
|
try {
|
||||||
return this;
|
this.props.load(reader);
|
||||||
} finally {
|
return this;
|
||||||
inputStream.close();
|
} finally {
|
||||||
}
|
reader.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder withProps(Map kvprops) {
|
|
||||||
props.putAll(kvprops);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withPath(String basePath) {
|
|
||||||
props.setProperty(BASE_PATH_PROP, basePath);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withSchema(String schemaStr) {
|
|
||||||
props.setProperty(AVRO_SCHEMA, schemaStr);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder forTable(String tableName) {
|
|
||||||
props.setProperty(TABLE_NAME, tableName);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withBulkInsertParallelism(int bulkInsertParallelism) {
|
|
||||||
props.setProperty(BULKINSERT_PARALLELISM, String.valueOf(bulkInsertParallelism));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withParallelism(int insertShuffleParallelism, int upsertShuffleParallelism) {
|
|
||||||
props.setProperty(INSERT_PARALLELISM, String.valueOf(insertShuffleParallelism));
|
|
||||||
props.setProperty(UPSERT_PARALLELISM, String.valueOf(upsertShuffleParallelism));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder combineInput(boolean onInsert, boolean onUpsert) {
|
|
||||||
props.setProperty(COMBINE_BEFORE_INSERT_PROP, String.valueOf(onInsert));
|
|
||||||
props.setProperty(COMBINE_BEFORE_UPSERT_PROP, String.valueOf(onUpsert));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withWriteStatusStorageLevel(String level) {
|
|
||||||
props.setProperty(WRITE_STATUS_STORAGE_LEVEL, level);
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withIndexConfig(HoodieIndexConfig indexConfig) {
|
|
||||||
props.putAll(indexConfig.getProps());
|
|
||||||
isIndexConfigSet = true;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withStorageConfig(HoodieStorageConfig storageConfig) {
|
|
||||||
props.putAll(storageConfig.getProps());
|
|
||||||
isStorageConfigSet = true;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withCompactionConfig(HoodieCompactionConfig compactionConfig) {
|
|
||||||
props.putAll(compactionConfig.getProps());
|
|
||||||
isCompactionConfigSet = true;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withMetricsConfig(HoodieMetricsConfig metricsConfig) {
|
|
||||||
props.putAll(metricsConfig.getProps());
|
|
||||||
isMetricsConfigSet = true;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withAutoCommit(boolean autoCommit) {
|
|
||||||
props.setProperty(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withAssumeDatePartitioning(boolean assumeDatePartitioning) {
|
|
||||||
props.setProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP, String.valueOf(assumeDatePartitioning));
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder withWriteStatusClass(Class<? extends WriteStatus> writeStatusClass) {
|
|
||||||
props.setProperty(HOODIE_WRITE_STATUS_CLASS_PROP, writeStatusClass.getName());
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public HoodieWriteConfig build() {
|
|
||||||
HoodieWriteConfig config = new HoodieWriteConfig(props);
|
|
||||||
// Check for mandatory properties
|
|
||||||
Preconditions.checkArgument(config.getBasePath() != null);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM,
|
|
||||||
DEFAULT_PARALLELISM);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM), BULKINSERT_PARALLELISM,
|
|
||||||
DEFAULT_PARALLELISM);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM,
|
|
||||||
DEFAULT_PARALLELISM);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP),
|
|
||||||
COMBINE_BEFORE_INSERT_PROP, DEFAULT_COMBINE_BEFORE_INSERT);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_UPSERT_PROP),
|
|
||||||
COMBINE_BEFORE_UPSERT_PROP, DEFAULT_COMBINE_BEFORE_UPSERT);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(WRITE_STATUS_STORAGE_LEVEL),
|
|
||||||
WRITE_STATUS_STORAGE_LEVEL, DEFAULT_WRITE_STATUS_STORAGE_LEVEL);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_AUTO_COMMIT_PROP),
|
|
||||||
HOODIE_AUTO_COMMIT_PROP, DEFAULT_HOODIE_AUTO_COMMIT);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_ASSUME_DATE_PARTITIONING_PROP),
|
|
||||||
HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING);
|
|
||||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP),
|
|
||||||
HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS);
|
|
||||||
|
|
||||||
// Make sure the props is propagated
|
|
||||||
setDefaultOnCondition(props, !isIndexConfigSet,
|
|
||||||
HoodieIndexConfig.newBuilder().fromProperties(props).build());
|
|
||||||
setDefaultOnCondition(props, !isStorageConfigSet,
|
|
||||||
HoodieStorageConfig.newBuilder().fromProperties(props).build());
|
|
||||||
setDefaultOnCondition(props, !isCompactionConfigSet,
|
|
||||||
HoodieCompactionConfig.newBuilder().fromProperties(props).build());
|
|
||||||
setDefaultOnCondition(props, !isMetricsConfigSet,
|
|
||||||
HoodieMetricsConfig.newBuilder().fromProperties(props).build());
|
|
||||||
return config;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder fromInputStream(InputStream inputStream) throws IOException {
|
||||||
|
try {
|
||||||
|
this.props.load(inputStream);
|
||||||
|
return this;
|
||||||
|
} finally {
|
||||||
|
inputStream.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withProps(Map kvprops) {
|
||||||
|
props.putAll(kvprops);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withPath(String basePath) {
|
||||||
|
props.setProperty(BASE_PATH_PROP, basePath);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withSchema(String schemaStr) {
|
||||||
|
props.setProperty(AVRO_SCHEMA, schemaStr);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder forTable(String tableName) {
|
||||||
|
props.setProperty(TABLE_NAME, tableName);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withBulkInsertParallelism(int bulkInsertParallelism) {
|
||||||
|
props.setProperty(BULKINSERT_PARALLELISM, String.valueOf(bulkInsertParallelism));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withParallelism(int insertShuffleParallelism, int upsertShuffleParallelism) {
|
||||||
|
props.setProperty(INSERT_PARALLELISM, String.valueOf(insertShuffleParallelism));
|
||||||
|
props.setProperty(UPSERT_PARALLELISM, String.valueOf(upsertShuffleParallelism));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder combineInput(boolean onInsert, boolean onUpsert) {
|
||||||
|
props.setProperty(COMBINE_BEFORE_INSERT_PROP, String.valueOf(onInsert));
|
||||||
|
props.setProperty(COMBINE_BEFORE_UPSERT_PROP, String.valueOf(onUpsert));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withWriteStatusStorageLevel(String level) {
|
||||||
|
props.setProperty(WRITE_STATUS_STORAGE_LEVEL, level);
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withIndexConfig(HoodieIndexConfig indexConfig) {
|
||||||
|
props.putAll(indexConfig.getProps());
|
||||||
|
isIndexConfigSet = true;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withStorageConfig(HoodieStorageConfig storageConfig) {
|
||||||
|
props.putAll(storageConfig.getProps());
|
||||||
|
isStorageConfigSet = true;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withCompactionConfig(HoodieCompactionConfig compactionConfig) {
|
||||||
|
props.putAll(compactionConfig.getProps());
|
||||||
|
isCompactionConfigSet = true;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withMetricsConfig(HoodieMetricsConfig metricsConfig) {
|
||||||
|
props.putAll(metricsConfig.getProps());
|
||||||
|
isMetricsConfigSet = true;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withAutoCommit(boolean autoCommit) {
|
||||||
|
props.setProperty(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withAssumeDatePartitioning(boolean assumeDatePartitioning) {
|
||||||
|
props.setProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP,
|
||||||
|
String.valueOf(assumeDatePartitioning));
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Builder withWriteStatusClass(Class<? extends WriteStatus> writeStatusClass) {
|
||||||
|
props.setProperty(HOODIE_WRITE_STATUS_CLASS_PROP, writeStatusClass.getName());
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieWriteConfig build() {
|
||||||
|
HoodieWriteConfig config = new HoodieWriteConfig(props);
|
||||||
|
// Check for mandatory properties
|
||||||
|
Preconditions.checkArgument(config.getBasePath() != null);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM,
|
||||||
|
DEFAULT_PARALLELISM);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM),
|
||||||
|
BULKINSERT_PARALLELISM,
|
||||||
|
DEFAULT_PARALLELISM);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM,
|
||||||
|
DEFAULT_PARALLELISM);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP),
|
||||||
|
COMBINE_BEFORE_INSERT_PROP, DEFAULT_COMBINE_BEFORE_INSERT);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_UPSERT_PROP),
|
||||||
|
COMBINE_BEFORE_UPSERT_PROP, DEFAULT_COMBINE_BEFORE_UPSERT);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(WRITE_STATUS_STORAGE_LEVEL),
|
||||||
|
WRITE_STATUS_STORAGE_LEVEL, DEFAULT_WRITE_STATUS_STORAGE_LEVEL);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(HOODIE_AUTO_COMMIT_PROP),
|
||||||
|
HOODIE_AUTO_COMMIT_PROP, DEFAULT_HOODIE_AUTO_COMMIT);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(HOODIE_ASSUME_DATE_PARTITIONING_PROP),
|
||||||
|
HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING);
|
||||||
|
setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP),
|
||||||
|
HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS);
|
||||||
|
|
||||||
|
// Make sure the props is propagated
|
||||||
|
setDefaultOnCondition(props, !isIndexConfigSet,
|
||||||
|
HoodieIndexConfig.newBuilder().fromProperties(props).build());
|
||||||
|
setDefaultOnCondition(props, !isStorageConfigSet,
|
||||||
|
HoodieStorageConfig.newBuilder().fromProperties(props).build());
|
||||||
|
setDefaultOnCondition(props, !isCompactionConfigSet,
|
||||||
|
HoodieCompactionConfig.newBuilder().fromProperties(props).build());
|
||||||
|
setDefaultOnCondition(props, !isMetricsConfigSet,
|
||||||
|
HoodieMetricsConfig.newBuilder().fromProperties(props).build());
|
||||||
|
return config;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,16 +17,16 @@
|
|||||||
package com.uber.hoodie.exception;
|
package com.uber.hoodie.exception;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>
|
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a delta
|
||||||
* Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a delta commit
|
* commit </p>
|
||||||
* </p>
|
|
||||||
*/
|
*/
|
||||||
public class HoodieAppendException extends HoodieException {
|
public class HoodieAppendException extends HoodieException {
|
||||||
public HoodieAppendException(String msg, Throwable e) {
|
|
||||||
super(msg, e);
|
|
||||||
}
|
|
||||||
|
|
||||||
public HoodieAppendException(String msg) {
|
public HoodieAppendException(String msg, Throwable e) {
|
||||||
super(msg);
|
super(msg, e);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public HoodieAppendException(String msg) {
|
||||||
|
super(msg);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,16 +17,16 @@
|
|||||||
package com.uber.hoodie.exception;
|
package com.uber.hoodie.exception;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>
|
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a Commit
|
||||||
* Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a Commit
|
|
||||||
* </p>
|
* </p>
|
||||||
*/
|
*/
|
||||||
public class HoodieCommitException extends HoodieException {
|
public class HoodieCommitException extends HoodieException {
|
||||||
public HoodieCommitException(String msg) {
|
|
||||||
super(msg);
|
|
||||||
}
|
|
||||||
|
|
||||||
public HoodieCommitException(String msg, Throwable e) {
|
public HoodieCommitException(String msg) {
|
||||||
super(msg, e);
|
super(msg);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public HoodieCommitException(String msg, Throwable e) {
|
||||||
|
super(msg, e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,6 +17,7 @@
|
|||||||
package com.uber.hoodie.exception;
|
package com.uber.hoodie.exception;
|
||||||
|
|
||||||
public class HoodieCompactionException extends HoodieException {
|
public class HoodieCompactionException extends HoodieException {
|
||||||
|
|
||||||
public HoodieCompactionException(String msg) {
|
public HoodieCompactionException(String msg) {
|
||||||
super(msg);
|
super(msg);
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,18 +18,17 @@ package com.uber.hoodie.exception;
|
|||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>
|
* <p> Exception thrown when dependent system is not available </p>
|
||||||
* Exception thrown when dependent system is not available
|
|
||||||
* </p>
|
|
||||||
*/
|
*/
|
||||||
public class HoodieDependentSystemUnavailableException extends HoodieException {
|
public class HoodieDependentSystemUnavailableException extends HoodieException {
|
||||||
public static final String HBASE = "HBASE";
|
|
||||||
|
|
||||||
public HoodieDependentSystemUnavailableException(String system, String connectURL) {
|
public static final String HBASE = "HBASE";
|
||||||
super(getLogMessage(system, connectURL));
|
|
||||||
}
|
|
||||||
|
|
||||||
private static String getLogMessage(String system, String connectURL) {
|
public HoodieDependentSystemUnavailableException(String system, String connectURL) {
|
||||||
return "System " + system + " unavailable. Tried to connect to " + connectURL;
|
super(getLogMessage(system, connectURL));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static String getLogMessage(String system, String connectURL) {
|
||||||
|
return "System " + system + " unavailable. Tried to connect to " + connectURL;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,15 +16,13 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.exception;
|
package com.uber.hoodie.exception;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>
|
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a bulk
|
||||||
* Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a bulk insert
|
* insert </p>
|
||||||
* </p>
|
|
||||||
*/
|
*/
|
||||||
public class HoodieInsertException extends HoodieException {
|
public class HoodieInsertException extends HoodieException {
|
||||||
public HoodieInsertException(String msg, Throwable e) {
|
|
||||||
super(msg, e);
|
public HoodieInsertException(String msg, Throwable e) {
|
||||||
}
|
super(msg, e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,11 +18,11 @@ package com.uber.hoodie.exception;
|
|||||||
|
|
||||||
public class HoodieRollbackException extends HoodieException {
|
public class HoodieRollbackException extends HoodieException {
|
||||||
|
|
||||||
public HoodieRollbackException(String msg, Throwable e) {
|
public HoodieRollbackException(String msg, Throwable e) {
|
||||||
super(msg, e);
|
super(msg, e);
|
||||||
}
|
}
|
||||||
|
|
||||||
public HoodieRollbackException(String msg) {
|
public HoodieRollbackException(String msg) {
|
||||||
super(msg);
|
super(msg);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,11 +18,11 @@ package com.uber.hoodie.exception;
|
|||||||
|
|
||||||
public class HoodieSavepointException extends HoodieException {
|
public class HoodieSavepointException extends HoodieException {
|
||||||
|
|
||||||
public HoodieSavepointException(String msg, Throwable e) {
|
public HoodieSavepointException(String msg, Throwable e) {
|
||||||
super(msg, e);
|
super(msg, e);
|
||||||
}
|
}
|
||||||
|
|
||||||
public HoodieSavepointException(String msg) {
|
public HoodieSavepointException(String msg) {
|
||||||
super(msg);
|
super(msg);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,16 +17,16 @@
|
|||||||
package com.uber.hoodie.exception;
|
package com.uber.hoodie.exception;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>
|
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a
|
||||||
* Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a incremental upsert
|
* incremental upsert </p>
|
||||||
* </p>
|
|
||||||
*/
|
*/
|
||||||
public class HoodieUpsertException extends HoodieException {
|
public class HoodieUpsertException extends HoodieException {
|
||||||
public HoodieUpsertException(String msg, Throwable e) {
|
|
||||||
super(msg, e);
|
|
||||||
}
|
|
||||||
|
|
||||||
public HoodieUpsertException(String msg) {
|
public HoodieUpsertException(String msg, Throwable e) {
|
||||||
super(msg);
|
super(msg, e);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public HoodieUpsertException(String msg) {
|
||||||
|
super(msg);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,16 +16,14 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.func;
|
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.WriteStatus;
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||||
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
import org.apache.spark.api.java.function.Function2;
|
|
||||||
|
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import org.apache.spark.api.java.function.Function2;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -34,20 +32,21 @@ import java.util.List;
|
|||||||
public class BulkInsertMapFunction<T extends HoodieRecordPayload>
|
public class BulkInsertMapFunction<T extends HoodieRecordPayload>
|
||||||
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>> {
|
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>> {
|
||||||
|
|
||||||
private String commitTime;
|
private String commitTime;
|
||||||
private HoodieWriteConfig config;
|
private HoodieWriteConfig config;
|
||||||
private HoodieTable<T> hoodieTable;
|
private HoodieTable<T> hoodieTable;
|
||||||
|
|
||||||
public BulkInsertMapFunction(String commitTime, HoodieWriteConfig config,
|
public BulkInsertMapFunction(String commitTime, HoodieWriteConfig config,
|
||||||
HoodieTable<T> hoodieTable) {
|
HoodieTable<T> hoodieTable) {
|
||||||
this.commitTime = commitTime;
|
this.commitTime = commitTime;
|
||||||
this.config = config;
|
this.config = config;
|
||||||
this.hoodieTable = hoodieTable;
|
this.hoodieTable = hoodieTable;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<List<WriteStatus>> call(Integer partition, Iterator<HoodieRecord<T>> sortedRecordItr)
|
public Iterator<List<WriteStatus>> call(Integer partition,
|
||||||
throws Exception {
|
Iterator<HoodieRecord<T>> sortedRecordItr)
|
||||||
return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable);
|
throws Exception {
|
||||||
}
|
return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,99 +16,101 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.func;
|
package com.uber.hoodie.func;
|
||||||
|
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
|
||||||
import com.uber.hoodie.WriteStatus;
|
import com.uber.hoodie.WriteStatus;
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||||
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.io.HoodieIOHandle;
|
|
||||||
import com.uber.hoodie.io.HoodieCreateHandle;
|
import com.uber.hoodie.io.HoodieCreateHandle;
|
||||||
|
import com.uber.hoodie.io.HoodieIOHandle;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
import org.apache.spark.TaskContext;
|
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import org.apache.spark.TaskContext;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath,
|
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
|
||||||
* into new files.
|
* files.
|
||||||
*/
|
*/
|
||||||
public class LazyInsertIterable<T extends HoodieRecordPayload> extends LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> {
|
public class LazyInsertIterable<T extends HoodieRecordPayload> extends
|
||||||
|
LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> {
|
||||||
|
|
||||||
private final HoodieWriteConfig hoodieConfig;
|
private final HoodieWriteConfig hoodieConfig;
|
||||||
private final String commitTime;
|
private final String commitTime;
|
||||||
private final HoodieTable<T> hoodieTable;
|
private final HoodieTable<T> hoodieTable;
|
||||||
private Set<String> partitionsCleaned;
|
private Set<String> partitionsCleaned;
|
||||||
private HoodieCreateHandle handle;
|
private HoodieCreateHandle handle;
|
||||||
|
|
||||||
public LazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
|
public LazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
|
||||||
String commitTime, HoodieTable<T> hoodieTable) {
|
String commitTime, HoodieTable<T> hoodieTable) {
|
||||||
super(sortedRecordItr);
|
super(sortedRecordItr);
|
||||||
this.partitionsCleaned = new HashSet<>();
|
this.partitionsCleaned = new HashSet<>();
|
||||||
this.hoodieConfig = config;
|
this.hoodieConfig = config;
|
||||||
this.commitTime = commitTime;
|
this.commitTime = commitTime;
|
||||||
this.hoodieTable = hoodieTable;
|
this.hoodieTable = hoodieTable;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void start() {
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected List<WriteStatus> computeNext() {
|
||||||
|
List<WriteStatus> statuses = new ArrayList<>();
|
||||||
|
|
||||||
|
while (inputItr.hasNext()) {
|
||||||
|
HoodieRecord record = inputItr.next();
|
||||||
|
|
||||||
|
// clean up any partial failures
|
||||||
|
if (!partitionsCleaned.contains(record.getPartitionPath())) {
|
||||||
|
// This insert task could fail multiple times, but Spark will faithfully retry with
|
||||||
|
// the same data again. Thus, before we open any files under a given partition, we
|
||||||
|
// first delete any files in the same partitionPath written by same Spark partition
|
||||||
|
HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig,
|
||||||
|
commitTime,
|
||||||
|
record.getPartitionPath(),
|
||||||
|
TaskContext.getPartitionId());
|
||||||
|
partitionsCleaned.add(record.getPartitionPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
// lazily initialize the handle, for the first time
|
||||||
|
if (handle == null) {
|
||||||
|
handle =
|
||||||
|
new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable,
|
||||||
|
record.getPartitionPath());
|
||||||
|
}
|
||||||
|
|
||||||
|
if (handle.canWrite(record)) {
|
||||||
|
// write the record, if the handle has capacity
|
||||||
|
handle.write(record);
|
||||||
|
} else {
|
||||||
|
// handle is full.
|
||||||
|
statuses.add(handle.close());
|
||||||
|
// Need to handle the rejected record & open new handle
|
||||||
|
handle =
|
||||||
|
new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable,
|
||||||
|
record.getPartitionPath());
|
||||||
|
handle.write(record); // we should be able to write 1 record.
|
||||||
|
break;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override protected void start() {
|
// If we exited out, because we ran out of records, just close the pending handle.
|
||||||
|
if (!inputItr.hasNext()) {
|
||||||
|
if (handle != null) {
|
||||||
|
statuses.add(handle.close());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
assert statuses.size() > 0; // should never return empty statuses
|
||||||
|
return statuses;
|
||||||
|
}
|
||||||
|
|
||||||
@Override protected List<WriteStatus> computeNext() {
|
@Override
|
||||||
List<WriteStatus> statuses = new ArrayList<>();
|
protected void end() {
|
||||||
|
|
||||||
while (inputItr.hasNext()) {
|
}
|
||||||
HoodieRecord record = inputItr.next();
|
|
||||||
|
|
||||||
// clean up any partial failures
|
|
||||||
if (!partitionsCleaned.contains(record.getPartitionPath())) {
|
|
||||||
// This insert task could fail multiple times, but Spark will faithfully retry with
|
|
||||||
// the same data again. Thus, before we open any files under a given partition, we
|
|
||||||
// first delete any files in the same partitionPath written by same Spark partition
|
|
||||||
HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig,
|
|
||||||
commitTime,
|
|
||||||
record.getPartitionPath(),
|
|
||||||
TaskContext.getPartitionId());
|
|
||||||
partitionsCleaned.add(record.getPartitionPath());
|
|
||||||
}
|
|
||||||
|
|
||||||
// lazily initialize the handle, for the first time
|
|
||||||
if (handle == null) {
|
|
||||||
handle =
|
|
||||||
new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable,
|
|
||||||
record.getPartitionPath());
|
|
||||||
}
|
|
||||||
|
|
||||||
if (handle.canWrite(record)) {
|
|
||||||
// write the record, if the handle has capacity
|
|
||||||
handle.write(record);
|
|
||||||
} else {
|
|
||||||
// handle is full.
|
|
||||||
statuses.add(handle.close());
|
|
||||||
// Need to handle the rejected record & open new handle
|
|
||||||
handle =
|
|
||||||
new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable,
|
|
||||||
record.getPartitionPath());
|
|
||||||
handle.write(record); // we should be able to write 1 record.
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// If we exited out, because we ran out of records, just close the pending handle.
|
|
||||||
if (!inputItr.hasNext()) {
|
|
||||||
if (handle != null) {
|
|
||||||
statuses.add(handle.close());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
assert statuses.size() > 0; // should never return empty statuses
|
|
||||||
return statuses;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override protected void end() {
|
|
||||||
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -31,98 +31,99 @@ import java.util.Iterator;
|
|||||||
* responsible for calling inputIterator.next() and doing the processing in computeNext()
|
* responsible for calling inputIterator.next() and doing the processing in computeNext()
|
||||||
*/
|
*/
|
||||||
public abstract class LazyIterableIterator<I, O> implements Iterable<O>, Iterator<O> {
|
public abstract class LazyIterableIterator<I, O> implements Iterable<O>, Iterator<O> {
|
||||||
protected Iterator<I> inputItr = null;
|
|
||||||
private boolean consumed = false;
|
|
||||||
private boolean startCalled = false;
|
|
||||||
private boolean endCalled = false;
|
|
||||||
|
|
||||||
public LazyIterableIterator(Iterator<I> in) {
|
protected Iterator<I> inputItr = null;
|
||||||
inputItr = in;
|
private boolean consumed = false;
|
||||||
|
private boolean startCalled = false;
|
||||||
|
private boolean endCalled = false;
|
||||||
|
|
||||||
|
public LazyIterableIterator(Iterator<I> in) {
|
||||||
|
inputItr = in;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Called once, before any elements are processed
|
||||||
|
*/
|
||||||
|
protected abstract void start();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Block computation to be overwritten by sub classes.
|
||||||
|
*/
|
||||||
|
protected abstract O computeNext();
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Called once, after all elements are processed.
|
||||||
|
*/
|
||||||
|
protected abstract void end();
|
||||||
|
|
||||||
|
//////////////////
|
||||||
|
// iterable implementation
|
||||||
|
|
||||||
|
private void invokeStartIfNeeded() {
|
||||||
|
if (!startCalled) {
|
||||||
|
startCalled = true;
|
||||||
|
try {
|
||||||
|
start();
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException("Error in start()");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void invokeEndIfNeeded() {
|
||||||
|
// make the calls out to begin() & end()
|
||||||
|
if (!endCalled) {
|
||||||
|
endCalled = true;
|
||||||
|
// if we are out of elements, and end has not been called yet
|
||||||
|
try {
|
||||||
|
end();
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException("Error in end()");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Iterator<O> iterator() {
|
||||||
|
//check for consumed inputItr
|
||||||
|
if (consumed) {
|
||||||
|
throw new RuntimeException("Invalid repeated inputItr consumption.");
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
//hand out self as inputItr exactly once (note: do not hand out the input
|
||||||
* Called once, before any elements are processed
|
//inputItr since it is consumed by the self inputItr implementation)
|
||||||
*/
|
consumed = true;
|
||||||
protected abstract void start();
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
//////////////////
|
||||||
* Block computation to be overwritten by sub classes.
|
// inputItr implementation
|
||||||
*/
|
|
||||||
protected abstract O computeNext();
|
|
||||||
|
|
||||||
|
@Override
|
||||||
/**
|
public boolean hasNext() {
|
||||||
* Called once, after all elements are processed.
|
boolean ret = inputItr.hasNext();
|
||||||
*/
|
// make sure, there is exactly one call to start()
|
||||||
protected abstract void end();
|
invokeStartIfNeeded();
|
||||||
|
if (!ret) {
|
||||||
|
// if we are out of elements, and end has not been called yet
|
||||||
//////////////////
|
invokeEndIfNeeded();
|
||||||
// iterable implementation
|
|
||||||
|
|
||||||
private void invokeStartIfNeeded() {
|
|
||||||
if (!startCalled) {
|
|
||||||
startCalled = true;
|
|
||||||
try {
|
|
||||||
start();
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new RuntimeException("Error in start()");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private void invokeEndIfNeeded() {
|
return ret;
|
||||||
// make the calls out to begin() & end()
|
}
|
||||||
if (!endCalled) {
|
|
||||||
endCalled = true;
|
@Override
|
||||||
// if we are out of elements, and end has not been called yet
|
public O next() {
|
||||||
try {
|
try {
|
||||||
end();
|
return computeNext();
|
||||||
} catch (Exception e) {
|
} catch (Exception ex) {
|
||||||
throw new RuntimeException("Error in end()");
|
throw new RuntimeException(ex);
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<O> iterator() {
|
public void remove() {
|
||||||
//check for consumed inputItr
|
throw new RuntimeException("Unsupported remove operation.");
|
||||||
if (consumed)
|
}
|
||||||
throw new RuntimeException("Invalid repeated inputItr consumption.");
|
|
||||||
|
|
||||||
//hand out self as inputItr exactly once (note: do not hand out the input
|
|
||||||
//inputItr since it is consumed by the self inputItr implementation)
|
|
||||||
consumed = true;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
//////////////////
|
|
||||||
// inputItr implementation
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean hasNext() {
|
|
||||||
boolean ret = inputItr.hasNext();
|
|
||||||
// make sure, there is exactly one call to start()
|
|
||||||
invokeStartIfNeeded();
|
|
||||||
if (!ret) {
|
|
||||||
// if we are out of elements, and end has not been called yet
|
|
||||||
invokeEndIfNeeded();
|
|
||||||
}
|
|
||||||
|
|
||||||
return ret;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public O next() {
|
|
||||||
try {
|
|
||||||
return computeNext();
|
|
||||||
} catch (Exception ex) {
|
|
||||||
throw new RuntimeException(ex);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void remove() {
|
|
||||||
throw new RuntimeException("Unsupported remove operation.");
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,118 +17,108 @@
|
|||||||
package com.uber.hoodie.index;
|
package com.uber.hoodie.index;
|
||||||
|
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
|
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
|
||||||
import com.uber.hoodie.WriteStatus;
|
import com.uber.hoodie.WriteStatus;
|
||||||
import com.uber.hoodie.common.model.HoodieKey;
|
import com.uber.hoodie.common.model.HoodieKey;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
|
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||||
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.exception.HoodieIndexException;
|
import com.uber.hoodie.exception.HoodieIndexException;
|
||||||
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
|
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
|
||||||
import com.uber.hoodie.index.bucketed.BucketedIndex;
|
import com.uber.hoodie.index.bucketed.BucketedIndex;
|
||||||
import com.uber.hoodie.index.hbase.HBaseIndex;
|
import com.uber.hoodie.index.hbase.HBaseIndex;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
import java.io.Serializable;
|
||||||
import org.apache.spark.api.java.JavaPairRDD;
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
|
||||||
import java.io.Serializable;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Base class for different types of indexes to determine the mapping from uuid
|
* Base class for different types of indexes to determine the mapping from uuid
|
||||||
*
|
|
||||||
*/
|
*/
|
||||||
public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Serializable {
|
public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Serializable {
|
||||||
protected transient JavaSparkContext jsc = null;
|
|
||||||
|
|
||||||
public enum IndexType {
|
protected transient JavaSparkContext jsc = null;
|
||||||
HBASE,
|
|
||||||
INMEMORY,
|
public enum IndexType {
|
||||||
BLOOM,
|
HBASE,
|
||||||
BUCKETED
|
INMEMORY,
|
||||||
}
|
BLOOM,
|
||||||
|
BUCKETED
|
||||||
protected final HoodieWriteConfig config;
|
}
|
||||||
|
|
||||||
protected HoodieIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
protected final HoodieWriteConfig config;
|
||||||
this.config = config;
|
|
||||||
this.jsc = jsc;
|
protected HoodieIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||||
}
|
this.config = config;
|
||||||
|
this.jsc = jsc;
|
||||||
/**
|
}
|
||||||
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Optional[FullFilePath]]
|
|
||||||
* If the optional FullFilePath value is not present, then the key is not found. If the FullFilePath
|
/**
|
||||||
* value is present, it is the path component (without scheme) of the URI underlying file
|
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Optional[FullFilePath]]
|
||||||
*
|
* If the optional FullFilePath value is not present, then the key is not found. If the
|
||||||
* @param hoodieKeys
|
* FullFilePath value is present, it is the path component (without scheme) of the URI underlying
|
||||||
* @param table
|
* file
|
||||||
* @return
|
*/
|
||||||
*/
|
public abstract JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||||
public abstract JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table);
|
||||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table);
|
|
||||||
|
/**
|
||||||
/**
|
* Looks up the index and tags each incoming record with a location of a file that contains the
|
||||||
* Looks up the index and tags each incoming record with a location of a file that contains the
|
* row (if it is actually present)
|
||||||
* row (if it is actually present)
|
*/
|
||||||
*/
|
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||||
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||||
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
|
||||||
|
/**
|
||||||
/**
|
* Extracts the location of written records, and updates the index.
|
||||||
* Extracts the location of written records, and updates the index.
|
*
|
||||||
*
|
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
|
||||||
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
|
*/
|
||||||
*/
|
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||||
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||||
HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
|
||||||
|
/**
|
||||||
/**
|
* Rollback the efffects of the commit made at commitTime.
|
||||||
* Rollback the efffects of the commit made at commitTime.
|
*/
|
||||||
*/
|
public abstract boolean rollbackCommit(String commitTime);
|
||||||
public abstract boolean rollbackCommit(String commitTime);
|
|
||||||
|
/**
|
||||||
/**
|
* An index is `global` if {@link HoodieKey} to fileID mapping, does not depend on the
|
||||||
* An index is `global` if {@link HoodieKey} to fileID mapping, does not depend on the `partitionPath`.
|
* `partitionPath`. Such an implementation is able to obtain the same mapping, for two hoodie keys
|
||||||
* Such an implementation is able to obtain the same mapping, for two hoodie keys with same `recordKey`
|
* with same `recordKey` but different `partitionPath`
|
||||||
* but different `partitionPath`
|
*
|
||||||
*
|
* @return whether or not, the index implementation is global in nature
|
||||||
* @return whether or not, the index implementation is global in nature
|
*/
|
||||||
*/
|
public abstract boolean isGlobal();
|
||||||
public abstract boolean isGlobal();
|
|
||||||
|
/**
|
||||||
/**
|
* This is used by storage to determine, if its safe to send inserts, straight to the log, i.e
|
||||||
* This is used by storage to determine, if its safe to send inserts, straight to the log,
|
* having a {@link com.uber.hoodie.common.model.FileSlice}, with no data file.
|
||||||
* i.e having a {@link com.uber.hoodie.common.model.FileSlice}, with no data file.
|
*
|
||||||
*
|
* @return Returns true/false depending on whether the impl has this capability
|
||||||
* @return Returns true/false depending on whether the impl has this capability
|
*/
|
||||||
*/
|
public abstract boolean canIndexLogFiles();
|
||||||
public abstract boolean canIndexLogFiles();
|
|
||||||
|
|
||||||
|
/**
|
||||||
/**
|
* An index is "implicit" with respect to storage, if just writing new data to a file slice,
|
||||||
*
|
* updates the index as well. This is used by storage, to save memory footprint in certain cases.
|
||||||
* An index is "implicit" with respect to storage, if just writing new data to a file slice,
|
*/
|
||||||
* updates the index as well. This is used by storage, to save memory footprint in
|
public abstract boolean isImplicitWithStorage();
|
||||||
* certain cases.
|
|
||||||
*
|
|
||||||
* @return
|
public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(
|
||||||
*/
|
HoodieWriteConfig config, JavaSparkContext jsc) throws HoodieIndexException {
|
||||||
public abstract boolean isImplicitWithStorage();
|
switch (config.getIndexType()) {
|
||||||
|
case HBASE:
|
||||||
|
return new HBaseIndex<>(config, jsc);
|
||||||
public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(
|
case INMEMORY:
|
||||||
HoodieWriteConfig config, JavaSparkContext jsc) throws HoodieIndexException {
|
return new InMemoryHashIndex<>(config, jsc);
|
||||||
switch (config.getIndexType()) {
|
case BLOOM:
|
||||||
case HBASE:
|
return new HoodieBloomIndex<>(config, jsc);
|
||||||
return new HBaseIndex<>(config, jsc);
|
case BUCKETED:
|
||||||
case INMEMORY:
|
return new BucketedIndex<>(config, jsc);
|
||||||
return new InMemoryHashIndex<>(config, jsc);
|
|
||||||
case BLOOM:
|
|
||||||
return new HoodieBloomIndex<>(config, jsc);
|
|
||||||
case BUCKETED:
|
|
||||||
return new BucketedIndex<>(config, jsc);
|
|
||||||
}
|
|
||||||
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
|
|
||||||
}
|
}
|
||||||
|
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,129 +17,119 @@
|
|||||||
package com.uber.hoodie.index;
|
package com.uber.hoodie.index;
|
||||||
|
|
||||||
import com.google.common.base.Optional;
|
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.WriteStatus;
|
||||||
import com.uber.hoodie.common.model.HoodieKey;
|
import com.uber.hoodie.common.model.HoodieKey;
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||||
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import org.apache.spark.api.java.JavaPairRDD;
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.apache.spark.api.java.function.Function;
|
import org.apache.spark.api.java.function.Function;
|
||||||
import org.apache.spark.api.java.function.Function2;
|
import org.apache.spark.api.java.function.Function2;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
|
||||||
import java.util.concurrent.ConcurrentMap;
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Hoodie Index implementation backed by an in-memory Hash map.
|
* Hoodie Index implementation backed by an in-memory Hash map. <p> ONLY USE FOR LOCAL TESTING
|
||||||
* <p>
|
|
||||||
* ONLY USE FOR LOCAL TESTING
|
|
||||||
*/
|
*/
|
||||||
public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||||
|
|
||||||
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
|
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
|
||||||
|
|
||||||
public InMemoryHashIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
public InMemoryHashIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||||
super(config, jsc);
|
super(config, jsc);
|
||||||
recordLocationMap = new ConcurrentHashMap<>();
|
recordLocationMap = new ConcurrentHashMap<>();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||||
|
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table) {
|
||||||
|
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Function that tags each HoodieRecord with an existing location, if known.
|
||||||
|
*/
|
||||||
|
class LocationTagFunction
|
||||||
|
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
|
||||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table) {
|
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
|
||||||
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
|
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||||
}
|
while (hoodieRecordIterator.hasNext()) {
|
||||||
|
HoodieRecord<T> rec = hoodieRecordIterator.next();
|
||||||
/**
|
if (recordLocationMap.containsKey(rec.getKey())) {
|
||||||
* Function that tags each HoodieRecord with an existing location, if known.
|
rec.setCurrentLocation(recordLocationMap.get(rec.getKey()));
|
||||||
*/
|
|
||||||
class LocationTagFunction
|
|
||||||
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
|
|
||||||
@Override
|
|
||||||
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
|
|
||||||
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
|
|
||||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
|
||||||
while (hoodieRecordIterator.hasNext()) {
|
|
||||||
HoodieRecord<T> rec = hoodieRecordIterator.next();
|
|
||||||
if (recordLocationMap.containsKey(rec.getKey())) {
|
|
||||||
rec.setCurrentLocation(recordLocationMap.get(rec.getKey()));
|
|
||||||
}
|
|
||||||
taggedRecords.add(rec);
|
|
||||||
}
|
|
||||||
return taggedRecords.iterator();
|
|
||||||
}
|
}
|
||||||
|
taggedRecords.add(rec);
|
||||||
|
}
|
||||||
|
return taggedRecords.iterator();
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||||
HoodieTable<T> hoodieTable) {
|
HoodieTable<T> hoodieTable) {
|
||||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
|
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||||
HoodieTable<T> hoodieTable) {
|
HoodieTable<T> hoodieTable) {
|
||||||
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
|
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
|
||||||
@Override
|
@Override
|
||||||
public WriteStatus call(WriteStatus writeStatus) {
|
public WriteStatus call(WriteStatus writeStatus) {
|
||||||
for (HoodieRecord record : writeStatus.getWrittenRecords()) {
|
for (HoodieRecord record : writeStatus.getWrittenRecords()) {
|
||||||
if (!writeStatus.isErrored(record.getKey())) {
|
if (!writeStatus.isErrored(record.getKey())) {
|
||||||
HoodieKey key = record.getKey();
|
HoodieKey key = record.getKey();
|
||||||
java.util.Optional<HoodieRecordLocation> newLocation = record.getNewLocation();
|
java.util.Optional<HoodieRecordLocation> newLocation = record.getNewLocation();
|
||||||
if (newLocation.isPresent()) {
|
if (newLocation.isPresent()) {
|
||||||
recordLocationMap.put(key, newLocation.get());
|
recordLocationMap.put(key, newLocation.get());
|
||||||
} else {
|
} else {
|
||||||
//Delete existing index for a deleted record
|
//Delete existing index for a deleted record
|
||||||
recordLocationMap.remove(key);
|
recordLocationMap.remove(key);
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return writeStatus;
|
|
||||||
}
|
}
|
||||||
});
|
}
|
||||||
}
|
}
|
||||||
|
return writeStatus;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean rollbackCommit(String commitTime) {
|
public boolean rollbackCommit(String commitTime) {
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Only looks up by recordKey
|
* Only looks up by recordKey
|
||||||
*
|
*/
|
||||||
* @return
|
@Override
|
||||||
*/
|
public boolean isGlobal() {
|
||||||
@Override
|
return true;
|
||||||
public boolean isGlobal() {
|
}
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Mapping is available in HBase already.
|
* Mapping is available in HBase already.
|
||||||
*
|
*/
|
||||||
* @return
|
@Override
|
||||||
*/
|
public boolean canIndexLogFiles() {
|
||||||
@Override
|
return true;
|
||||||
public boolean canIndexLogFiles() {
|
}
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Index needs to be explicitly updated after storage write.
|
* Index needs to be explicitly updated after storage write.
|
||||||
*
|
*/
|
||||||
* @return
|
@Override
|
||||||
*/
|
public boolean isImplicitWithStorage() {
|
||||||
@Override
|
return false;
|
||||||
public boolean isImplicitWithStorage() {
|
}
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -19,7 +19,6 @@
|
|||||||
package com.uber.hoodie.index.bloom;
|
package com.uber.hoodie.index.bloom;
|
||||||
|
|
||||||
import com.google.common.base.Objects;
|
import com.google.common.base.Objects;
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@@ -27,73 +26,75 @@ import java.io.Serializable;
|
|||||||
*/
|
*/
|
||||||
public class BloomIndexFileInfo implements Serializable {
|
public class BloomIndexFileInfo implements Serializable {
|
||||||
|
|
||||||
private final String fileName;
|
private final String fileName;
|
||||||
|
|
||||||
private final String minRecordKey;
|
private final String minRecordKey;
|
||||||
|
|
||||||
private final String maxRecordKey;
|
private final String maxRecordKey;
|
||||||
|
|
||||||
public BloomIndexFileInfo(String fileName, String minRecordKey, String maxRecordKey) {
|
public BloomIndexFileInfo(String fileName, String minRecordKey, String maxRecordKey) {
|
||||||
this.fileName = fileName;
|
this.fileName = fileName;
|
||||||
this.minRecordKey = minRecordKey;
|
this.minRecordKey = minRecordKey;
|
||||||
this.maxRecordKey = maxRecordKey;
|
this.maxRecordKey = maxRecordKey;
|
||||||
|
}
|
||||||
|
|
||||||
|
public BloomIndexFileInfo(String fileName) {
|
||||||
|
this.fileName = fileName;
|
||||||
|
this.minRecordKey = null;
|
||||||
|
this.maxRecordKey = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getFileName() {
|
||||||
|
return fileName;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getMinRecordKey() {
|
||||||
|
return minRecordKey;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getMaxRecordKey() {
|
||||||
|
return maxRecordKey;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean hasKeyRanges() {
|
||||||
|
return minRecordKey != null && maxRecordKey != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Does the given key fall within the range (inclusive)
|
||||||
|
*/
|
||||||
|
public boolean isKeyInRange(String recordKey) {
|
||||||
|
return minRecordKey.compareTo(recordKey) <= 0 &&
|
||||||
|
maxRecordKey.compareTo(recordKey) >= 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
if (o == null || getClass() != o.getClass()) {
|
||||||
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
public BloomIndexFileInfo(String fileName) {
|
BloomIndexFileInfo that = (BloomIndexFileInfo) o;
|
||||||
this.fileName = fileName;
|
return Objects.equal(that.fileName, fileName) &&
|
||||||
this.minRecordKey = null;
|
Objects.equal(that.minRecordKey, minRecordKey) &&
|
||||||
this.maxRecordKey = null;
|
Objects.equal(that.maxRecordKey, maxRecordKey);
|
||||||
}
|
|
||||||
|
|
||||||
public String getFileName() {
|
}
|
||||||
return fileName;
|
|
||||||
}
|
|
||||||
|
|
||||||
public String getMinRecordKey() {
|
@Override
|
||||||
return minRecordKey;
|
public int hashCode() {
|
||||||
}
|
return Objects.hashCode(fileName, minRecordKey, maxRecordKey);
|
||||||
|
}
|
||||||
|
|
||||||
public String getMaxRecordKey() {
|
public String toString() {
|
||||||
return maxRecordKey;
|
final StringBuilder sb = new StringBuilder("BloomIndexFileInfo {");
|
||||||
}
|
sb.append(" fileName=").append(fileName);
|
||||||
|
sb.append(" minRecordKey=").append(minRecordKey);
|
||||||
public boolean hasKeyRanges() {
|
sb.append(" maxRecordKey=").append(maxRecordKey);
|
||||||
return minRecordKey != null && maxRecordKey != null;
|
sb.append('}');
|
||||||
}
|
return sb.toString();
|
||||||
|
}
|
||||||
/**
|
|
||||||
* Does the given key fall within the range (inclusive)
|
|
||||||
* @param recordKey
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
public boolean isKeyInRange(String recordKey) {
|
|
||||||
return minRecordKey.compareTo(recordKey) <= 0 &&
|
|
||||||
maxRecordKey.compareTo(recordKey) >= 0;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean equals(Object o) {
|
|
||||||
if (this == o) return true;
|
|
||||||
if (o == null || getClass() != o.getClass()) return false;
|
|
||||||
|
|
||||||
BloomIndexFileInfo that = (BloomIndexFileInfo) o;
|
|
||||||
return Objects.equal(that.fileName, fileName) &&
|
|
||||||
Objects.equal(that.minRecordKey, minRecordKey) &&
|
|
||||||
Objects.equal(that.maxRecordKey, maxRecordKey);
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int hashCode() {
|
|
||||||
return Objects.hashCode(fileName, minRecordKey, maxRecordKey);
|
|
||||||
}
|
|
||||||
|
|
||||||
public String toString() {
|
|
||||||
final StringBuilder sb = new StringBuilder("BloomIndexFileInfo {");
|
|
||||||
sb.append(" fileName=").append(fileName);
|
|
||||||
sb.append(" minRecordKey=").append(minRecordKey);
|
|
||||||
sb.append(" maxRecordKey=").append(maxRecordKey);
|
|
||||||
sb.append('}');
|
|
||||||
return sb.toString();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,9 +18,12 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.index.bloom;
|
package com.uber.hoodie.index.bloom;
|
||||||
|
|
||||||
|
import static java.util.stream.Collectors.groupingBy;
|
||||||
|
import static java.util.stream.Collectors.mapping;
|
||||||
|
import static java.util.stream.Collectors.toList;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
|
|
||||||
import com.uber.hoodie.WriteStatus;
|
import com.uber.hoodie.WriteStatus;
|
||||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||||
import com.uber.hoodie.common.model.HoodieKey;
|
import com.uber.hoodie.common.model.HoodieKey;
|
||||||
@@ -34,7 +37,10 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
|||||||
import com.uber.hoodie.exception.MetadataNotFoundException;
|
import com.uber.hoodie.exception.MetadataNotFoundException;
|
||||||
import com.uber.hoodie.index.HoodieIndex;
|
import com.uber.hoodie.index.HoodieIndex;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
@@ -42,369 +48,370 @@ import org.apache.spark.api.java.JavaPairRDD;
|
|||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.apache.spark.storage.StorageLevel;
|
import org.apache.spark.storage.StorageLevel;
|
||||||
|
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
import static java.util.stream.Collectors.*;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in
|
* Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in
|
||||||
* its metadata.
|
* its metadata.
|
||||||
*/
|
*/
|
||||||
public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||||
|
|
||||||
private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class);
|
private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class);
|
||||||
|
|
||||||
// we need to limit the join such that it stays within 1.5GB per Spark partition. (SPARK-1476)
|
// we need to limit the join such that it stays within 1.5GB per Spark partition. (SPARK-1476)
|
||||||
private static final int SPARK_MAXIMUM_BYTES_PER_PARTITION = 1500 * 1024 * 1024;
|
private static final int SPARK_MAXIMUM_BYTES_PER_PARTITION = 1500 * 1024 * 1024;
|
||||||
// this is how much a triplet of (partitionPath, fileId, recordKey) costs.
|
// this is how much a triplet of (partitionPath, fileId, recordKey) costs.
|
||||||
private static final int BYTES_PER_PARTITION_FILE_KEY_TRIPLET = 300;
|
private static final int BYTES_PER_PARTITION_FILE_KEY_TRIPLET = 300;
|
||||||
private static int MAX_ITEMS_PER_SHUFFLE_PARTITION = SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET;
|
private static int MAX_ITEMS_PER_SHUFFLE_PARTITION =
|
||||||
|
SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET;
|
||||||
|
|
||||||
public HoodieBloomIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
public HoodieBloomIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||||
super(config, jsc);
|
super(config, jsc);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||||
|
final HoodieTable<T> hoodieTable) {
|
||||||
|
|
||||||
|
// Step 0: cache the input record RDD
|
||||||
|
if (config.getBloomIndexUseCaching()) {
|
||||||
|
recordRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
// Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
|
||||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, final HoodieTable<T> hoodieTable) {
|
JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD
|
||||||
|
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()));
|
||||||
|
|
||||||
// Step 0: cache the input record RDD
|
// Lookup indexes for all the partition/recordkey pair
|
||||||
if (config.getBloomIndexUseCaching()) {
|
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD,
|
||||||
recordRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
hoodieTable);
|
||||||
}
|
|
||||||
|
|
||||||
// Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
|
// Cache the result, for subsequent stages.
|
||||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD
|
if (config.getBloomIndexUseCaching()) {
|
||||||
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()));
|
rowKeyFilenamePairRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
||||||
|
}
|
||||||
// Lookup indexes for all the partition/recordkey pair
|
if (logger.isDebugEnabled()) {
|
||||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, hoodieTable);
|
long totalTaggedRecords = rowKeyFilenamePairRDD.count();
|
||||||
|
logger.debug("Number of update records (ones tagged with a fileID): " + totalTaggedRecords);
|
||||||
// Cache the result, for subsequent stages.
|
|
||||||
if (config.getBloomIndexUseCaching()) {
|
|
||||||
rowKeyFilenamePairRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
|
||||||
}
|
|
||||||
if (logger.isDebugEnabled()) {
|
|
||||||
long totalTaggedRecords = rowKeyFilenamePairRDD.count();
|
|
||||||
logger.debug("Number of update records (ones tagged with a fileID): " + totalTaggedRecords);
|
|
||||||
}
|
|
||||||
|
|
||||||
// Step 4: Tag the incoming records, as inserts or updates, by joining with existing record keys
|
|
||||||
// Cost: 4 sec.
|
|
||||||
JavaRDD<HoodieRecord<T>> taggedRecordRDD = tagLocationBacktoRecords(rowKeyFilenamePairRDD, recordRDD);
|
|
||||||
|
|
||||||
if (config.getBloomIndexUseCaching()) {
|
|
||||||
recordRDD.unpersist(); // unpersist the input Record RDD
|
|
||||||
rowKeyFilenamePairRDD.unpersist();
|
|
||||||
}
|
|
||||||
|
|
||||||
return taggedRecordRDD;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
// Step 4: Tag the incoming records, as inserts or updates, by joining with existing record keys
|
||||||
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table) {
|
// Cost: 4 sec.
|
||||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
|
JavaRDD<HoodieRecord<T>> taggedRecordRDD = tagLocationBacktoRecords(rowKeyFilenamePairRDD,
|
||||||
hoodieKeys.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
|
recordRDD);
|
||||||
|
|
||||||
// Lookup indexes for all the partition/recordkey pair
|
if (config.getBloomIndexUseCaching()) {
|
||||||
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
|
recordRDD.unpersist(); // unpersist the input Record RDD
|
||||||
lookupIndex(partitionRecordKeyPairRDD, table);
|
rowKeyFilenamePairRDD.unpersist();
|
||||||
|
|
||||||
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD =
|
|
||||||
hoodieKeys.mapToPair(key -> new Tuple2<>(key.getRecordKey(), key));
|
|
||||||
|
|
||||||
return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD)
|
|
||||||
.mapToPair(keyPathTuple -> {
|
|
||||||
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(table.getMetaClient().getBasePath(), partitionPath),
|
|
||||||
fileName).toUri().getPath());
|
|
||||||
} else {
|
|
||||||
recordLocationPath = Optional.absent();
|
|
||||||
}
|
|
||||||
return new Tuple2<>(keyPathTuple._2._1, recordLocationPath);
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
return taggedRecordRDD;
|
||||||
* Lookup the location for each record key and return the pair<record_key,location> for all
|
}
|
||||||
* record keys already present and drop the record keys if not present
|
|
||||||
*/
|
|
||||||
private JavaPairRDD<String, String> lookupIndex(
|
|
||||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTable<T> hoodieTable) {
|
|
||||||
// Obtain records per partition, in the incoming records
|
|
||||||
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
|
|
||||||
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
|
|
||||||
|
|
||||||
// Step 2: Load all involved files as <Partition, filename> pairs
|
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||||
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(affectedPartitionPathList, hoodieTable);
|
JavaRDD<HoodieKey> hoodieKeys, final HoodieTable<T> table) {
|
||||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream()
|
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
|
||||||
.collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
|
hoodieKeys.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
|
||||||
|
|
||||||
// Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, that contains it.
|
// Lookup indexes for all the partition/recordkey pair
|
||||||
int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo, partitionRecordKeyPairRDD);
|
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
|
||||||
return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD, parallelism);
|
lookupIndex(partitionRecordKeyPairRDD, table);
|
||||||
|
|
||||||
|
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD =
|
||||||
|
hoodieKeys.mapToPair(key -> new Tuple2<>(key.getRecordKey(), key));
|
||||||
|
|
||||||
|
return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD)
|
||||||
|
.mapToPair(keyPathTuple -> {
|
||||||
|
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(table.getMetaClient().getBasePath(), partitionPath),
|
||||||
|
fileName).toUri().getPath());
|
||||||
|
} else {
|
||||||
|
recordLocationPath = Optional.absent();
|
||||||
|
}
|
||||||
|
return new Tuple2<>(keyPathTuple._2._1, recordLocationPath);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Lookup the location for each record key and return the pair<record_key,location> for all record
|
||||||
|
* keys already present and drop the record keys if not present
|
||||||
|
*/
|
||||||
|
private JavaPairRDD<String, String> lookupIndex(
|
||||||
|
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTable<T> hoodieTable) {
|
||||||
|
// Obtain records per partition, in the incoming records
|
||||||
|
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
|
||||||
|
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
|
||||||
|
|
||||||
|
// Step 2: Load all involved files as <Partition, filename> pairs
|
||||||
|
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(
|
||||||
|
affectedPartitionPathList, hoodieTable);
|
||||||
|
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream()
|
||||||
|
.collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
|
||||||
|
|
||||||
|
// Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, that contains it.
|
||||||
|
int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo,
|
||||||
|
partitionRecordKeyPairRDD);
|
||||||
|
return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD,
|
||||||
|
parallelism);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The index lookup can be skewed in three dimensions : #files, #partitions, #records
|
||||||
|
*
|
||||||
|
* To be able to smoothly handle skews, we need to compute how to split each partitions into
|
||||||
|
* subpartitions. We do it here, in a way that keeps the amount of each Spark join partition to <
|
||||||
|
* 2GB.
|
||||||
|
*
|
||||||
|
* If {@link com.uber.hoodie.config.HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is specified
|
||||||
|
* as a NON-zero number, then that is used explicitly.
|
||||||
|
*/
|
||||||
|
private int autoComputeParallelism(final Map<String, Long> recordsPerPartition,
|
||||||
|
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
||||||
|
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||||
|
|
||||||
|
long totalComparisons = 0;
|
||||||
|
if (config.getBloomIndexPruneByRanges()) {
|
||||||
|
// we will just try exploding the input and then count to determine comparisons
|
||||||
|
totalComparisons = explodeRecordRDDWithFileComparisons(partitionToFileInfo,
|
||||||
|
partitionRecordKeyPairRDD).count();
|
||||||
|
} else {
|
||||||
|
// if not pruning by ranges, then each file in a partition needs to compared against all
|
||||||
|
// records for a partition.
|
||||||
|
Map<String, Long> filesPerPartition = partitionToFileInfo.entrySet().stream()
|
||||||
|
.collect(Collectors.toMap(Map.Entry::getKey, e -> Long.valueOf(e.getValue().size())));
|
||||||
|
long totalFiles = 0, totalRecords = 0;
|
||||||
|
for (String partitionPath : recordsPerPartition.keySet()) {
|
||||||
|
long numRecords = recordsPerPartition.get(partitionPath);
|
||||||
|
long numFiles =
|
||||||
|
filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath)
|
||||||
|
: 1L;
|
||||||
|
|
||||||
|
totalComparisons += numFiles * numRecords;
|
||||||
|
totalFiles +=
|
||||||
|
filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath)
|
||||||
|
: 0L;
|
||||||
|
totalRecords += numRecords;
|
||||||
|
}
|
||||||
|
logger.info("TotalRecords: " + totalRecords + ", TotalFiles: " + totalFiles
|
||||||
|
+ ", TotalAffectedPartitions:" + recordsPerPartition.size());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
// each partition will have an item per comparison.
|
||||||
* The index lookup can be skewed in three dimensions : #files, #partitions, #records
|
int parallelism = (int) (totalComparisons / MAX_ITEMS_PER_SHUFFLE_PARTITION + 1);
|
||||||
*
|
logger.info(
|
||||||
* To be able to smoothly handle skews, we need to compute how to split each partitions into
|
"Auto computed parallelism :" + parallelism + ", totalComparisons: " + totalComparisons);
|
||||||
* subpartitions. We do it here, in a way that keeps the amount of each Spark join partition to
|
return parallelism;
|
||||||
* < 2GB.
|
}
|
||||||
*
|
|
||||||
* If {@link com.uber.hoodie.config.HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is specified as a NON-zero number,
|
|
||||||
* then that is used explicitly.
|
|
||||||
*
|
|
||||||
*/
|
|
||||||
private int autoComputeParallelism(final Map<String, Long> recordsPerPartition,
|
|
||||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
|
||||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
|
||||||
|
|
||||||
long totalComparisons = 0;
|
/**
|
||||||
if (config.getBloomIndexPruneByRanges()) {
|
* Its crucial to pick the right parallelism.
|
||||||
// we will just try exploding the input and then count to determine comparisons
|
*
|
||||||
totalComparisons = explodeRecordRDDWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairRDD).count();
|
* totalSubPartitions : this is deemed safe limit, to be nice with Spark. inputParallelism :
|
||||||
} else {
|
* typically number of input file splits
|
||||||
// if not pruning by ranges, then each file in a partition needs to compared against all
|
*
|
||||||
// records for a partition.
|
* We pick the max such that, we are always safe, but go higher if say a there are a lot of input
|
||||||
Map<String, Long> filesPerPartition = partitionToFileInfo.entrySet().stream()
|
* files. (otherwise, we will fallback to number of partitions in input and end up with slow
|
||||||
.collect(Collectors.toMap(Map.Entry::getKey, e -> Long.valueOf(e.getValue().size())));
|
* performance)
|
||||||
long totalFiles = 0, totalRecords = 0;
|
*/
|
||||||
for (String partitionPath : recordsPerPartition.keySet()) {
|
private int determineParallelism(int inputParallelism, int totalSubPartitions) {
|
||||||
long numRecords = recordsPerPartition.get(partitionPath);
|
// If bloom index parallelism is set, use it to to check against the input parallelism and take the max
|
||||||
long numFiles = filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath) : 1L;
|
int indexParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism());
|
||||||
|
int joinParallelism = Math.max(totalSubPartitions, indexParallelism);
|
||||||
|
logger.info("InputParallelism: ${" + inputParallelism + "}, " +
|
||||||
|
"IndexParallelism: ${" + config.getBloomIndexParallelism() + "}, " +
|
||||||
|
"TotalSubParts: ${" + totalSubPartitions + "}, " +
|
||||||
|
"Join Parallelism set to : " + joinParallelism);
|
||||||
|
return joinParallelism;
|
||||||
|
}
|
||||||
|
|
||||||
totalComparisons += numFiles * numRecords;
|
/**
|
||||||
totalFiles += filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath) : 0L;
|
* Load all involved files as <Partition, filename> pair RDD.
|
||||||
totalRecords += numRecords;
|
*/
|
||||||
}
|
@VisibleForTesting
|
||||||
logger.info("TotalRecords: " + totalRecords + ", TotalFiles: " + totalFiles + ", TotalAffectedPartitions:" + recordsPerPartition.size());
|
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions,
|
||||||
}
|
final HoodieTable<T> hoodieTable) {
|
||||||
|
// Obtain the latest data files from all the partitions.
|
||||||
// each partition will have an item per comparison.
|
List<Tuple2<String, HoodieDataFile>> dataFilesList = jsc
|
||||||
int parallelism = (int) (totalComparisons/ MAX_ITEMS_PER_SHUFFLE_PARTITION + 1);
|
.parallelize(partitions, Math.max(partitions.size(), 1))
|
||||||
logger.info("Auto computed parallelism :" + parallelism + ", totalComparisons: " + totalComparisons);
|
.flatMapToPair(partitionPath -> {
|
||||||
return parallelism;
|
java.util.Optional<HoodieInstant> latestCommitTime =
|
||||||
}
|
hoodieTable.getCommitTimeline().filterCompletedInstants().lastInstant();
|
||||||
|
List<Tuple2<String, HoodieDataFile>> filteredFiles = new ArrayList<>();
|
||||||
/**
|
if (latestCommitTime.isPresent()) {
|
||||||
* Its crucial to pick the right parallelism.
|
filteredFiles =
|
||||||
*
|
hoodieTable.getROFileSystemView().getLatestDataFilesBeforeOrOn(partitionPath,
|
||||||
* totalSubPartitions : this is deemed safe limit, to be nice with Spark.
|
latestCommitTime.get().getTimestamp())
|
||||||
* inputParallelism : typically number of input file splits
|
.map(f -> new Tuple2<>(partitionPath, f))
|
||||||
*
|
|
||||||
* We pick the max such that, we are always safe, but go higher if say a there are a lot of
|
|
||||||
* input files. (otherwise, we will fallback to number of partitions in input and end up with
|
|
||||||
* slow performance)
|
|
||||||
*/
|
|
||||||
private int determineParallelism(int inputParallelism, int totalSubPartitions) {
|
|
||||||
// If bloom index parallelism is set, use it to to check against the input parallelism and take the max
|
|
||||||
int indexParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism());
|
|
||||||
int joinParallelism = Math.max(totalSubPartitions, indexParallelism);
|
|
||||||
logger.info("InputParallelism: ${" + inputParallelism + "}, " +
|
|
||||||
"IndexParallelism: ${" + config.getBloomIndexParallelism() + "}, " +
|
|
||||||
"TotalSubParts: ${" + totalSubPartitions + "}, " +
|
|
||||||
"Join Parallelism set to : " + joinParallelism);
|
|
||||||
return joinParallelism;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Load all involved files as <Partition, filename> pair RDD.
|
|
||||||
*/
|
|
||||||
@VisibleForTesting
|
|
||||||
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final HoodieTable<T> hoodieTable) {
|
|
||||||
// Obtain the latest data files from all the partitions.
|
|
||||||
List<Tuple2<String, HoodieDataFile>> dataFilesList = jsc.parallelize(partitions, Math.max(partitions.size(), 1))
|
|
||||||
.flatMapToPair(partitionPath -> {
|
|
||||||
java.util.Optional<HoodieInstant> latestCommitTime =
|
|
||||||
hoodieTable.getCommitTimeline().filterCompletedInstants().lastInstant();
|
|
||||||
List<Tuple2<String, HoodieDataFile>> filteredFiles = new ArrayList<>();
|
|
||||||
if (latestCommitTime.isPresent()) {
|
|
||||||
filteredFiles =
|
|
||||||
hoodieTable.getROFileSystemView().getLatestDataFilesBeforeOrOn(partitionPath,
|
|
||||||
latestCommitTime.get().getTimestamp())
|
|
||||||
.map(f -> new Tuple2<>(partitionPath, f))
|
|
||||||
.collect(toList());
|
|
||||||
}
|
|
||||||
return filteredFiles.iterator();
|
|
||||||
}).collect();
|
|
||||||
|
|
||||||
if (config.getBloomIndexPruneByRanges()) {
|
|
||||||
// also obtain file ranges, if range pruning is enabled
|
|
||||||
return jsc.parallelize(dataFilesList, Math.max(dataFilesList.size(), 1))
|
|
||||||
.mapToPair(ft -> {
|
|
||||||
try {
|
|
||||||
String[] minMaxKeys = ParquetUtils.readMinMaxRecordKeys(ft._2().getFileStatus().getPath());
|
|
||||||
return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1]));
|
|
||||||
} catch (MetadataNotFoundException me) {
|
|
||||||
logger.warn("Unable to find range metadata in file :" + ft._2());
|
|
||||||
return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName()));
|
|
||||||
}
|
|
||||||
}).collect();
|
|
||||||
} else {
|
|
||||||
return dataFilesList.stream()
|
|
||||||
.map(ft -> new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName())))
|
|
||||||
.collect(toList());
|
.collect(toList());
|
||||||
|
}
|
||||||
|
return filteredFiles.iterator();
|
||||||
|
}).collect();
|
||||||
|
|
||||||
|
if (config.getBloomIndexPruneByRanges()) {
|
||||||
|
// also obtain file ranges, if range pruning is enabled
|
||||||
|
return jsc.parallelize(dataFilesList, Math.max(dataFilesList.size(), 1))
|
||||||
|
.mapToPair(ft -> {
|
||||||
|
try {
|
||||||
|
String[] minMaxKeys = ParquetUtils
|
||||||
|
.readMinMaxRecordKeys(ft._2().getFileStatus().getPath());
|
||||||
|
return new Tuple2<>(ft._1(),
|
||||||
|
new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1]));
|
||||||
|
} catch (MetadataNotFoundException me) {
|
||||||
|
logger.warn("Unable to find range metadata in file :" + ft._2());
|
||||||
|
return new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName()));
|
||||||
|
}
|
||||||
|
}).collect();
|
||||||
|
} else {
|
||||||
|
return dataFilesList.stream()
|
||||||
|
.map(ft -> new Tuple2<>(ft._1(), new BloomIndexFileInfo(ft._2().getFileName())))
|
||||||
|
.collect(toList());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean rollbackCommit(String commitTime) {
|
||||||
|
// Nope, don't need to do anything.
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This is not global, since we depend on the partitionPath to do the lookup
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public boolean isGlobal() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* No indexes into log files yet.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public boolean canIndexLogFiles() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Bloom filters are stored, into the same data files.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public boolean isImplicitWithStorage() {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* if we dont have key ranges, then also we need to compare against the file. no other choice if
|
||||||
|
* we do, then only compare the file if the record key falls in range.
|
||||||
|
*/
|
||||||
|
private boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recordKey) {
|
||||||
|
return !indexInfo.hasKeyRanges() || indexInfo.isKeyInRange(recordKey);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* For each incoming record, produce N output records, 1 each for each file against which the
|
||||||
|
* record's key needs to be checked. For datasets, where the keys have a definite insert order
|
||||||
|
* (e.g: timestamp as prefix), the number of files to be compared gets cut down a lot from range
|
||||||
|
* pruning.
|
||||||
|
*/
|
||||||
|
// sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on recordKey
|
||||||
|
// ranges in the index info.
|
||||||
|
@VisibleForTesting
|
||||||
|
JavaPairRDD<String, Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
|
||||||
|
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||||
|
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||||
|
return partitionRecordKeyPairRDD
|
||||||
|
.map(partitionRecordKeyPair -> {
|
||||||
|
String recordKey = partitionRecordKeyPair._2();
|
||||||
|
String partitionPath = partitionRecordKeyPair._1();
|
||||||
|
|
||||||
|
List<BloomIndexFileInfo> indexInfos = partitionToFileIndexInfo.get(partitionPath);
|
||||||
|
List<Tuple2<String, Tuple2<String, HoodieKey>>> recordComparisons = new ArrayList<>();
|
||||||
|
if (indexInfos
|
||||||
|
!= null) { // could be null, if there are no files in a given partition yet.
|
||||||
|
// for each candidate file in partition, that needs to be compared.
|
||||||
|
for (BloomIndexFileInfo indexInfo : indexInfos) {
|
||||||
|
if (shouldCompareWithFile(indexInfo, recordKey)) {
|
||||||
|
recordComparisons.add(
|
||||||
|
new Tuple2<>(String.format("%s#%s", indexInfo.getFileName(), recordKey),
|
||||||
|
new Tuple2<>(indexInfo.getFileName(),
|
||||||
|
new HoodieKey(recordKey, partitionPath))));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return recordComparisons;
|
||||||
|
})
|
||||||
|
.flatMapToPair(t -> t.iterator());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Find out <RowKey, filename> pair. All workload grouped by file-level.
|
||||||
|
*
|
||||||
|
* 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
|
||||||
|
*/
|
||||||
|
@VisibleForTesting
|
||||||
|
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(
|
||||||
|
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||||
|
JavaPairRDD<String, String> partitionRecordKeyPairRDD,
|
||||||
|
int totalSubpartitions) {
|
||||||
|
|
||||||
|
int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(),
|
||||||
|
totalSubpartitions);
|
||||||
|
|
||||||
|
JavaPairRDD<String, Tuple2<String, HoodieKey>> fileSortedTripletRDD = explodeRecordRDDWithFileComparisons(
|
||||||
|
partitionToFileIndexInfo, partitionRecordKeyPairRDD)
|
||||||
|
// sort further based on filename, such that all checking for the file can happen within a single partition, on-the-fly
|
||||||
|
.sortByKey(true, joinParallelism);
|
||||||
|
|
||||||
|
return fileSortedTripletRDD
|
||||||
|
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(config.getBasePath()), true)
|
||||||
|
.flatMap(indexLookupResults -> indexLookupResults.iterator())
|
||||||
|
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
|
||||||
|
.flatMapToPair(lookupResult -> {
|
||||||
|
List<Tuple2<String, String>> vals = new ArrayList<>();
|
||||||
|
for (String recordKey : lookupResult.getMatchingRecordKeys()) {
|
||||||
|
vals.add(new Tuple2<>(recordKey, lookupResult.getFileName()));
|
||||||
|
}
|
||||||
|
return vals.iterator();
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tag the <rowKey, filename> back to the original HoodieRecord RDD.
|
||||||
|
*/
|
||||||
|
private JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||||
|
JavaPairRDD<String, String> rowKeyFilenamePairRDD,
|
||||||
|
JavaRDD<HoodieRecord<T>> recordRDD) {
|
||||||
|
JavaPairRDD<String, HoodieRecord<T>> rowKeyRecordPairRDD = recordRDD
|
||||||
|
.mapToPair(record -> new Tuple2<>(record.getRecordKey(), record));
|
||||||
|
|
||||||
|
// 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(
|
||||||
|
v1 -> {
|
||||||
|
HoodieRecord<T> record = v1._1();
|
||||||
|
if (v1._2().isPresent()) {
|
||||||
|
String filename = v1._2().get();
|
||||||
|
if (filename != null && !filename.isEmpty()) {
|
||||||
|
record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename),
|
||||||
|
FSUtils.getFileId(filename)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return record;
|
||||||
}
|
}
|
||||||
}
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
@Override
|
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||||
public boolean rollbackCommit(String commitTime) {
|
HoodieTable<T> hoodieTable) {
|
||||||
// Nope, don't need to do anything.
|
return writeStatusRDD;
|
||||||
return true;
|
}
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* This is not global, since we depend on the partitionPath to do the lookup
|
|
||||||
*
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
public boolean isGlobal() {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* No indexes into log files yet.
|
|
||||||
*
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
public boolean canIndexLogFiles() {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Bloom filters are stored, into the same data files.
|
|
||||||
*
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
public boolean isImplicitWithStorage() {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* if we dont have key ranges, then also we need to compare against the file. no other choice
|
|
||||||
* if we do, then only compare the file if the record key falls in range.
|
|
||||||
|
|
||||||
* @param indexInfo
|
|
||||||
* @param recordKey
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
private boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recordKey) {
|
|
||||||
return !indexInfo.hasKeyRanges() || indexInfo.isKeyInRange(recordKey);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
|
||||||
* For each incoming record, produce N output records, 1 each for each file against which the record's key
|
|
||||||
* needs to be checked. For datasets, where the keys have a definite insert order (e.g: timestamp as prefix),
|
|
||||||
* the number of files to be compared gets cut down a lot from range pruning.
|
|
||||||
*
|
|
||||||
*
|
|
||||||
* @param partitionToFileIndexInfo
|
|
||||||
* @param partitionRecordKeyPairRDD
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
// sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on recordKey
|
|
||||||
// ranges in the index info.
|
|
||||||
@VisibleForTesting
|
|
||||||
JavaPairRDD<String, Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
|
||||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
|
||||||
return partitionRecordKeyPairRDD
|
|
||||||
.map(partitionRecordKeyPair -> {
|
|
||||||
String recordKey = partitionRecordKeyPair._2();
|
|
||||||
String partitionPath = partitionRecordKeyPair._1();
|
|
||||||
|
|
||||||
List<BloomIndexFileInfo> indexInfos = partitionToFileIndexInfo.get(partitionPath);
|
|
||||||
List<Tuple2<String, Tuple2<String, HoodieKey>>> recordComparisons = new ArrayList<>();
|
|
||||||
if (indexInfos != null) { // could be null, if there are no files in a given partition yet.
|
|
||||||
// for each candidate file in partition, that needs to be compared.
|
|
||||||
for (BloomIndexFileInfo indexInfo : indexInfos) {
|
|
||||||
if (shouldCompareWithFile(indexInfo, recordKey)) {
|
|
||||||
recordComparisons.add(
|
|
||||||
new Tuple2<>(String.format("%s#%s", indexInfo.getFileName(), recordKey),
|
|
||||||
new Tuple2<>(indexInfo.getFileName(), new HoodieKey(recordKey, partitionPath))));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return recordComparisons;
|
|
||||||
})
|
|
||||||
.flatMapToPair(t -> t.iterator());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Find out <RowKey, filename> pair. All workload grouped by file-level.
|
|
||||||
*
|
|
||||||
* 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
|
|
||||||
*/
|
|
||||||
@VisibleForTesting
|
|
||||||
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
|
||||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD,
|
|
||||||
int totalSubpartitions) {
|
|
||||||
|
|
||||||
int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(), totalSubpartitions);
|
|
||||||
|
|
||||||
JavaPairRDD<String, Tuple2<String, HoodieKey>> fileSortedTripletRDD = explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD)
|
|
||||||
// sort further based on filename, such that all checking for the file can happen within a single partition, on-the-fly
|
|
||||||
.sortByKey(true, joinParallelism);
|
|
||||||
|
|
||||||
return fileSortedTripletRDD
|
|
||||||
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(config.getBasePath()), true)
|
|
||||||
.flatMap(indexLookupResults -> indexLookupResults.iterator())
|
|
||||||
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
|
|
||||||
.flatMapToPair(lookupResult -> {
|
|
||||||
List<Tuple2<String, String>> vals = new ArrayList<>();
|
|
||||||
for (String recordKey : lookupResult.getMatchingRecordKeys()) {
|
|
||||||
vals.add(new Tuple2<>(recordKey, lookupResult.getFileName()));
|
|
||||||
}
|
|
||||||
return vals.iterator();
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Tag the <rowKey, filename> back to the original HoodieRecord RDD.
|
|
||||||
*/
|
|
||||||
private JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(JavaPairRDD<String, String> rowKeyFilenamePairRDD,
|
|
||||||
JavaRDD<HoodieRecord<T>> recordRDD) {
|
|
||||||
JavaPairRDD<String, HoodieRecord<T>> rowKeyRecordPairRDD = recordRDD
|
|
||||||
.mapToPair(record -> new Tuple2<>(record.getRecordKey(), record));
|
|
||||||
|
|
||||||
// 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(
|
|
||||||
v1 -> {
|
|
||||||
HoodieRecord<T> record = v1._1();
|
|
||||||
if (v1._2().isPresent()) {
|
|
||||||
String filename = v1._2().get();
|
|
||||||
if (filename != null && !filename.isEmpty()) {
|
|
||||||
record.setCurrentLocation(new HoodieRecordLocation(FSUtils.getCommitTime(filename),
|
|
||||||
FSUtils.getFileId(filename)));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return record;
|
|
||||||
}
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTable<T> hoodieTable) {
|
|
||||||
return writeStatusRDD;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -24,172 +24,182 @@ import com.uber.hoodie.common.util.ParquetUtils;
|
|||||||
import com.uber.hoodie.exception.HoodieException;
|
import com.uber.hoodie.exception.HoodieException;
|
||||||
import com.uber.hoodie.exception.HoodieIndexException;
|
import com.uber.hoodie.exception.HoodieIndexException;
|
||||||
import com.uber.hoodie.func.LazyIterableIterator;
|
import com.uber.hoodie.func.LazyIterableIterator;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.log4j.LogManager;
|
|
||||||
import org.apache.log4j.Logger;
|
|
||||||
import org.apache.spark.api.java.function.Function2;
|
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
import org.apache.spark.api.java.function.Function2;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Function performing actual checking of RDD parition containing (fileId, hoodieKeys) against the
|
* Function performing actual checking of RDD parition containing (fileId, hoodieKeys) against the
|
||||||
* actual files
|
* actual files
|
||||||
*/
|
*/
|
||||||
public class HoodieBloomIndexCheckFunction implements Function2<Integer, Iterator<Tuple2<String, Tuple2<String, HoodieKey>>>, Iterator<List<IndexLookupResult>>> {
|
public class HoodieBloomIndexCheckFunction implements
|
||||||
|
Function2<Integer, Iterator<Tuple2<String, Tuple2<String, HoodieKey>>>, Iterator<List<IndexLookupResult>>> {
|
||||||
|
|
||||||
private static Logger logger = LogManager.getLogger(HoodieBloomIndexCheckFunction.class);
|
private static Logger logger = LogManager.getLogger(HoodieBloomIndexCheckFunction.class);
|
||||||
|
|
||||||
private final String basePath;
|
private final String basePath;
|
||||||
|
|
||||||
public HoodieBloomIndexCheckFunction(String basePath) {
|
public HoodieBloomIndexCheckFunction(String basePath) {
|
||||||
this.basePath = basePath;
|
this.basePath = basePath;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Given a list of row keys and one file, return only row keys existing in that file.
|
||||||
|
*/
|
||||||
|
public static List<String> checkCandidatesAgainstFile(List<String> candidateRecordKeys,
|
||||||
|
Path filePath) throws HoodieIndexException {
|
||||||
|
List<String> foundRecordKeys = new ArrayList<>();
|
||||||
|
try {
|
||||||
|
// Load all rowKeys from the file, to double-confirm
|
||||||
|
if (!candidateRecordKeys.isEmpty()) {
|
||||||
|
Set<String> fileRowKeys = ParquetUtils.readRowKeysFromParquet(filePath);
|
||||||
|
logger.info("Loading " + fileRowKeys.size() + " row keys from " + filePath);
|
||||||
|
if (logger.isDebugEnabled()) {
|
||||||
|
logger.debug("Keys from " + filePath + " => " + fileRowKeys);
|
||||||
|
}
|
||||||
|
for (String rowKey : candidateRecordKeys) {
|
||||||
|
if (fileRowKeys.contains(rowKey)) {
|
||||||
|
foundRecordKeys.add(rowKey);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
logger.info("After checking with row keys, we have " + foundRecordKeys.size()
|
||||||
|
+ " results, for file " + filePath + " => " + foundRecordKeys);
|
||||||
|
if (logger.isDebugEnabled()) {
|
||||||
|
logger.debug("Keys matching for file " + filePath + " => " + foundRecordKeys);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new HoodieIndexException("Error checking candidate keys against file.", e);
|
||||||
}
|
}
|
||||||
|
return foundRecordKeys;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
class LazyKeyCheckIterator extends
|
||||||
* Given a list of row keys and one file, return only row keys existing in that file.
|
LazyIterableIterator<Tuple2<String, Tuple2<String, HoodieKey>>, List<IndexLookupResult>> {
|
||||||
*/
|
|
||||||
public static List<String> checkCandidatesAgainstFile(List<String> candidateRecordKeys, Path filePath) throws HoodieIndexException {
|
private List<String> candidateRecordKeys;
|
||||||
List<String> foundRecordKeys = new ArrayList<>();
|
|
||||||
try {
|
private BloomFilter bloomFilter;
|
||||||
// Load all rowKeys from the file, to double-confirm
|
|
||||||
if (!candidateRecordKeys.isEmpty()) {
|
private String currentFile;
|
||||||
Set<String> fileRowKeys = ParquetUtils.readRowKeysFromParquet(filePath);
|
|
||||||
logger.info("Loading " + fileRowKeys.size() + " row keys from " + filePath);
|
private String currentParitionPath;
|
||||||
if (logger.isDebugEnabled()) {
|
|
||||||
logger.debug("Keys from " + filePath + " => " + fileRowKeys);
|
LazyKeyCheckIterator(
|
||||||
}
|
Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr) {
|
||||||
for (String rowKey : candidateRecordKeys) {
|
super(fileParitionRecordKeyTripletItr);
|
||||||
if (fileRowKeys.contains(rowKey)) {
|
currentFile = null;
|
||||||
foundRecordKeys.add(rowKey);
|
candidateRecordKeys = new ArrayList<>();
|
||||||
}
|
bloomFilter = null;
|
||||||
}
|
currentParitionPath = null;
|
||||||
logger.info("After checking with row keys, we have " + foundRecordKeys.size() + " results, for file " + filePath + " => " + foundRecordKeys);
|
|
||||||
if (logger.isDebugEnabled()) {
|
|
||||||
logger.debug("Keys matching for file " + filePath + " => " + foundRecordKeys);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} catch (Exception e){
|
|
||||||
throw new HoodieIndexException("Error checking candidate keys against file.", e);
|
|
||||||
}
|
|
||||||
return foundRecordKeys;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class LazyKeyCheckIterator extends LazyIterableIterator<Tuple2<String, Tuple2<String, HoodieKey>>, List<IndexLookupResult>> {
|
|
||||||
|
|
||||||
private List<String> candidateRecordKeys;
|
|
||||||
|
|
||||||
private BloomFilter bloomFilter;
|
|
||||||
|
|
||||||
private String currentFile;
|
|
||||||
|
|
||||||
private String currentParitionPath;
|
|
||||||
|
|
||||||
LazyKeyCheckIterator(Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr) {
|
|
||||||
super(fileParitionRecordKeyTripletItr);
|
|
||||||
currentFile = null;
|
|
||||||
candidateRecordKeys = new ArrayList<>();
|
|
||||||
bloomFilter = null;
|
|
||||||
currentParitionPath = null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void start() {
|
|
||||||
}
|
|
||||||
|
|
||||||
private void initState(String fileName, String partitionPath) throws HoodieIndexException {
|
|
||||||
try {
|
|
||||||
Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName);
|
|
||||||
bloomFilter = ParquetUtils.readBloomFilterFromParquetMetadata(filePath);
|
|
||||||
candidateRecordKeys = new ArrayList<>();
|
|
||||||
currentFile = fileName;
|
|
||||||
currentParitionPath = partitionPath;
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new HoodieIndexException("Error checking candidate keys against file.", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected List<IndexLookupResult> computeNext() {
|
|
||||||
|
|
||||||
List<IndexLookupResult> ret = new ArrayList<>();
|
|
||||||
try {
|
|
||||||
// process one file in each go.
|
|
||||||
while (inputItr.hasNext()) {
|
|
||||||
|
|
||||||
Tuple2<String, Tuple2<String, HoodieKey>> currentTuple = inputItr.next();
|
|
||||||
String fileName = currentTuple._2._1;
|
|
||||||
String partitionPath = currentTuple._2._2.getPartitionPath();
|
|
||||||
String recordKey = currentTuple._2._2.getRecordKey();
|
|
||||||
|
|
||||||
// lazily init state
|
|
||||||
if (currentFile == null) {
|
|
||||||
initState(fileName, partitionPath);
|
|
||||||
}
|
|
||||||
|
|
||||||
// if continue on current file)
|
|
||||||
if (fileName.equals(currentFile)) {
|
|
||||||
// check record key against bloom filter of current file & add to possible keys if needed
|
|
||||||
if (bloomFilter.mightContain(recordKey)) {
|
|
||||||
if (logger.isDebugEnabled()) {
|
|
||||||
logger.debug("#1 Adding " + recordKey + " as candidate for file " + fileName);
|
|
||||||
}
|
|
||||||
candidateRecordKeys.add(recordKey);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
// do the actual checking of file & break out
|
|
||||||
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
|
|
||||||
logger.info("#1 After bloom filter, the candidate row keys is reduced to " + candidateRecordKeys.size() + " for " + filePath);
|
|
||||||
if (logger.isDebugEnabled()) {
|
|
||||||
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
|
|
||||||
}
|
|
||||||
ret.add(new IndexLookupResult(currentFile, checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
|
|
||||||
|
|
||||||
initState(fileName, partitionPath);
|
|
||||||
if (bloomFilter.mightContain(recordKey)) {
|
|
||||||
if (logger.isDebugEnabled()) {
|
|
||||||
logger.debug("#2 Adding " + recordKey + " as candidate for file " + fileName);
|
|
||||||
}
|
|
||||||
candidateRecordKeys.add(recordKey);
|
|
||||||
}
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// handle case, where we ran out of input, finish pending work, update return val
|
|
||||||
if (!inputItr.hasNext()) {
|
|
||||||
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
|
|
||||||
logger.info("#2 After bloom filter, the candidate row keys is reduced to " + candidateRecordKeys.size() + " for " + filePath);
|
|
||||||
if (logger.isDebugEnabled()) {
|
|
||||||
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
|
|
||||||
}
|
|
||||||
ret.add(new IndexLookupResult(currentFile, checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
|
|
||||||
}
|
|
||||||
|
|
||||||
} catch (Throwable e) {
|
|
||||||
if (e instanceof HoodieException) {
|
|
||||||
throw e;
|
|
||||||
}
|
|
||||||
throw new HoodieIndexException("Error checking bloom filter index. ", e);
|
|
||||||
}
|
|
||||||
|
|
||||||
return ret;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void end() {
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<List<IndexLookupResult>> call(Integer partition,
|
protected void start() {
|
||||||
Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr) throws Exception {
|
|
||||||
return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void initState(String fileName, String partitionPath) throws HoodieIndexException {
|
||||||
|
try {
|
||||||
|
Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName);
|
||||||
|
bloomFilter = ParquetUtils.readBloomFilterFromParquetMetadata(filePath);
|
||||||
|
candidateRecordKeys = new ArrayList<>();
|
||||||
|
currentFile = fileName;
|
||||||
|
currentParitionPath = partitionPath;
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new HoodieIndexException("Error checking candidate keys against file.", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected List<IndexLookupResult> computeNext() {
|
||||||
|
|
||||||
|
List<IndexLookupResult> ret = new ArrayList<>();
|
||||||
|
try {
|
||||||
|
// process one file in each go.
|
||||||
|
while (inputItr.hasNext()) {
|
||||||
|
|
||||||
|
Tuple2<String, Tuple2<String, HoodieKey>> currentTuple = inputItr.next();
|
||||||
|
String fileName = currentTuple._2._1;
|
||||||
|
String partitionPath = currentTuple._2._2.getPartitionPath();
|
||||||
|
String recordKey = currentTuple._2._2.getRecordKey();
|
||||||
|
|
||||||
|
// lazily init state
|
||||||
|
if (currentFile == null) {
|
||||||
|
initState(fileName, partitionPath);
|
||||||
|
}
|
||||||
|
|
||||||
|
// if continue on current file)
|
||||||
|
if (fileName.equals(currentFile)) {
|
||||||
|
// check record key against bloom filter of current file & add to possible keys if needed
|
||||||
|
if (bloomFilter.mightContain(recordKey)) {
|
||||||
|
if (logger.isDebugEnabled()) {
|
||||||
|
logger.debug("#1 Adding " + recordKey + " as candidate for file " + fileName);
|
||||||
|
}
|
||||||
|
candidateRecordKeys.add(recordKey);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// do the actual checking of file & break out
|
||||||
|
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
|
||||||
|
logger.info(
|
||||||
|
"#1 After bloom filter, the candidate row keys is reduced to " + candidateRecordKeys
|
||||||
|
.size() + " for " + filePath);
|
||||||
|
if (logger.isDebugEnabled()) {
|
||||||
|
logger
|
||||||
|
.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
|
||||||
|
}
|
||||||
|
ret.add(new IndexLookupResult(currentFile,
|
||||||
|
checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
|
||||||
|
|
||||||
|
initState(fileName, partitionPath);
|
||||||
|
if (bloomFilter.mightContain(recordKey)) {
|
||||||
|
if (logger.isDebugEnabled()) {
|
||||||
|
logger.debug("#2 Adding " + recordKey + " as candidate for file " + fileName);
|
||||||
|
}
|
||||||
|
candidateRecordKeys.add(recordKey);
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// handle case, where we ran out of input, finish pending work, update return val
|
||||||
|
if (!inputItr.hasNext()) {
|
||||||
|
Path filePath = new Path(basePath + "/" + currentParitionPath + "/" + currentFile);
|
||||||
|
logger.info(
|
||||||
|
"#2 After bloom filter, the candidate row keys is reduced to " + candidateRecordKeys
|
||||||
|
.size() + " for " + filePath);
|
||||||
|
if (logger.isDebugEnabled()) {
|
||||||
|
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
|
||||||
|
}
|
||||||
|
ret.add(new IndexLookupResult(currentFile,
|
||||||
|
checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
|
||||||
|
}
|
||||||
|
|
||||||
|
} catch (Throwable e) {
|
||||||
|
if (e instanceof HoodieException) {
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
throw new HoodieIndexException("Error checking bloom filter index. ", e);
|
||||||
|
}
|
||||||
|
|
||||||
|
return ret;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void end() {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Iterator<List<IndexLookupResult>> call(Integer partition,
|
||||||
|
Iterator<Tuple2<String, Tuple2<String, HoodieKey>>> fileParitionRecordKeyTripletItr)
|
||||||
|
throws Exception {
|
||||||
|
return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -25,21 +25,21 @@ import java.util.List;
|
|||||||
*/
|
*/
|
||||||
public class IndexLookupResult {
|
public class IndexLookupResult {
|
||||||
|
|
||||||
private String fileName;
|
private String fileName;
|
||||||
|
|
||||||
|
|
||||||
private List<String> matchingRecordKeys;
|
private List<String> matchingRecordKeys;
|
||||||
|
|
||||||
public IndexLookupResult(String fileName, List<String> matchingRecordKeys) {
|
public IndexLookupResult(String fileName, List<String> matchingRecordKeys) {
|
||||||
this.fileName = fileName;
|
this.fileName = fileName;
|
||||||
this.matchingRecordKeys = matchingRecordKeys;
|
this.matchingRecordKeys = matchingRecordKeys;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getFileName() {
|
public String getFileName() {
|
||||||
return fileName;
|
return fileName;
|
||||||
}
|
}
|
||||||
|
|
||||||
public List<String> getMatchingRecordKeys() {
|
public List<String> getMatchingRecordKeys() {
|
||||||
return matchingRecordKeys;
|
return matchingRecordKeys;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -19,7 +19,6 @@
|
|||||||
package com.uber.hoodie.index.bucketed;
|
package com.uber.hoodie.index.bucketed;
|
||||||
|
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
|
|
||||||
import com.uber.hoodie.WriteStatus;
|
import com.uber.hoodie.WriteStatus;
|
||||||
import com.uber.hoodie.common.model.HoodieKey;
|
import com.uber.hoodie.common.model.HoodieKey;
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
@@ -29,96 +28,86 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
|||||||
import com.uber.hoodie.exception.HoodieIndexException;
|
import com.uber.hoodie.exception.HoodieIndexException;
|
||||||
import com.uber.hoodie.index.HoodieIndex;
|
import com.uber.hoodie.index.HoodieIndex;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.spark.api.java.JavaPairRDD;
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An `stateless` index implementation that will using a deterministic mapping function to
|
* An `stateless` index implementation that will using a deterministic mapping function to determine
|
||||||
* determine the fileID for a given record.
|
* the fileID for a given record.
|
||||||
*
|
|
||||||
* Pros:
|
|
||||||
* - Fast
|
|
||||||
*
|
|
||||||
* Cons :
|
|
||||||
* - Need to tune the number of buckets per partition path manually (FIXME: Need to autotune this)
|
|
||||||
* - Could increase write amplification on copy-on-write storage since inserts always rewrite files
|
|
||||||
* - Not global.
|
|
||||||
*
|
|
||||||
*
|
*
|
||||||
|
* Pros: - Fast
|
||||||
*
|
*
|
||||||
|
* Cons : - Need to tune the number of buckets per partition path manually (FIXME: Need to autotune
|
||||||
|
* this) - Could increase write amplification on copy-on-write storage since inserts always rewrite
|
||||||
|
* files - Not global.
|
||||||
*/
|
*/
|
||||||
public class BucketedIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
public class BucketedIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||||
|
|
||||||
private static Logger logger = LogManager.getLogger(BucketedIndex.class);
|
private static Logger logger = LogManager.getLogger(BucketedIndex.class);
|
||||||
|
|
||||||
public BucketedIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
public BucketedIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||||
super(config, jsc);
|
super(config, jsc);
|
||||||
}
|
}
|
||||||
|
|
||||||
private String getBucket(String recordKey) {
|
private String getBucket(String recordKey) {
|
||||||
return String.valueOf(recordKey.hashCode() % config.getNumBucketsPerPartition());
|
return String.valueOf(recordKey.hashCode() % config.getNumBucketsPerPartition());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys, HoodieTable<T> table) {
|
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||||
return hoodieKeys.mapToPair(hk -> new Tuple2<>(hk, Optional.of(getBucket(hk.getRecordKey()))));
|
HoodieTable<T> table) {
|
||||||
}
|
return hoodieKeys.mapToPair(hk -> new Tuple2<>(hk, Optional.of(getBucket(hk.getRecordKey()))));
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieTable<T> hoodieTable) throws HoodieIndexException {
|
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||||
return recordRDD.map(record -> {
|
HoodieTable<T> hoodieTable) throws HoodieIndexException {
|
||||||
String bucket = getBucket(record.getRecordKey());
|
return recordRDD.map(record -> {
|
||||||
//HACK(vc) a non-existent commit is provided here.
|
String bucket = getBucket(record.getRecordKey());
|
||||||
record.setCurrentLocation(new HoodieRecordLocation("000", bucket));
|
//HACK(vc) a non-existent commit is provided here.
|
||||||
return record;
|
record.setCurrentLocation(new HoodieRecordLocation("000", bucket));
|
||||||
});
|
return record;
|
||||||
}
|
});
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTable<T> hoodieTable) throws HoodieIndexException {
|
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||||
return writeStatusRDD;
|
HoodieTable<T> hoodieTable) throws HoodieIndexException {
|
||||||
}
|
return writeStatusRDD;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean rollbackCommit(String commitTime) {
|
public boolean rollbackCommit(String commitTime) {
|
||||||
// nothing to rollback in the index.
|
// nothing to rollback in the index.
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Bucketing is still done within each partition.
|
* Bucketing is still done within each partition.
|
||||||
*
|
*/
|
||||||
* @return
|
@Override
|
||||||
*/
|
public boolean isGlobal() {
|
||||||
@Override
|
return false;
|
||||||
public boolean isGlobal() {
|
}
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Since indexing is just a deterministic hash, we can identify file group correctly even without an index
|
* Since indexing is just a deterministic hash, we can identify file group correctly even without
|
||||||
* on the actual log file.
|
* an index on the actual log file.
|
||||||
*
|
*/
|
||||||
* @return
|
@Override
|
||||||
*/
|
public boolean canIndexLogFiles() {
|
||||||
@Override
|
return true;
|
||||||
public boolean canIndexLogFiles() {
|
}
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Indexing is just a hash function.
|
* Indexing is just a hash function.
|
||||||
*
|
*/
|
||||||
* @return
|
@Override
|
||||||
*/
|
public boolean isImplicitWithStorage() {
|
||||||
@Override
|
return true;
|
||||||
public boolean isImplicitWithStorage() {
|
}
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -19,24 +19,33 @@
|
|||||||
package com.uber.hoodie.index.hbase;
|
package com.uber.hoodie.index.hbase;
|
||||||
|
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
|
||||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
|
||||||
import com.uber.hoodie.WriteStatus;
|
import com.uber.hoodie.WriteStatus;
|
||||||
import com.uber.hoodie.common.model.HoodieKey;
|
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.HoodieRecordLocation;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||||
|
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||||
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.exception.HoodieDependentSystemUnavailableException;
|
import com.uber.hoodie.exception.HoodieDependentSystemUnavailableException;
|
||||||
import com.uber.hoodie.exception.HoodieIndexException;
|
import com.uber.hoodie.exception.HoodieIndexException;
|
||||||
import com.uber.hoodie.index.HoodieIndex;
|
import com.uber.hoodie.index.HoodieIndex;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.*;
|
import org.apache.hadoop.hbase.client.Connection;
|
||||||
|
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||||
|
import org.apache.hadoop.hbase.client.Delete;
|
||||||
|
import org.apache.hadoop.hbase.client.Get;
|
||||||
|
import org.apache.hadoop.hbase.client.HTable;
|
||||||
|
import org.apache.hadoop.hbase.client.Put;
|
||||||
|
import org.apache.hadoop.hbase.client.Result;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
@@ -45,230 +54,221 @@ import org.apache.spark.api.java.JavaRDD;
|
|||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.apache.spark.api.java.function.Function2;
|
import org.apache.spark.api.java.function.Function2;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Hoodie Index implementation backed by HBase
|
* Hoodie Index implementation backed by HBase
|
||||||
*/
|
*/
|
||||||
public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||||
private final static byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s");
|
|
||||||
private final static byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts");
|
|
||||||
private final static byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name");
|
|
||||||
private final static byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path");
|
|
||||||
|
|
||||||
private static Logger logger = LogManager.getLogger(HBaseIndex.class);
|
private final static byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s");
|
||||||
|
private final static byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts");
|
||||||
|
private final static byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name");
|
||||||
|
private final static byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path");
|
||||||
|
|
||||||
private final String tableName;
|
private static Logger logger = LogManager.getLogger(HBaseIndex.class);
|
||||||
|
|
||||||
public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
private final String tableName;
|
||||||
super(config, jsc);
|
|
||||||
this.tableName = config.getProps().getProperty(HoodieIndexConfig.HBASE_TABLENAME_PROP);
|
public HBaseIndex(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||||
|
super(config, jsc);
|
||||||
|
this.tableName = config.getProps().getProperty(HoodieIndexConfig.HBASE_TABLENAME_PROP);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
||||||
|
JavaRDD<HoodieKey> hoodieKeys, HoodieTable<T> table) {
|
||||||
|
throw new UnsupportedOperationException("HBase index does not implement check exist yet");
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Connection hbaseConnection = null;
|
||||||
|
|
||||||
|
private Connection getHBaseConnection() {
|
||||||
|
Configuration hbaseConfig = HBaseConfiguration.create();
|
||||||
|
String quorum = config.getProps().getProperty(HoodieIndexConfig.HBASE_ZKQUORUM_PROP);
|
||||||
|
hbaseConfig.set("hbase.zookeeper.quorum", quorum);
|
||||||
|
String port = config.getProps().getProperty(HoodieIndexConfig.HBASE_ZKPORT_PROP);
|
||||||
|
hbaseConfig.set("hbase.zookeeper.property.clientPort", port);
|
||||||
|
try {
|
||||||
|
return ConnectionFactory.createConnection(hbaseConfig);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieDependentSystemUnavailableException(
|
||||||
|
HoodieDependentSystemUnavailableException.HBASE, quorum + ":" + port);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Function that tags each HoodieRecord with an existing location, if known.
|
||||||
|
*/
|
||||||
|
class LocationTagFunction
|
||||||
|
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
|
||||||
|
|
||||||
|
private final HoodieTable<T> hoodieTable;
|
||||||
|
|
||||||
|
LocationTagFunction(HoodieTable<T> hoodieTable) {
|
||||||
|
this.hoodieTable = hoodieTable;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
|
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
|
||||||
JavaRDD<HoodieKey> hoodieKeys, HoodieTable<T> table) {
|
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
|
||||||
throw new UnsupportedOperationException("HBase index does not implement check exist yet");
|
// Grab the global HBase connection
|
||||||
}
|
synchronized (HBaseIndex.class) {
|
||||||
|
if (hbaseConnection == null) {
|
||||||
private static Connection hbaseConnection = null;
|
hbaseConnection = getHBaseConnection();
|
||||||
|
|
||||||
private Connection getHBaseConnection() {
|
|
||||||
Configuration hbaseConfig = HBaseConfiguration.create();
|
|
||||||
String quorum = config.getProps().getProperty(HoodieIndexConfig.HBASE_ZKQUORUM_PROP);
|
|
||||||
hbaseConfig.set("hbase.zookeeper.quorum", quorum);
|
|
||||||
String port = config.getProps().getProperty(HoodieIndexConfig.HBASE_ZKPORT_PROP);
|
|
||||||
hbaseConfig.set("hbase.zookeeper.property.clientPort", port);
|
|
||||||
try {
|
|
||||||
return ConnectionFactory.createConnection(hbaseConfig);
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieDependentSystemUnavailableException(
|
|
||||||
HoodieDependentSystemUnavailableException.HBASE, quorum + ":" + port);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||||
|
HTable hTable = null;
|
||||||
|
try {
|
||||||
|
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
|
||||||
|
// Do the tagging.
|
||||||
|
while (hoodieRecordIterator.hasNext()) {
|
||||||
|
HoodieRecord rec = hoodieRecordIterator.next();
|
||||||
|
// TODO(vc): This may need to be a multi get.
|
||||||
|
Result result = hTable.get(
|
||||||
|
new Get(Bytes.toBytes(rec.getRecordKey())).setMaxVersions(1)
|
||||||
|
.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)
|
||||||
|
.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)
|
||||||
|
.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
|
||||||
|
|
||||||
/**
|
// first, attempt to grab location from HBase
|
||||||
* Function that tags each HoodieRecord with an existing location, if known.
|
if (result.getRow() != null) {
|
||||||
*/
|
String commitTs =
|
||||||
class LocationTagFunction
|
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
|
||||||
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
|
String fileId =
|
||||||
|
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
|
||||||
|
|
||||||
private final HoodieTable<T> hoodieTable;
|
HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||||
|
// if the last commit ts for this row is less than the system commit ts
|
||||||
LocationTagFunction(HoodieTable<T> hoodieTable) {
|
if (!commitTimeline.empty() && commitTimeline.containsInstant(
|
||||||
this.hoodieTable = hoodieTable;
|
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))) {
|
||||||
|
rec.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
taggedRecords.add(rec);
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieIndexException(
|
||||||
|
"Failed to Tag indexed locations because of exception with HBase Client", e);
|
||||||
|
} finally {
|
||||||
|
if (hTable != null) {
|
||||||
|
try {
|
||||||
|
hTable.close();
|
||||||
|
} catch (IOException e) {
|
||||||
|
// Ignore
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
}
|
||||||
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
|
return taggedRecords.iterator();
|
||||||
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
|
}
|
||||||
// Grab the global HBase connection
|
}
|
||||||
synchronized (HBaseIndex.class) {
|
|
||||||
if (hbaseConnection == null) {
|
|
||||||
hbaseConnection = getHBaseConnection();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
|
||||||
HTable hTable = null;
|
|
||||||
try {
|
|
||||||
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
|
|
||||||
// Do the tagging.
|
|
||||||
while (hoodieRecordIterator.hasNext()) {
|
|
||||||
HoodieRecord rec = hoodieRecordIterator.next();
|
|
||||||
// TODO(vc): This may need to be a multi get.
|
|
||||||
Result result = hTable.get(
|
|
||||||
new Get(Bytes.toBytes(rec.getRecordKey())).setMaxVersions(1)
|
|
||||||
.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)
|
|
||||||
.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)
|
|
||||||
.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
|
|
||||||
|
|
||||||
// first, attempt to grab location from HBase
|
@Override
|
||||||
if (result.getRow() != null) {
|
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||||
String commitTs =
|
HoodieTable<T> hoodieTable) {
|
||||||
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
|
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(hoodieTable), true);
|
||||||
String fileId =
|
}
|
||||||
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
|
|
||||||
|
|
||||||
HoodieTimeline commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
class UpdateLocationTask implements
|
||||||
// if the last commit ts for this row is less than the system commit ts
|
Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> {
|
||||||
if (!commitTimeline.empty() && commitTimeline.containsInstant(
|
|
||||||
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))) {
|
|
||||||
rec.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
taggedRecords.add(rec);
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieIndexException(
|
|
||||||
"Failed to Tag indexed locations because of exception with HBase Client", e);
|
|
||||||
}
|
|
||||||
|
|
||||||
finally {
|
@Override
|
||||||
if (hTable != null) {
|
public Iterator<WriteStatus> call(Integer partition, Iterator<WriteStatus> statusIterator) {
|
||||||
try {
|
|
||||||
hTable.close();
|
|
||||||
} catch (IOException e) {
|
|
||||||
// Ignore
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
List<WriteStatus> writeStatusList = new ArrayList<>();
|
||||||
return taggedRecords.iterator();
|
// Grab the global HBase connection
|
||||||
|
synchronized (HBaseIndex.class) {
|
||||||
|
if (hbaseConnection == null) {
|
||||||
|
hbaseConnection = getHBaseConnection();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
HTable hTable = null;
|
||||||
@Override
|
try {
|
||||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieTable<T> hoodieTable) {
|
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
|
||||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(hoodieTable), true);
|
while (statusIterator.hasNext()) {
|
||||||
}
|
WriteStatus writeStatus = statusIterator.next();
|
||||||
|
List<Put> puts = new ArrayList<>();
|
||||||
class UpdateLocationTask implements Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> {
|
List<Delete> deletes = new ArrayList<>();
|
||||||
@Override
|
try {
|
||||||
public Iterator<WriteStatus> call(Integer partition, Iterator<WriteStatus> statusIterator) {
|
for (HoodieRecord rec : writeStatus.getWrittenRecords()) {
|
||||||
|
if (!writeStatus.isErrored(rec.getKey())) {
|
||||||
List<WriteStatus> writeStatusList = new ArrayList<>();
|
java.util.Optional<HoodieRecordLocation> loc = rec.getNewLocation();
|
||||||
// Grab the global HBase connection
|
if (loc.isPresent()) {
|
||||||
synchronized (HBaseIndex.class) {
|
Put put = new Put(Bytes.toBytes(rec.getRecordKey()));
|
||||||
if (hbaseConnection == null) {
|
put.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN,
|
||||||
hbaseConnection = getHBaseConnection();
|
Bytes.toBytes(loc.get().getCommitTime()));
|
||||||
|
put.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN,
|
||||||
|
Bytes.toBytes(loc.get().getFileId()));
|
||||||
|
put.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN,
|
||||||
|
Bytes.toBytes(rec.getPartitionPath()));
|
||||||
|
puts.add(put);
|
||||||
|
} else {
|
||||||
|
//Delete existing index for a deleted record
|
||||||
|
Delete delete = new Delete(Bytes.toBytes(rec.getRecordKey()));
|
||||||
|
deletes.add(delete);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
HTable hTable = null;
|
hTable.put(puts);
|
||||||
try {
|
hTable.delete(deletes);
|
||||||
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
|
hTable.flushCommits();
|
||||||
while (statusIterator.hasNext()) {
|
} catch (Exception e) {
|
||||||
WriteStatus writeStatus = statusIterator.next();
|
Exception we = new Exception("Error updating index for " + writeStatus, e);
|
||||||
List<Put> puts = new ArrayList<>();
|
logger.error(we);
|
||||||
List<Delete> deletes = new ArrayList<>();
|
writeStatus.setGlobalError(we);
|
||||||
try {
|
}
|
||||||
for (HoodieRecord rec : writeStatus.getWrittenRecords()) {
|
writeStatusList.add(writeStatus);
|
||||||
if (!writeStatus.isErrored(rec.getKey())) {
|
|
||||||
java.util.Optional<HoodieRecordLocation> loc = rec.getNewLocation();
|
|
||||||
if(loc.isPresent()) {
|
|
||||||
Put put = new Put(Bytes.toBytes(rec.getRecordKey()));
|
|
||||||
put.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN,
|
|
||||||
Bytes.toBytes(loc.get().getCommitTime()));
|
|
||||||
put.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN,
|
|
||||||
Bytes.toBytes(loc.get().getFileId()));
|
|
||||||
put.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN,
|
|
||||||
Bytes.toBytes(rec.getPartitionPath()));
|
|
||||||
puts.add(put);
|
|
||||||
} else {
|
|
||||||
//Delete existing index for a deleted record
|
|
||||||
Delete delete = new Delete(Bytes.toBytes(rec.getRecordKey()));
|
|
||||||
deletes.add(delete);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
hTable.put(puts);
|
|
||||||
hTable.delete(deletes);
|
|
||||||
hTable.flushCommits();
|
|
||||||
} catch (Exception e) {
|
|
||||||
Exception we = new Exception("Error updating index for " + writeStatus, e);
|
|
||||||
logger.error(we);
|
|
||||||
writeStatus.setGlobalError(we);
|
|
||||||
}
|
|
||||||
writeStatusList.add(writeStatus);
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieIndexException(
|
|
||||||
"Failed to Update Index locations because of exception with HBase Client", e);
|
|
||||||
} finally {
|
|
||||||
if (hTable != null) {
|
|
||||||
try {
|
|
||||||
hTable.close();
|
|
||||||
} catch (IOException e) {
|
|
||||||
// Ignore
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return writeStatusList.iterator();
|
|
||||||
}
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieIndexException(
|
||||||
|
"Failed to Update Index locations because of exception with HBase Client", e);
|
||||||
|
} finally {
|
||||||
|
if (hTable != null) {
|
||||||
|
try {
|
||||||
|
hTable.close();
|
||||||
|
} catch (IOException e) {
|
||||||
|
// Ignore
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return writeStatusList.iterator();
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||||
HoodieTable<T> hoodieTable) {
|
HoodieTable<T> hoodieTable) {
|
||||||
return writeStatusRDD.mapPartitionsWithIndex(new UpdateLocationTask(), true);
|
return writeStatusRDD.mapPartitionsWithIndex(new UpdateLocationTask(), true);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean rollbackCommit(String commitTime) {
|
public boolean rollbackCommit(String commitTime) {
|
||||||
// Can't really rollback here. HBase only can let you go from recordKey to fileID,
|
// Can't really rollback here. HBase only can let you go from recordKey to fileID,
|
||||||
// not the other way around
|
// not the other way around
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Only looks up by recordKey
|
* Only looks up by recordKey
|
||||||
*
|
*/
|
||||||
* @return
|
@Override
|
||||||
*/
|
public boolean isGlobal() {
|
||||||
@Override
|
return true;
|
||||||
public boolean isGlobal() {
|
}
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Mapping is available in HBase already.
|
* Mapping is available in HBase already.
|
||||||
*
|
*/
|
||||||
* @return
|
@Override
|
||||||
*/
|
public boolean canIndexLogFiles() {
|
||||||
@Override
|
return true;
|
||||||
public boolean canIndexLogFiles() {
|
}
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Index needs to be explicitly updated after storage write.
|
* Index needs to be explicitly updated after storage write.
|
||||||
*
|
*/
|
||||||
* @return
|
@Override
|
||||||
*/
|
public boolean isImplicitWithStorage() {
|
||||||
@Override
|
return false;
|
||||||
public boolean isImplicitWithStorage() {
|
}
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -36,13 +36,6 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
|||||||
import com.uber.hoodie.exception.HoodieAppendException;
|
import com.uber.hoodie.exception.HoodieAppendException;
|
||||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
|
||||||
import org.apache.avro.generic.IndexedRecord;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.log4j.LogManager;
|
|
||||||
import org.apache.log4j.Logger;
|
|
||||||
import org.apache.spark.TaskContext;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
@@ -50,155 +43,161 @@ import java.util.List;
|
|||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
import org.apache.avro.generic.GenericRecord;
|
||||||
|
import org.apache.avro.generic.IndexedRecord;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
import org.apache.spark.TaskContext;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* IO Operation to append data onto an existing file.
|
* IO Operation to append data onto an existing file.
|
||||||
*
|
|
||||||
* @param <T>
|
|
||||||
*/
|
*/
|
||||||
public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
||||||
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
|
|
||||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
|
||||||
|
|
||||||
private final WriteStatus writeStatus;
|
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
|
||||||
private final String fileId;
|
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||||
private String partitionPath;
|
|
||||||
private List<HoodieRecord<T>> records;
|
|
||||||
private long recordsWritten = 0;
|
|
||||||
private long recordsDeleted = 0;
|
|
||||||
private HoodieLogFile currentLogFile;
|
|
||||||
private Writer writer;
|
|
||||||
|
|
||||||
public HoodieAppendHandle(HoodieWriteConfig config,
|
private final WriteStatus writeStatus;
|
||||||
String commitTime,
|
private final String fileId;
|
||||||
HoodieTable<T> hoodieTable,
|
private String partitionPath;
|
||||||
String fileId,
|
private List<HoodieRecord<T>> records;
|
||||||
Iterator<HoodieRecord<T>> recordItr) {
|
private long recordsWritten = 0;
|
||||||
super(config, commitTime, hoodieTable);
|
private long recordsDeleted = 0;
|
||||||
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
private HoodieLogFile currentLogFile;
|
||||||
writeStatus.setStat(new HoodieDeltaWriteStat());
|
private Writer writer;
|
||||||
this.writeStatus = writeStatus;
|
|
||||||
this.fileId = fileId;
|
|
||||||
init(recordItr);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void init(Iterator<HoodieRecord<T>> recordItr) {
|
public HoodieAppendHandle(HoodieWriteConfig config,
|
||||||
List<HoodieRecord<T>> records = Lists.newArrayList();
|
String commitTime,
|
||||||
recordItr.forEachRemaining(record -> {
|
HoodieTable<T> hoodieTable,
|
||||||
records.add(record);
|
String fileId,
|
||||||
// extract some information from the first record
|
Iterator<HoodieRecord<T>> recordItr) {
|
||||||
if (partitionPath == null) {
|
super(config, commitTime, hoodieTable);
|
||||||
partitionPath = record.getPartitionPath();
|
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||||
// HACK(vc) This also assumes a base file. It will break, if appending without one.
|
writeStatus.setStat(new HoodieDeltaWriteStat());
|
||||||
String latestValidFilePath =
|
this.writeStatus = writeStatus;
|
||||||
fileSystemView.getLatestDataFiles(record.getPartitionPath())
|
this.fileId = fileId;
|
||||||
.filter(dataFile -> dataFile.getFileId().equals(fileId))
|
init(recordItr);
|
||||||
.findFirst().get().getFileName();
|
}
|
||||||
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
|
|
||||||
writeStatus.getStat().setPrevCommit(baseCommitTime);
|
|
||||||
writeStatus.setFileId(fileId);
|
|
||||||
writeStatus.setPartitionPath(record.getPartitionPath());
|
|
||||||
writeStatus.getStat().setFileId(fileId);
|
|
||||||
|
|
||||||
try {
|
private void init(Iterator<HoodieRecord<T>> recordItr) {
|
||||||
this.writer = HoodieLogFormat.newWriterBuilder()
|
List<HoodieRecord<T>> records = Lists.newArrayList();
|
||||||
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
recordItr.forEachRemaining(record -> {
|
||||||
.withFileId(fileId).overBaseCommit(baseCommitTime)
|
records.add(record);
|
||||||
.withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
// extract some information from the first record
|
||||||
this.currentLogFile = writer.getLogFile();
|
if (partitionPath == null) {
|
||||||
((HoodieDeltaWriteStat) writeStatus.getStat())
|
partitionPath = record.getPartitionPath();
|
||||||
.setLogVersion(currentLogFile.getLogVersion());
|
// HACK(vc) This also assumes a base file. It will break, if appending without one.
|
||||||
((HoodieDeltaWriteStat) writeStatus.getStat())
|
String latestValidFilePath =
|
||||||
.setLogOffset(writer.getCurrentSize());
|
fileSystemView.getLatestDataFiles(record.getPartitionPath())
|
||||||
} catch (Exception e) {
|
.filter(dataFile -> dataFile.getFileId().equals(fileId))
|
||||||
logger.error("Error in update task at commit " + commitTime, e);
|
.findFirst().get().getFileName();
|
||||||
writeStatus.setGlobalError(e);
|
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
|
||||||
throw new HoodieUpsertException(
|
writeStatus.getStat().setPrevCommit(baseCommitTime);
|
||||||
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId
|
writeStatus.setFileId(fileId);
|
||||||
+ " on commit " + commitTime + " on HDFS path " + hoodieTable
|
writeStatus.setPartitionPath(record.getPartitionPath());
|
||||||
.getMetaClient().getBasePath() + partitionPath, e);
|
writeStatus.getStat().setFileId(fileId);
|
||||||
}
|
|
||||||
Path path = new Path(record.getPartitionPath(),
|
|
||||||
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
|
|
||||||
writeStatus.getStat().setPath(path.toString());
|
|
||||||
}
|
|
||||||
// update the new location of the record, so we know where to find it next
|
|
||||||
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
|
||||||
});
|
|
||||||
this.records = records;
|
|
||||||
}
|
|
||||||
|
|
||||||
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
|
||||||
Optional recordMetadata = hoodieRecord.getData().getMetadata();
|
|
||||||
try {
|
try {
|
||||||
Optional<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(schema);
|
this.writer = HoodieLogFormat.newWriterBuilder()
|
||||||
|
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||||
if(avroRecord.isPresent()) {
|
.withFileId(fileId).overBaseCommit(baseCommitTime)
|
||||||
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
.withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||||
recordIndex.getAndIncrement());
|
this.currentLogFile = writer.getLogFile();
|
||||||
HoodieAvroUtils
|
((HoodieDeltaWriteStat) writeStatus.getStat())
|
||||||
.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(),
|
.setLogVersion(currentLogFile.getLogVersion());
|
||||||
hoodieRecord.getPartitionPath(), fileId);
|
((HoodieDeltaWriteStat) writeStatus.getStat())
|
||||||
HoodieAvroUtils
|
.setLogOffset(writer.getCurrentSize());
|
||||||
.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), commitTime, seqId);
|
|
||||||
recordsWritten++;
|
|
||||||
} else {
|
|
||||||
recordsDeleted++;
|
|
||||||
}
|
|
||||||
|
|
||||||
hoodieRecord.deflate();
|
|
||||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
|
||||||
return avroRecord;
|
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
logger.error("Error writing record " + hoodieRecord, e);
|
logger.error("Error in update task at commit " + commitTime, e);
|
||||||
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
|
writeStatus.setGlobalError(e);
|
||||||
|
throw new HoodieUpsertException(
|
||||||
|
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId
|
||||||
|
+ " on commit " + commitTime + " on HDFS path " + hoodieTable
|
||||||
|
.getMetaClient().getBasePath() + partitionPath, e);
|
||||||
}
|
}
|
||||||
return Optional.empty();
|
Path path = new Path(record.getPartitionPath(),
|
||||||
}
|
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
|
||||||
|
writeStatus.getStat().setPath(path.toString());
|
||||||
|
}
|
||||||
|
// update the new location of the record, so we know where to find it next
|
||||||
|
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
||||||
|
});
|
||||||
|
this.records = records;
|
||||||
|
}
|
||||||
|
|
||||||
public void doAppend() {
|
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
||||||
|
Optional recordMetadata = hoodieRecord.getData().getMetadata();
|
||||||
|
try {
|
||||||
|
Optional<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(schema);
|
||||||
|
|
||||||
List<IndexedRecord> recordList = new ArrayList<>();
|
if (avroRecord.isPresent()) {
|
||||||
List<String> keysToDelete = new ArrayList<>();
|
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
||||||
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
|
recordIndex.getAndIncrement());
|
||||||
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, commitTime);
|
HoodieAvroUtils
|
||||||
records.stream().forEach(record -> {
|
.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(),
|
||||||
Optional<IndexedRecord> indexedRecord = getIndexedRecord(record);
|
hoodieRecord.getPartitionPath(), fileId);
|
||||||
if(indexedRecord.isPresent()) {
|
HoodieAvroUtils
|
||||||
recordList.add(indexedRecord.get());
|
.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), commitTime, seqId);
|
||||||
} else {
|
recordsWritten++;
|
||||||
keysToDelete.add(record.getRecordKey());
|
} else {
|
||||||
}
|
recordsDeleted++;
|
||||||
});
|
}
|
||||||
try {
|
|
||||||
if(recordList.size() > 0) {
|
|
||||||
writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, schema, metadata));
|
|
||||||
}
|
|
||||||
if(keysToDelete.size() > 0) {
|
|
||||||
writer = writer.appendBlock(new HoodieDeleteBlock(keysToDelete.stream().toArray(String[]::new), metadata));
|
|
||||||
}
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new HoodieAppendException(
|
|
||||||
"Failed while appeding records to " + currentLogFile.getPath(), e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public void close() {
|
hoodieRecord.deflate();
|
||||||
try {
|
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||||
if (writer != null) {
|
return avroRecord;
|
||||||
writer.close();
|
} catch (Exception e) {
|
||||||
}
|
logger.error("Error writing record " + hoodieRecord, e);
|
||||||
writeStatus.getStat().setNumWrites(recordsWritten);
|
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
|
||||||
writeStatus.getStat().setNumDeletes(recordsDeleted);
|
|
||||||
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
return Optional.empty();
|
||||||
|
}
|
||||||
|
|
||||||
public WriteStatus getWriteStatus() {
|
public void doAppend() {
|
||||||
return writeStatus;
|
|
||||||
|
List<IndexedRecord> recordList = new ArrayList<>();
|
||||||
|
List<String> keysToDelete = new ArrayList<>();
|
||||||
|
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
|
||||||
|
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, commitTime);
|
||||||
|
records.stream().forEach(record -> {
|
||||||
|
Optional<IndexedRecord> indexedRecord = getIndexedRecord(record);
|
||||||
|
if (indexedRecord.isPresent()) {
|
||||||
|
recordList.add(indexedRecord.get());
|
||||||
|
} else {
|
||||||
|
keysToDelete.add(record.getRecordKey());
|
||||||
|
}
|
||||||
|
});
|
||||||
|
try {
|
||||||
|
if (recordList.size() > 0) {
|
||||||
|
writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, schema, metadata));
|
||||||
|
}
|
||||||
|
if (keysToDelete.size() > 0) {
|
||||||
|
writer = writer.appendBlock(
|
||||||
|
new HoodieDeleteBlock(keysToDelete.stream().toArray(String[]::new), metadata));
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new HoodieAppendException(
|
||||||
|
"Failed while appeding records to " + currentLogFile.getPath(), e);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void close() {
|
||||||
|
try {
|
||||||
|
if (writer != null) {
|
||||||
|
writer.close();
|
||||||
|
}
|
||||||
|
writeStatus.getStat().setNumWrites(recordsWritten);
|
||||||
|
writeStatus.getStat().setNumDeletes(recordsDeleted);
|
||||||
|
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public WriteStatus getWriteStatus() {
|
||||||
|
return writeStatus;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -27,226 +27,212 @@ import com.uber.hoodie.common.table.TableFileSystemView;
|
|||||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.log4j.LogManager;
|
|
||||||
import org.apache.log4j.Logger;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
|
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
|
||||||
* <p>
|
* <p> 1) It provides sufficient time for existing queries running on older versions, to finish <p>
|
||||||
* 1) It provides sufficient time for existing queries running on older versions, to finish
|
* 2) It bounds the growth of the files in the file system <p> TODO: Should all cleaning be done
|
||||||
* <p>
|
* based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata}
|
||||||
* 2) It bounds the growth of the files in the file system
|
|
||||||
* <p>
|
|
||||||
* TODO: Should all cleaning be done based on {@link com.uber.hoodie.common.model.HoodieCommitMetadata}
|
|
||||||
*/
|
*/
|
||||||
public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||||
private static Logger logger = LogManager.getLogger(HoodieCleanHelper.class);
|
|
||||||
|
|
||||||
private final TableFileSystemView fileSystemView;
|
private static Logger logger = LogManager.getLogger(HoodieCleanHelper.class);
|
||||||
private final HoodieTimeline commitTimeline;
|
|
||||||
private HoodieTable<T> hoodieTable;
|
|
||||||
private HoodieWriteConfig config;
|
|
||||||
private FileSystem fs;
|
|
||||||
|
|
||||||
public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
|
private final TableFileSystemView fileSystemView;
|
||||||
this.hoodieTable = hoodieTable;
|
private final HoodieTimeline commitTimeline;
|
||||||
this.fileSystemView = hoodieTable.getCompletedFileSystemView();
|
private HoodieTable<T> hoodieTable;
|
||||||
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
private HoodieWriteConfig config;
|
||||||
this.config = config;
|
private FileSystem fs;
|
||||||
this.fs = hoodieTable.getFs();
|
|
||||||
|
public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
|
||||||
|
this.hoodieTable = hoodieTable;
|
||||||
|
this.fileSystemView = hoodieTable.getCompletedFileSystemView();
|
||||||
|
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||||
|
this.config = config;
|
||||||
|
this.fs = hoodieTable.getFs();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Selects the older versions of files for cleaning, such that it bounds the number of versions of
|
||||||
|
* each file. This policy is useful, if you are simply interested in querying the table, and you
|
||||||
|
* don't want too many versions for a single file (i.e run it with versionsRetained = 1)
|
||||||
|
*/
|
||||||
|
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath)
|
||||||
|
throws IOException {
|
||||||
|
logger.info("Cleaning " + partitionPath + ", retaining latest " + config
|
||||||
|
.getCleanerFileVersionsRetained() + " file versions. ");
|
||||||
|
List<HoodieFileGroup> fileGroups =
|
||||||
|
fileSystemView.getAllFileGroups(partitionPath)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
List<String> deletePaths = new ArrayList<>();
|
||||||
|
// Collect all the datafiles savepointed by all the savepoints
|
||||||
|
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
|
||||||
|
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
|
||||||
|
|
||||||
|
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||||
|
int keepVersions = config.getCleanerFileVersionsRetained();
|
||||||
|
Iterator<FileSlice> fileSliceIterator = fileGroup.getAllFileSlices().iterator();
|
||||||
|
while (fileSliceIterator.hasNext() && keepVersions > 0) {
|
||||||
|
// Skip this most recent version
|
||||||
|
FileSlice nextSlice = fileSliceIterator.next();
|
||||||
|
HoodieDataFile dataFile = nextSlice.getDataFile().get();
|
||||||
|
if (savepointedFiles.contains(dataFile.getFileName())) {
|
||||||
|
// do not clean up a savepoint data file
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
keepVersions--;
|
||||||
|
}
|
||||||
|
// Delete the remaining files
|
||||||
|
while (fileSliceIterator.hasNext()) {
|
||||||
|
FileSlice nextSlice = fileSliceIterator.next();
|
||||||
|
HoodieDataFile dataFile = nextSlice.getDataFile().get();
|
||||||
|
deletePaths.add(dataFile.getFileStatus().getPath().toString());
|
||||||
|
if (hoodieTable.getMetaClient().getTableType()
|
||||||
|
== HoodieTableType.MERGE_ON_READ) {
|
||||||
|
// If merge on read, then clean the log files for the commits as well
|
||||||
|
deletePaths.addAll(nextSlice.getLogFiles()
|
||||||
|
.map(file -> file.getPath().toString())
|
||||||
|
.collect(Collectors.toList()));
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
return deletePaths;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Selects the older versions of files for cleaning, such that it bounds the number of versions of each file.
|
* Selects the versions for file for cleaning, such that it <p> - Leaves the latest version of the
|
||||||
* This policy is useful, if you are simply interested in querying the table, and you don't want too many
|
* file untouched - For older versions, - It leaves all the commits untouched which has occured in
|
||||||
* versions for a single file (i.e run it with versionsRetained = 1)
|
* last <code>config.getCleanerCommitsRetained()</code> commits - It leaves ONE commit before this
|
||||||
*
|
* window. We assume that the max(query execution time) == commit_batch_time *
|
||||||
* @param partitionPath
|
* config.getCleanerCommitsRetained(). This is 12 hours by default. This is essential to leave the
|
||||||
* @return
|
* file used by the query thats running for the max time. <p> This provides the effect of having
|
||||||
* @throws IOException
|
* lookback into all changes that happened in the last X commits. (eg: if you retain 24 commits,
|
||||||
*/
|
* and commit batch time is 30 mins, then you have 12 hrs of lookback) <p> This policy is the
|
||||||
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath)
|
* default.
|
||||||
throws IOException {
|
*/
|
||||||
logger.info("Cleaning " + partitionPath + ", retaining latest " + config
|
private List<String> getFilesToCleanKeepingLatestCommits(String partitionPath)
|
||||||
.getCleanerFileVersionsRetained() + " file versions. ");
|
throws IOException {
|
||||||
List<HoodieFileGroup> fileGroups =
|
int commitsRetained = config.getCleanerCommitsRetained();
|
||||||
fileSystemView.getAllFileGroups(partitionPath)
|
logger.info(
|
||||||
.collect(Collectors.toList());
|
"Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. ");
|
||||||
List<String> deletePaths = new ArrayList<>();
|
List<String> deletePaths = new ArrayList<>();
|
||||||
// Collect all the datafiles savepointed by all the savepoints
|
|
||||||
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
|
|
||||||
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
|
|
||||||
|
|
||||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
// Collect all the datafiles savepointed by all the savepoints
|
||||||
int keepVersions = config.getCleanerFileVersionsRetained();
|
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
|
||||||
Iterator<FileSlice> fileSliceIterator = fileGroup.getAllFileSlices().iterator();
|
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
|
||||||
while (fileSliceIterator.hasNext() && keepVersions > 0) {
|
|
||||||
// Skip this most recent version
|
// determine if we have enough commits, to start cleaning.
|
||||||
FileSlice nextSlice = fileSliceIterator.next();
|
if (commitTimeline.countInstants() > commitsRetained) {
|
||||||
HoodieDataFile dataFile = nextSlice.getDataFile().get();
|
HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get();
|
||||||
if(savepointedFiles.contains(dataFile.getFileName())) {
|
List<HoodieFileGroup> fileGroups =
|
||||||
// do not clean up a savepoint data file
|
fileSystemView.getAllFileGroups(partitionPath)
|
||||||
continue;
|
.collect(Collectors.toList());
|
||||||
}
|
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||||
keepVersions--;
|
List<FileSlice> fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList());
|
||||||
}
|
HoodieDataFile dataFile = fileSliceList.get(0).getDataFile().get();
|
||||||
// Delete the remaining files
|
String lastVersion = dataFile.getCommitTime();
|
||||||
while (fileSliceIterator.hasNext()) {
|
String lastVersionBeforeEarliestCommitToRetain =
|
||||||
FileSlice nextSlice = fileSliceIterator.next();
|
getLatestVersionBeforeCommit(fileSliceList, earliestCommitToRetain);
|
||||||
HoodieDataFile dataFile = nextSlice.getDataFile().get();
|
|
||||||
deletePaths.add(dataFile.getFileStatus().getPath().toString());
|
// Ensure there are more than 1 version of the file (we only clean old files from updates)
|
||||||
if (hoodieTable.getMetaClient().getTableType()
|
// i.e always spare the last commit.
|
||||||
== HoodieTableType.MERGE_ON_READ) {
|
for (FileSlice aSlice : fileSliceList) {
|
||||||
// If merge on read, then clean the log files for the commits as well
|
HoodieDataFile aFile = aSlice.getDataFile().get();
|
||||||
deletePaths.addAll(nextSlice.getLogFiles()
|
String fileCommitTime = aFile.getCommitTime();
|
||||||
.map(file -> file.getPath().toString())
|
if (savepointedFiles.contains(aFile.getFileName())) {
|
||||||
.collect(Collectors.toList()));
|
// do not clean up a savepoint data file
|
||||||
}
|
continue;
|
||||||
|
}
|
||||||
|
// 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.
|
||||||
|
if (fileCommitTime.equals(lastVersion) || (
|
||||||
|
lastVersionBeforeEarliestCommitToRetain != null && fileCommitTime
|
||||||
|
.equals(lastVersionBeforeEarliestCommitToRetain))) {
|
||||||
|
// move on to the next file
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Always keep the last commit
|
||||||
|
if (HoodieTimeline.compareTimestamps(
|
||||||
|
earliestCommitToRetain.getTimestamp(),
|
||||||
|
fileCommitTime,
|
||||||
|
HoodieTimeline.GREATER)) {
|
||||||
|
// this is a commit, that should be cleaned.
|
||||||
|
deletePaths.add(aFile.getFileStatus().getPath().toString());
|
||||||
|
if (hoodieTable.getMetaClient().getTableType()
|
||||||
|
== HoodieTableType.MERGE_ON_READ) {
|
||||||
|
// If merge on read, then clean the log files for the commits as well
|
||||||
|
deletePaths.addAll(aSlice.getLogFiles()
|
||||||
|
.map(file -> file.getPath().toString())
|
||||||
|
.collect(Collectors.toList()));
|
||||||
}
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
return deletePaths;
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
return deletePaths;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Selects the versions for file for cleaning, such that it
|
* Gets the latest version < commitTime. This version file could still be used by queries.
|
||||||
* <p>
|
*/
|
||||||
* - Leaves the latest version of the file untouched
|
private String getLatestVersionBeforeCommit(List<FileSlice> fileSliceList,
|
||||||
* - For older versions,
|
HoodieInstant commitTime) {
|
||||||
* - It leaves all the commits untouched which has occured in last <code>config.getCleanerCommitsRetained()</code> commits
|
for (FileSlice file : fileSliceList) {
|
||||||
* - It leaves ONE commit before this window. We assume that the max(query execution time) == commit_batch_time * config.getCleanerCommitsRetained(). This is 12 hours by default.
|
String fileCommitTime = file.getDataFile().get().getCommitTime();
|
||||||
* This is essential to leave the file used by the query thats running for the max time.
|
if (HoodieTimeline.compareTimestamps(commitTime.getTimestamp(), fileCommitTime,
|
||||||
* <p>
|
HoodieTimeline.GREATER)) {
|
||||||
* This provides the effect of having lookback into all changes that happened in the last X
|
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the one we want
|
||||||
* commits. (eg: if you retain 24 commits, and commit batch time is 30 mins, then you have 12 hrs of lookback)
|
return fileCommitTime;
|
||||||
* <p>
|
}
|
||||||
* This policy is the default.
|
|
||||||
*
|
|
||||||
* @param partitionPath
|
|
||||||
* @return
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
private List<String> getFilesToCleanKeepingLatestCommits(String partitionPath)
|
|
||||||
throws IOException {
|
|
||||||
int commitsRetained = config.getCleanerCommitsRetained();
|
|
||||||
logger.info(
|
|
||||||
"Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. ");
|
|
||||||
List<String> deletePaths = new ArrayList<>();
|
|
||||||
|
|
||||||
// Collect all the datafiles savepointed by all the savepoints
|
|
||||||
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
|
|
||||||
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
|
|
||||||
|
|
||||||
// determine if we have enough commits, to start cleaning.
|
|
||||||
if (commitTimeline.countInstants() > commitsRetained) {
|
|
||||||
HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get();
|
|
||||||
List<HoodieFileGroup> fileGroups =
|
|
||||||
fileSystemView.getAllFileGroups(partitionPath)
|
|
||||||
.collect(Collectors.toList());
|
|
||||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
|
||||||
List<FileSlice> fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList());
|
|
||||||
HoodieDataFile dataFile = fileSliceList.get(0).getDataFile().get();
|
|
||||||
String lastVersion = dataFile.getCommitTime();
|
|
||||||
String lastVersionBeforeEarliestCommitToRetain =
|
|
||||||
getLatestVersionBeforeCommit(fileSliceList, 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 (FileSlice aSlice : fileSliceList) {
|
|
||||||
HoodieDataFile aFile = aSlice.getDataFile().get();
|
|
||||||
String fileCommitTime = aFile.getCommitTime();
|
|
||||||
if(savepointedFiles.contains(aFile.getFileName())) {
|
|
||||||
// do not clean up a savepoint data file
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
// 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.
|
|
||||||
if (fileCommitTime.equals(lastVersion) || (
|
|
||||||
lastVersionBeforeEarliestCommitToRetain != null && fileCommitTime
|
|
||||||
.equals(lastVersionBeforeEarliestCommitToRetain))) {
|
|
||||||
// move on to the next file
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
// Always keep the last commit
|
|
||||||
if (HoodieTimeline.compareTimestamps(
|
|
||||||
earliestCommitToRetain.getTimestamp(),
|
|
||||||
fileCommitTime,
|
|
||||||
HoodieTimeline.GREATER)) {
|
|
||||||
// this is a commit, that should be cleaned.
|
|
||||||
deletePaths.add(aFile.getFileStatus().getPath().toString());
|
|
||||||
if (hoodieTable.getMetaClient().getTableType()
|
|
||||||
== HoodieTableType.MERGE_ON_READ) {
|
|
||||||
// If merge on read, then clean the log files for the commits as well
|
|
||||||
deletePaths.addAll(aSlice.getLogFiles()
|
|
||||||
.map(file -> file.getPath().toString())
|
|
||||||
.collect(Collectors.toList()));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return deletePaths;
|
|
||||||
}
|
}
|
||||||
|
// There is no version of this file which is <= commitTime
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Gets the latest version < commitTime. This version file could still be used by queries.
|
* Returns files to be cleaned for the given partitionPath based on cleaning policy.
|
||||||
*/
|
*/
|
||||||
private String getLatestVersionBeforeCommit(List<FileSlice> fileSliceList,
|
public List<String> getDeletePaths(String partitionPath) throws IOException {
|
||||||
HoodieInstant commitTime) {
|
HoodieCleaningPolicy policy = config.getCleanerPolicy();
|
||||||
for (FileSlice file : fileSliceList) {
|
List<String> deletePaths;
|
||||||
String fileCommitTime = file.getDataFile().get().getCommitTime();
|
if (policy == HoodieCleaningPolicy.KEEP_LATEST_COMMITS) {
|
||||||
if (HoodieTimeline.compareTimestamps(commitTime.getTimestamp(), fileCommitTime,
|
deletePaths = getFilesToCleanKeepingLatestCommits(partitionPath);
|
||||||
HoodieTimeline.GREATER)) {
|
} else if (policy == HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) {
|
||||||
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the one we want
|
deletePaths = getFilesToCleanKeepingLatestVersions(partitionPath);
|
||||||
return fileCommitTime;
|
} else {
|
||||||
}
|
throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name());
|
||||||
}
|
|
||||||
// There is no version of this file which is <= commitTime
|
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
|
logger.info(
|
||||||
|
deletePaths.size() + " patterns used to delete in partition path:" + partitionPath);
|
||||||
|
|
||||||
/**
|
return deletePaths;
|
||||||
* Returns files to be cleaned for the given partitionPath based on cleaning policy.
|
}
|
||||||
*/
|
|
||||||
public List<String> getDeletePaths(String partitionPath) throws IOException {
|
|
||||||
HoodieCleaningPolicy policy = config.getCleanerPolicy();
|
|
||||||
List<String> deletePaths;
|
|
||||||
if (policy == HoodieCleaningPolicy.KEEP_LATEST_COMMITS) {
|
|
||||||
deletePaths = getFilesToCleanKeepingLatestCommits(partitionPath);
|
|
||||||
} else if (policy == HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) {
|
|
||||||
deletePaths = getFilesToCleanKeepingLatestVersions(partitionPath);
|
|
||||||
} else {
|
|
||||||
throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name());
|
|
||||||
}
|
|
||||||
logger.info(
|
|
||||||
deletePaths.size() + " patterns used to delete in partition path:" + partitionPath);
|
|
||||||
|
|
||||||
return deletePaths;
|
/**
|
||||||
}
|
* Returns earliest commit to retain based on cleaning policy.
|
||||||
|
*/
|
||||||
/**
|
public Optional<HoodieInstant> getEarliestCommitToRetain() {
|
||||||
* Returns earliest commit to retain based on cleaning policy.
|
Optional<HoodieInstant> earliestCommitToRetain = Optional.empty();
|
||||||
*/
|
int commitsRetained = config.getCleanerCommitsRetained();
|
||||||
public Optional<HoodieInstant> getEarliestCommitToRetain() {
|
if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS
|
||||||
Optional<HoodieInstant> earliestCommitToRetain = Optional.empty();
|
&& commitTimeline.countInstants() > commitsRetained) {
|
||||||
int commitsRetained = config.getCleanerCommitsRetained();
|
earliestCommitToRetain =
|
||||||
if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS
|
commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained);
|
||||||
&& commitTimeline.countInstants() > commitsRetained) {
|
|
||||||
earliestCommitToRetain =
|
|
||||||
commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained);
|
|
||||||
}
|
|
||||||
return earliestCommitToRetain;
|
|
||||||
}
|
}
|
||||||
|
return earliestCommitToRetain;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -39,6 +39,12 @@ import com.uber.hoodie.exception.HoodieCommitException;
|
|||||||
import com.uber.hoodie.exception.HoodieException;
|
import com.uber.hoodie.exception.HoodieException;
|
||||||
import com.uber.hoodie.exception.HoodieIOException;
|
import com.uber.hoodie.exception.HoodieIOException;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.Stream;
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.avro.generic.IndexedRecord;
|
import org.apache.avro.generic.IndexedRecord;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
@@ -46,225 +52,233 @@ import org.apache.hadoop.fs.Path;
|
|||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
import java.util.stream.Stream;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Archiver to bound the growth of <action>.commit files
|
* Archiver to bound the growth of <action>.commit files
|
||||||
*/
|
*/
|
||||||
public class HoodieCommitArchiveLog {
|
public class HoodieCommitArchiveLog {
|
||||||
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
|
|
||||||
|
|
||||||
private final Path archiveFilePath;
|
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
|
||||||
private final FileSystem fs;
|
|
||||||
private final HoodieWriteConfig config;
|
|
||||||
private HoodieLogFormat.Writer writer;
|
|
||||||
|
|
||||||
public HoodieCommitArchiveLog(HoodieWriteConfig config, FileSystem fs) {
|
private final Path archiveFilePath;
|
||||||
this.fs = fs;
|
private final FileSystem fs;
|
||||||
this.config = config;
|
private final HoodieWriteConfig config;
|
||||||
this.archiveFilePath = HoodieArchivedTimeline
|
private HoodieLogFormat.Writer writer;
|
||||||
.getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME);
|
|
||||||
|
public HoodieCommitArchiveLog(HoodieWriteConfig config, FileSystem fs) {
|
||||||
|
this.fs = fs;
|
||||||
|
this.config = config;
|
||||||
|
this.archiveFilePath = HoodieArchivedTimeline
|
||||||
|
.getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME);
|
||||||
|
}
|
||||||
|
|
||||||
|
private HoodieLogFormat.Writer openWriter() {
|
||||||
|
try {
|
||||||
|
if (this.writer == null) {
|
||||||
|
return HoodieLogFormat.newWriterBuilder()
|
||||||
|
.onParentPath(archiveFilePath.getParent())
|
||||||
|
.withFileId(archiveFilePath.getName())
|
||||||
|
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION)
|
||||||
|
.withFs(fs)
|
||||||
|
.overBaseCommit("").build();
|
||||||
|
} else {
|
||||||
|
return this.writer;
|
||||||
|
}
|
||||||
|
} catch (InterruptedException | IOException e) {
|
||||||
|
throw new HoodieException("Unable to initialize HoodieLogFormat writer", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void close() {
|
||||||
|
try {
|
||||||
|
if (this.writer != null) {
|
||||||
|
this.writer.close();
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieException("Unable to close HoodieLogFormat writer", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check if commits need to be archived. If yes, archive commits.
|
||||||
|
*/
|
||||||
|
public boolean archiveIfRequired() {
|
||||||
|
try {
|
||||||
|
List<HoodieInstant> instantsToArchive = getInstantsToArchive().collect(Collectors.toList());
|
||||||
|
boolean success = true;
|
||||||
|
if (instantsToArchive.iterator().hasNext()) {
|
||||||
|
this.writer = openWriter();
|
||||||
|
log.info("Archiving instants " + instantsToArchive);
|
||||||
|
archive(instantsToArchive);
|
||||||
|
success = deleteArchivedInstants(instantsToArchive);
|
||||||
|
} else {
|
||||||
|
log.info("No Instants to archive");
|
||||||
|
}
|
||||||
|
return success;
|
||||||
|
} finally {
|
||||||
|
close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private Stream<HoodieInstant> getInstantsToArchive() {
|
||||||
|
|
||||||
|
// TODO : rename to max/minInstantsToKeep
|
||||||
|
int maxCommitsToKeep = config.getMaxCommitsToKeep();
|
||||||
|
int minCommitsToKeep = config.getMinCommitsToKeep();
|
||||||
|
|
||||||
|
HoodieTable table = HoodieTable
|
||||||
|
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||||
|
|
||||||
|
// GroupBy each action and limit each action timeline to maxCommitsToKeep
|
||||||
|
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline()
|
||||||
|
.getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION,
|
||||||
|
HoodieTimeline.ROLLBACK_ACTION));
|
||||||
|
Stream<HoodieInstant> instants = cleanAndRollbackTimeline.getInstants()
|
||||||
|
.collect(Collectors.groupingBy(s -> s.getAction()))
|
||||||
|
.entrySet()
|
||||||
|
.stream()
|
||||||
|
.map(i -> {
|
||||||
|
if (i.getValue().size() > maxCommitsToKeep) {
|
||||||
|
return i.getValue().subList(0, i.getValue().size() - minCommitsToKeep);
|
||||||
|
} else {
|
||||||
|
return new ArrayList<HoodieInstant>();
|
||||||
|
}
|
||||||
|
})
|
||||||
|
.flatMap(i -> i.stream());
|
||||||
|
|
||||||
|
//TODO (na) : Add a way to return actions associated with a timeline and then merge/unify with logic above to avoid Stream.concats
|
||||||
|
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
|
||||||
|
// We cannot have any holes in the commit timeline. We cannot archive any commits which are made after the first savepoint present.
|
||||||
|
Optional<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
|
||||||
|
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
|
||||||
|
// Actually do the commits
|
||||||
|
instants = Stream.concat(instants, commitTimeline.getInstants().filter(s -> {
|
||||||
|
// if no savepoint present, then dont filter
|
||||||
|
return !(firstSavepoint.isPresent() && HoodieTimeline
|
||||||
|
.compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(),
|
||||||
|
HoodieTimeline.LESSER_OR_EQUAL));
|
||||||
|
}).limit(commitTimeline.countInstants() - minCommitsToKeep));
|
||||||
}
|
}
|
||||||
|
|
||||||
private HoodieLogFormat.Writer openWriter() {
|
return instants;
|
||||||
try {
|
}
|
||||||
if(this.writer == null) {
|
|
||||||
return HoodieLogFormat.newWriterBuilder()
|
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) {
|
||||||
.onParentPath(archiveFilePath.getParent())
|
log.info("Deleting instants " + archivedInstants);
|
||||||
.withFileId(archiveFilePath.getName())
|
HoodieTableMetaClient metaClient =
|
||||||
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION)
|
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||||
.withFs(fs)
|
|
||||||
.overBaseCommit("").build();
|
boolean success = true;
|
||||||
} else {
|
for (HoodieInstant archivedInstant : archivedInstants) {
|
||||||
return this.writer;
|
Path commitFile =
|
||||||
}
|
new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
|
||||||
} catch(InterruptedException | IOException e) {
|
try {
|
||||||
throw new HoodieException("Unable to initialize HoodieLogFormat writer", e);
|
if (fs.exists(commitFile)) {
|
||||||
|
success &= fs.delete(commitFile, false);
|
||||||
|
log.info("Archived and deleted instant file " + commitFile);
|
||||||
}
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieIOException("Failed to delete archived instant " + archivedInstant,
|
||||||
|
e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
return success;
|
||||||
|
}
|
||||||
|
|
||||||
private void close() {
|
public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
|
||||||
try {
|
|
||||||
if(this.writer != null) {
|
try {
|
||||||
this.writer.close();
|
HoodieTableMetaClient metaClient =
|
||||||
}
|
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||||
} catch(IOException e) {
|
HoodieTimeline commitTimeline =
|
||||||
throw new HoodieException("Unable to close HoodieLogFormat writer", e);
|
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();
|
||||||
}
|
|
||||||
|
Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
|
||||||
|
log.info("Wrapper schema " + wrapperSchema.toString());
|
||||||
|
List<IndexedRecord> records = new ArrayList<>();
|
||||||
|
for (HoodieInstant hoodieInstant : instants) {
|
||||||
|
records.add(convertToAvroRecord(commitTimeline, hoodieInstant));
|
||||||
|
}
|
||||||
|
HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, wrapperSchema);
|
||||||
|
this.writer = writer.appendBlock(block);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new HoodieCommitException("Failed to archive commits", e);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
public Path getArchiveFilePath() {
|
||||||
* Check if commits need to be archived. If yes, archive commits.
|
return archiveFilePath;
|
||||||
*/
|
}
|
||||||
public boolean archiveIfRequired() {
|
|
||||||
try {
|
private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline,
|
||||||
List<HoodieInstant> instantsToArchive = getInstantsToArchive().collect(Collectors.toList());
|
HoodieInstant hoodieInstant) throws IOException {
|
||||||
boolean success = true;
|
HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry();
|
||||||
if (instantsToArchive.iterator().hasNext()) {
|
archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp());
|
||||||
this.writer = openWriter();
|
switch (hoodieInstant.getAction()) {
|
||||||
log.info("Archiving instants " + instantsToArchive);
|
case HoodieTimeline.CLEAN_ACTION: {
|
||||||
archive(instantsToArchive);
|
archivedMetaWrapper.setHoodieCleanMetadata(AvroUtils
|
||||||
success = deleteArchivedInstants(instantsToArchive);
|
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
|
||||||
} else {
|
HoodieCleanMetadata.class));
|
||||||
log.info("No Instants to archive");
|
archivedMetaWrapper.setActionType(ActionType.clean.name());
|
||||||
}
|
break;
|
||||||
return success;
|
}
|
||||||
} finally {
|
case HoodieTimeline.COMMIT_ACTION: {
|
||||||
close();
|
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||||
}
|
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
|
||||||
|
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
|
||||||
|
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case HoodieTimeline.COMPACTION_ACTION: {
|
||||||
|
com.uber.hoodie.common.model.HoodieCompactionMetadata compactionMetadata = com.uber.hoodie.common.model.HoodieCompactionMetadata
|
||||||
|
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
|
||||||
|
archivedMetaWrapper
|
||||||
|
.setHoodieCompactionMetadata(compactionMetadataConverter(compactionMetadata));
|
||||||
|
archivedMetaWrapper.setActionType(ActionType.compaction.name());
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case HoodieTimeline.ROLLBACK_ACTION: {
|
||||||
|
archivedMetaWrapper.setHoodieRollbackMetadata(AvroUtils
|
||||||
|
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
|
||||||
|
HoodieRollbackMetadata.class));
|
||||||
|
archivedMetaWrapper.setActionType(ActionType.rollback.name());
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case HoodieTimeline.SAVEPOINT_ACTION: {
|
||||||
|
archivedMetaWrapper.setHoodieSavePointMetadata(AvroUtils
|
||||||
|
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
|
||||||
|
HoodieSavepointMetadata.class));
|
||||||
|
archivedMetaWrapper.setActionType(ActionType.savepoint.name());
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case HoodieTimeline.DELTA_COMMIT_ACTION: {
|
||||||
|
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||||
|
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
|
||||||
|
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
|
||||||
|
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||||
|
break;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
return archivedMetaWrapper;
|
||||||
|
}
|
||||||
|
|
||||||
private Stream<HoodieInstant> getInstantsToArchive() {
|
private com.uber.hoodie.avro.model.HoodieCommitMetadata commitMetadataConverter(
|
||||||
|
HoodieCommitMetadata hoodieCommitMetadata) {
|
||||||
|
ObjectMapper mapper = new ObjectMapper();
|
||||||
|
//Need this to ignore other public get() methods
|
||||||
|
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||||
|
com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData =
|
||||||
|
mapper.convertValue(hoodieCommitMetadata,
|
||||||
|
com.uber.hoodie.avro.model.HoodieCommitMetadata.class);
|
||||||
|
return avroMetaData;
|
||||||
|
}
|
||||||
|
|
||||||
// TODO : rename to max/minInstantsToKeep
|
private com.uber.hoodie.avro.model.HoodieCompactionMetadata compactionMetadataConverter(
|
||||||
int maxCommitsToKeep = config.getMaxCommitsToKeep();
|
HoodieCompactionMetadata hoodieCompactionMetadata) {
|
||||||
int minCommitsToKeep = config.getMinCommitsToKeep();
|
ObjectMapper mapper = new ObjectMapper();
|
||||||
|
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
com.uber.hoodie.avro.model.HoodieCompactionMetadata avroMetaData = mapper
|
||||||
|
.convertValue(hoodieCompactionMetadata,
|
||||||
// GroupBy each action and limit each action timeline to maxCommitsToKeep
|
com.uber.hoodie.avro.model.HoodieCompactionMetadata.class);
|
||||||
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline().getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION,
|
return avroMetaData;
|
||||||
HoodieTimeline.ROLLBACK_ACTION));
|
}
|
||||||
Stream<HoodieInstant> instants = cleanAndRollbackTimeline.getInstants()
|
|
||||||
.collect(Collectors.groupingBy(s -> s.getAction()))
|
|
||||||
.entrySet()
|
|
||||||
.stream()
|
|
||||||
.map(i -> {
|
|
||||||
if (i.getValue().size() > maxCommitsToKeep) {
|
|
||||||
return i.getValue().subList(0, i.getValue().size() - minCommitsToKeep);
|
|
||||||
} else {
|
|
||||||
return new ArrayList<HoodieInstant>();
|
|
||||||
}
|
|
||||||
})
|
|
||||||
.flatMap(i -> i.stream());
|
|
||||||
|
|
||||||
//TODO (na) : Add a way to return actions associated with a timeline and then merge/unify with logic above to avoid Stream.concats
|
|
||||||
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
|
|
||||||
// We cannot have any holes in the commit timeline. We cannot archive any commits which are made after the first savepoint present.
|
|
||||||
Optional<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
|
|
||||||
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
|
|
||||||
// Actually do the commits
|
|
||||||
instants = Stream.concat(instants, commitTimeline.getInstants().filter(s -> {
|
|
||||||
// if no savepoint present, then dont filter
|
|
||||||
return !(firstSavepoint.isPresent() && HoodieTimeline
|
|
||||||
.compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(),
|
|
||||||
HoodieTimeline.LESSER_OR_EQUAL));
|
|
||||||
}).limit(commitTimeline.countInstants() - minCommitsToKeep));
|
|
||||||
}
|
|
||||||
|
|
||||||
return instants;
|
|
||||||
}
|
|
||||||
|
|
||||||
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) {
|
|
||||||
log.info("Deleting instants " + archivedInstants);
|
|
||||||
HoodieTableMetaClient metaClient =
|
|
||||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
|
||||||
|
|
||||||
boolean success = true;
|
|
||||||
for (HoodieInstant archivedInstant : archivedInstants) {
|
|
||||||
Path commitFile =
|
|
||||||
new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
|
|
||||||
try {
|
|
||||||
if (fs.exists(commitFile)) {
|
|
||||||
success &= fs.delete(commitFile, false);
|
|
||||||
log.info("Archived and deleted instant file " + commitFile);
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieIOException("Failed to delete archived instant " + archivedInstant,
|
|
||||||
e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return success;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
|
|
||||||
|
|
||||||
try {
|
|
||||||
HoodieTableMetaClient metaClient =
|
|
||||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
|
||||||
HoodieTimeline commitTimeline =
|
|
||||||
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();
|
|
||||||
|
|
||||||
Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
|
|
||||||
log.info("Wrapper schema " + wrapperSchema.toString());
|
|
||||||
List<IndexedRecord> records = new ArrayList<>();
|
|
||||||
for (HoodieInstant hoodieInstant : instants) {
|
|
||||||
records.add(convertToAvroRecord(commitTimeline, hoodieInstant));
|
|
||||||
}
|
|
||||||
HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, wrapperSchema);
|
|
||||||
this.writer = writer.appendBlock(block);
|
|
||||||
} catch(Exception e) {
|
|
||||||
throw new HoodieCommitException("Failed to archive commits", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public Path getArchiveFilePath() {
|
|
||||||
return archiveFilePath;
|
|
||||||
}
|
|
||||||
|
|
||||||
private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline, HoodieInstant hoodieInstant) throws IOException {
|
|
||||||
HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry();
|
|
||||||
archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp());
|
|
||||||
switch(hoodieInstant.getAction()) {
|
|
||||||
case HoodieTimeline.CLEAN_ACTION:{
|
|
||||||
archivedMetaWrapper.setHoodieCleanMetadata(AvroUtils.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCleanMetadata.class));
|
|
||||||
archivedMetaWrapper.setActionType(ActionType.clean.name());
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case HoodieTimeline.COMMIT_ACTION:{
|
|
||||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
|
||||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
|
|
||||||
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
|
|
||||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case HoodieTimeline.COMPACTION_ACTION:{
|
|
||||||
com.uber.hoodie.common.model.HoodieCompactionMetadata compactionMetadata = com.uber.hoodie.common.model.HoodieCompactionMetadata
|
|
||||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
|
|
||||||
archivedMetaWrapper.setHoodieCompactionMetadata(compactionMetadataConverter(compactionMetadata));
|
|
||||||
archivedMetaWrapper.setActionType(ActionType.compaction.name());
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case HoodieTimeline.ROLLBACK_ACTION:{
|
|
||||||
archivedMetaWrapper.setHoodieRollbackMetadata(AvroUtils.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieRollbackMetadata.class));
|
|
||||||
archivedMetaWrapper.setActionType(ActionType.rollback.name());
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case HoodieTimeline.SAVEPOINT_ACTION:{
|
|
||||||
archivedMetaWrapper.setHoodieSavePointMetadata(AvroUtils.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieSavepointMetadata.class));
|
|
||||||
archivedMetaWrapper.setActionType(ActionType.savepoint.name());
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
case HoodieTimeline.DELTA_COMMIT_ACTION:{
|
|
||||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
|
||||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
|
|
||||||
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
|
|
||||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return archivedMetaWrapper;
|
|
||||||
}
|
|
||||||
|
|
||||||
private com.uber.hoodie.avro.model.HoodieCommitMetadata commitMetadataConverter(HoodieCommitMetadata hoodieCommitMetadata) {
|
|
||||||
ObjectMapper mapper = new ObjectMapper();
|
|
||||||
//Need this to ignore other public get() methods
|
|
||||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
|
||||||
com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData =
|
|
||||||
mapper.convertValue(hoodieCommitMetadata, com.uber.hoodie.avro.model.HoodieCommitMetadata.class);
|
|
||||||
return avroMetaData;
|
|
||||||
}
|
|
||||||
|
|
||||||
private com.uber.hoodie.avro.model.HoodieCompactionMetadata compactionMetadataConverter(HoodieCompactionMetadata hoodieCompactionMetadata) {
|
|
||||||
ObjectMapper mapper = new ObjectMapper();
|
|
||||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
|
||||||
com.uber.hoodie.avro.model.HoodieCompactionMetadata avroMetaData = mapper.convertValue(hoodieCompactionMetadata,
|
|
||||||
com.uber.hoodie.avro.model.HoodieCompactionMetadata.class);
|
|
||||||
return avroMetaData;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -29,116 +29,111 @@ import com.uber.hoodie.exception.HoodieInsertException;
|
|||||||
import com.uber.hoodie.io.storage.HoodieStorageWriter;
|
import com.uber.hoodie.io.storage.HoodieStorageWriter;
|
||||||
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
|
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.UUID;
|
||||||
import org.apache.avro.generic.IndexedRecord;
|
import org.apache.avro.generic.IndexedRecord;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.spark.TaskContext;
|
import org.apache.spark.TaskContext;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.UUID;
|
|
||||||
|
|
||||||
public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
||||||
private static Logger logger = LogManager.getLogger(HoodieCreateHandle.class);
|
|
||||||
|
|
||||||
private final WriteStatus status;
|
private static Logger logger = LogManager.getLogger(HoodieCreateHandle.class);
|
||||||
private final HoodieStorageWriter<IndexedRecord> storageWriter;
|
|
||||||
private final Path path;
|
|
||||||
private long recordsWritten = 0;
|
|
||||||
private long recordsDeleted = 0;
|
|
||||||
|
|
||||||
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime,
|
private final WriteStatus status;
|
||||||
HoodieTable<T> hoodieTable, String partitionPath) {
|
private final HoodieStorageWriter<IndexedRecord> storageWriter;
|
||||||
super(config, commitTime, hoodieTable);
|
private final Path path;
|
||||||
this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
private long recordsWritten = 0;
|
||||||
status.setFileId(UUID.randomUUID().toString());
|
private long recordsDeleted = 0;
|
||||||
status.setPartitionPath(partitionPath);
|
|
||||||
|
|
||||||
this.path = makeNewPath(partitionPath, TaskContext.getPartitionId(), status.getFileId());
|
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime,
|
||||||
try {
|
HoodieTable<T> hoodieTable, String partitionPath) {
|
||||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
|
super(config, commitTime, hoodieTable);
|
||||||
commitTime,
|
this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||||
new Path(config.getBasePath()),
|
status.setFileId(UUID.randomUUID().toString());
|
||||||
new Path(config.getBasePath(), partitionPath));
|
status.setPartitionPath(partitionPath);
|
||||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
|
||||||
this.storageWriter =
|
this.path = makeNewPath(partitionPath, TaskContext.getPartitionId(), status.getFileId());
|
||||||
HoodieStorageWriterFactory.getStorageWriter(commitTime, path, hoodieTable, config, schema);
|
try {
|
||||||
} catch (IOException e) {
|
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
|
||||||
throw new HoodieInsertException(
|
commitTime,
|
||||||
"Failed to initialize HoodieStorageWriter for path " + path, e);
|
new Path(config.getBasePath()),
|
||||||
}
|
new Path(config.getBasePath(), partitionPath));
|
||||||
logger.info("New InsertHandle for partition :" + partitionPath);
|
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||||
|
this.storageWriter =
|
||||||
|
HoodieStorageWriterFactory
|
||||||
|
.getStorageWriter(commitTime, path, hoodieTable, config, schema);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieInsertException(
|
||||||
|
"Failed to initialize HoodieStorageWriter for path " + path, e);
|
||||||
}
|
}
|
||||||
|
logger.info("New InsertHandle for partition :" + partitionPath);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Determines whether we can accept the incoming records, into the current file, depending on
|
* Determines whether we can accept the incoming records, into the current file, depending on
|
||||||
*
|
*
|
||||||
* - Whether it belongs to the same partitionPath as existing records
|
* - Whether it belongs to the same partitionPath as existing records - Whether the current file
|
||||||
* - Whether the current file written bytes lt max file size
|
* written bytes lt max file size
|
||||||
*
|
*/
|
||||||
* @return
|
public boolean canWrite(HoodieRecord record) {
|
||||||
*/
|
return storageWriter.canWrite() && record.getPartitionPath()
|
||||||
public boolean canWrite(HoodieRecord record) {
|
.equals(status.getPartitionPath());
|
||||||
return storageWriter.canWrite() && record.getPartitionPath()
|
}
|
||||||
.equals(status.getPartitionPath());
|
|
||||||
|
/**
|
||||||
|
* Perform the actual writing of the given record into the backing file.
|
||||||
|
*/
|
||||||
|
public void write(HoodieRecord record) {
|
||||||
|
Optional recordMetadata = record.getData().getMetadata();
|
||||||
|
try {
|
||||||
|
Optional<IndexedRecord> avroRecord = record.getData().getInsertValue(schema);
|
||||||
|
|
||||||
|
if (avroRecord.isPresent()) {
|
||||||
|
storageWriter.writeAvroWithMetadata(avroRecord.get(), record);
|
||||||
|
// update the new location of record, so we know where to find it next
|
||||||
|
record.setNewLocation(new HoodieRecordLocation(commitTime, status.getFileId()));
|
||||||
|
recordsWritten++;
|
||||||
|
} else {
|
||||||
|
recordsDeleted++;
|
||||||
|
}
|
||||||
|
record.deflate();
|
||||||
|
status.markSuccess(record, recordMetadata);
|
||||||
|
} catch (Throwable t) {
|
||||||
|
// Not throwing exception from here, since we don't want to fail the entire job
|
||||||
|
// for a single record
|
||||||
|
status.markFailure(record, t, recordMetadata);
|
||||||
|
logger.error("Error writing record " + record, t);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Perform the actual writing of the given record into the backing file.
|
* Performs actions to durably, persist the current changes and returns a WriteStatus object
|
||||||
*
|
*/
|
||||||
* @param record
|
public WriteStatus close() {
|
||||||
*/
|
logger.info(
|
||||||
public void write(HoodieRecord record) {
|
"Closing the file " + status.getFileId() + " as we are done with all the records "
|
||||||
Optional recordMetadata = record.getData().getMetadata();
|
+ recordsWritten);
|
||||||
try {
|
try {
|
||||||
Optional<IndexedRecord> avroRecord = record.getData().getInsertValue(schema);
|
storageWriter.close();
|
||||||
|
|
||||||
if(avroRecord.isPresent()) {
|
HoodieWriteStat stat = new HoodieWriteStat();
|
||||||
storageWriter.writeAvroWithMetadata(avroRecord.get(), record);
|
stat.setNumWrites(recordsWritten);
|
||||||
// update the new location of record, so we know where to find it next
|
stat.setNumDeletes(recordsDeleted);
|
||||||
record.setNewLocation(new HoodieRecordLocation(commitTime, status.getFileId()));
|
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
|
||||||
recordsWritten++;
|
stat.setFileId(status.getFileId());
|
||||||
} else {
|
String relativePath = path.toString().replace(new Path(config.getBasePath()) + "/", "");
|
||||||
recordsDeleted++;
|
stat.setPath(relativePath);
|
||||||
}
|
stat.setTotalWriteBytes(FSUtils.getFileSize(fs, path));
|
||||||
record.deflate();
|
stat.setTotalWriteErrors(status.getFailedRecords().size());
|
||||||
status.markSuccess(record, recordMetadata);
|
status.setStat(stat);
|
||||||
} catch (Throwable t) {
|
|
||||||
// Not throwing exception from here, since we don't want to fail the entire job
|
return status;
|
||||||
// for a single record
|
} catch (IOException e) {
|
||||||
status.markFailure(record, t, recordMetadata);
|
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path,
|
||||||
logger.error("Error writing record " + record, t);
|
e);
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Performs actions to durably, persist the current changes and returns a WriteStatus object
|
|
||||||
*
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
public WriteStatus close() {
|
|
||||||
logger.info(
|
|
||||||
"Closing the file " + status.getFileId() + " as we are done with all the records "
|
|
||||||
+ recordsWritten);
|
|
||||||
try {
|
|
||||||
storageWriter.close();
|
|
||||||
|
|
||||||
HoodieWriteStat stat = new HoodieWriteStat();
|
|
||||||
stat.setNumWrites(recordsWritten);
|
|
||||||
stat.setNumDeletes(recordsDeleted);
|
|
||||||
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
|
|
||||||
stat.setFileId(status.getFileId());
|
|
||||||
String relativePath = path.toString().replace(new Path(config.getBasePath()) + "/", "");
|
|
||||||
stat.setPath(relativePath);
|
|
||||||
stat.setTotalWriteBytes(FSUtils.getFileSize(fs, path));
|
|
||||||
stat.setTotalWriteErrors(status.getFailedRecords().size());
|
|
||||||
status.setStat(stat);
|
|
||||||
|
|
||||||
return status;
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path,
|
|
||||||
e);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -24,6 +24,7 @@ import com.uber.hoodie.common.util.HoodieAvroUtils;
|
|||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.exception.HoodieIOException;
|
import com.uber.hoodie.exception.HoodieIOException;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
import java.io.IOException;
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
@@ -31,68 +32,67 @@ import org.apache.hadoop.fs.Path;
|
|||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||||
private static Logger logger = LogManager.getLogger(HoodieIOHandle.class);
|
|
||||||
protected final String commitTime;
|
|
||||||
protected final HoodieWriteConfig config;
|
|
||||||
protected final FileSystem fs;
|
|
||||||
protected final HoodieTable<T> hoodieTable;
|
|
||||||
protected HoodieTimeline hoodieTimeline;
|
|
||||||
protected TableFileSystemView.ReadOptimizedView fileSystemView;
|
|
||||||
protected final Schema schema;
|
|
||||||
|
|
||||||
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
|
private static Logger logger = LogManager.getLogger(HoodieIOHandle.class);
|
||||||
HoodieTable<T> hoodieTable) {
|
protected final String commitTime;
|
||||||
this.commitTime = commitTime;
|
protected final HoodieWriteConfig config;
|
||||||
this.config = config;
|
protected final FileSystem fs;
|
||||||
this.fs = FSUtils.getFs();
|
protected final HoodieTable<T> hoodieTable;
|
||||||
this.hoodieTable = hoodieTable;
|
protected HoodieTimeline hoodieTimeline;
|
||||||
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
|
protected TableFileSystemView.ReadOptimizedView fileSystemView;
|
||||||
this.fileSystemView = hoodieTable.getROFileSystemView();
|
protected final Schema schema;
|
||||||
this.schema =
|
|
||||||
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
|
||||||
|
HoodieTable<T> hoodieTable) {
|
||||||
|
this.commitTime = commitTime;
|
||||||
|
this.config = config;
|
||||||
|
this.fs = FSUtils.getFs();
|
||||||
|
this.hoodieTable = hoodieTable;
|
||||||
|
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||||
|
this.fileSystemView = hoodieTable.getROFileSystemView();
|
||||||
|
this.schema =
|
||||||
|
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||||
|
}
|
||||||
|
|
||||||
|
public Path makeNewPath(String partitionPath, int taskPartitionId, String fileName) {
|
||||||
|
Path path = new Path(config.getBasePath(), partitionPath);
|
||||||
|
try {
|
||||||
|
fs.mkdirs(path); // create a new partition as needed.
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieIOException("Failed to make dir " + path, e);
|
||||||
}
|
}
|
||||||
|
|
||||||
public Path makeNewPath(String partitionPath, int taskPartitionId, String fileName) {
|
return new Path(path.toString(),
|
||||||
Path path = new Path(config.getBasePath(), partitionPath);
|
FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName));
|
||||||
try {
|
}
|
||||||
fs.mkdirs(path); // create a new partition as needed.
|
|
||||||
} catch (IOException e) {
|
/**
|
||||||
throw new HoodieIOException("Failed to make dir " + path, e);
|
* Deletes any new tmp files written during the current commit, into the partition
|
||||||
|
*/
|
||||||
|
public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config,
|
||||||
|
String commitTime,
|
||||||
|
String partitionPath,
|
||||||
|
int taskPartitionId) {
|
||||||
|
FileSystem fs = FSUtils.getFs();
|
||||||
|
try {
|
||||||
|
FileStatus[] prevFailedFiles = fs.globStatus(new Path(String
|
||||||
|
.format("%s/%s/%s", config.getBasePath(), partitionPath,
|
||||||
|
FSUtils.maskWithoutFileId(commitTime, taskPartitionId))));
|
||||||
|
if (prevFailedFiles != null) {
|
||||||
|
logger.info("Deleting " + prevFailedFiles.length
|
||||||
|
+ " files generated by previous failed attempts.");
|
||||||
|
for (FileStatus status : prevFailedFiles) {
|
||||||
|
fs.delete(status.getPath(), false);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
return new Path(path.toString(),
|
} catch (IOException e) {
|
||||||
FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName));
|
throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime,
|
||||||
|
e);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
public Schema getSchema() {
|
||||||
* Deletes any new tmp files written during the current commit, into the partition
|
return schema;
|
||||||
*/
|
}
|
||||||
public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config,
|
|
||||||
String commitTime,
|
|
||||||
String partitionPath,
|
|
||||||
int taskPartitionId) {
|
|
||||||
FileSystem fs = FSUtils.getFs();
|
|
||||||
try {
|
|
||||||
FileStatus[] prevFailedFiles = fs.globStatus(new Path(String
|
|
||||||
.format("%s/%s/%s", config.getBasePath(), partitionPath,
|
|
||||||
FSUtils.maskWithoutFileId(commitTime, taskPartitionId))));
|
|
||||||
if (prevFailedFiles != null) {
|
|
||||||
logger.info("Deleting " + prevFailedFiles.length
|
|
||||||
+ " files generated by previous failed attempts.");
|
|
||||||
for (FileStatus status : prevFailedFiles) {
|
|
||||||
fs.delete(status.getPath(), false);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime,
|
|
||||||
e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public Schema getSchema() {
|
|
||||||
return schema;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,19 +16,23 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.io;
|
package com.uber.hoodie.io;
|
||||||
|
|
||||||
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
|
||||||
import com.uber.hoodie.common.util.ReflectionUtils;
|
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
|
||||||
import com.uber.hoodie.WriteStatus;
|
import com.uber.hoodie.WriteStatus;
|
||||||
|
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
|
import com.uber.hoodie.common.util.ReflectionUtils;
|
||||||
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||||
import com.uber.hoodie.io.storage.HoodieStorageWriter;
|
import com.uber.hoodie.io.storage.HoodieStorageWriter;
|
||||||
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
|
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.Optional;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
import org.apache.avro.generic.IndexedRecord;
|
import org.apache.avro.generic.IndexedRecord;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
@@ -36,197 +40,197 @@ import org.apache.log4j.LogManager;
|
|||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
import org.apache.spark.TaskContext;
|
import org.apache.spark.TaskContext;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.Optional;
|
|
||||||
|
|
||||||
@SuppressWarnings("Duplicates")
|
@SuppressWarnings("Duplicates")
|
||||||
public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
||||||
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
|
|
||||||
|
|
||||||
private WriteStatus writeStatus;
|
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
|
||||||
private HashMap<String, HoodieRecord<T>> keyToNewRecords;
|
|
||||||
private HoodieStorageWriter<IndexedRecord> storageWriter;
|
|
||||||
private Path newFilePath;
|
|
||||||
private Path oldFilePath;
|
|
||||||
private long recordsWritten = 0;
|
|
||||||
private long recordsDeleted = 0;
|
|
||||||
private long updatedRecordsWritten = 0;
|
|
||||||
|
|
||||||
public HoodieMergeHandle(HoodieWriteConfig config,
|
private WriteStatus writeStatus;
|
||||||
String commitTime,
|
private HashMap<String, HoodieRecord<T>> keyToNewRecords;
|
||||||
HoodieTable<T> hoodieTable,
|
private HoodieStorageWriter<IndexedRecord> storageWriter;
|
||||||
Iterator<HoodieRecord<T>> recordItr,
|
private Path newFilePath;
|
||||||
String fileId) {
|
private Path oldFilePath;
|
||||||
super(config, commitTime, hoodieTable);
|
private long recordsWritten = 0;
|
||||||
init(fileId, recordItr);
|
private long recordsDeleted = 0;
|
||||||
}
|
private long updatedRecordsWritten = 0;
|
||||||
|
|
||||||
/**
|
public HoodieMergeHandle(HoodieWriteConfig config,
|
||||||
* Load the new incoming records in a map, and extract the old file path.
|
String commitTime,
|
||||||
*/
|
HoodieTable<T> hoodieTable,
|
||||||
private void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
Iterator<HoodieRecord<T>> recordItr,
|
||||||
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
String fileId) {
|
||||||
writeStatus.setStat(new HoodieWriteStat());
|
super(config, commitTime, hoodieTable);
|
||||||
this.writeStatus = writeStatus;
|
init(fileId, recordItr);
|
||||||
this.keyToNewRecords = new HashMap<>();
|
}
|
||||||
|
|
||||||
try {
|
/**
|
||||||
// Load the new records in a map
|
* Load the new incoming records in a map, and extract the old file path.
|
||||||
while (newRecordsItr.hasNext()) {
|
*/
|
||||||
HoodieRecord<T> record = newRecordsItr.next();
|
private void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||||
// If the first record, we need to extract some info out
|
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||||
if (oldFilePath == null) {
|
writeStatus.setStat(new HoodieWriteStat());
|
||||||
String latestValidFilePath = fileSystemView
|
this.writeStatus = writeStatus;
|
||||||
.getLatestDataFiles(record.getPartitionPath())
|
this.keyToNewRecords = new HashMap<>();
|
||||||
.filter(dataFile -> dataFile.getFileId().equals(fileId))
|
|
||||||
.findFirst()
|
|
||||||
.get().getFileName();
|
|
||||||
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
|
|
||||||
|
|
||||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
|
try {
|
||||||
commitTime,
|
// Load the new records in a map
|
||||||
new Path(config.getBasePath()),
|
while (newRecordsItr.hasNext()) {
|
||||||
new Path(config.getBasePath(), record.getPartitionPath()));
|
HoodieRecord<T> record = newRecordsItr.next();
|
||||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
// If the first record, we need to extract some info out
|
||||||
|
if (oldFilePath == null) {
|
||||||
|
String latestValidFilePath = fileSystemView
|
||||||
|
.getLatestDataFiles(record.getPartitionPath())
|
||||||
|
.filter(dataFile -> dataFile.getFileId().equals(fileId))
|
||||||
|
.findFirst()
|
||||||
|
.get().getFileName();
|
||||||
|
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
|
||||||
|
|
||||||
oldFilePath = new Path(
|
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
|
||||||
config.getBasePath() + "/" + record.getPartitionPath() + "/"
|
commitTime,
|
||||||
+ latestValidFilePath);
|
new Path(config.getBasePath()),
|
||||||
String relativePath = new Path( record.getPartitionPath() + "/" + FSUtils
|
new Path(config.getBasePath(), record.getPartitionPath()));
|
||||||
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString();
|
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||||
newFilePath = new Path(config.getBasePath(), relativePath);
|
|
||||||
|
|
||||||
// handle cases of partial failures, for update task
|
oldFilePath = new Path(
|
||||||
if (fs.exists(newFilePath)) {
|
config.getBasePath() + "/" + record.getPartitionPath() + "/"
|
||||||
fs.delete(newFilePath, false);
|
+ latestValidFilePath);
|
||||||
}
|
String relativePath = new Path(record.getPartitionPath() + "/" + FSUtils
|
||||||
|
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString();
|
||||||
|
newFilePath = new Path(config.getBasePath(), relativePath);
|
||||||
|
|
||||||
logger.info(String.format("Merging new data into oldPath %s, as newPath %s",
|
// handle cases of partial failures, for update task
|
||||||
oldFilePath.toString(), newFilePath.toString()));
|
if (fs.exists(newFilePath)) {
|
||||||
// file name is same for all records, in this bunch
|
fs.delete(newFilePath, false);
|
||||||
writeStatus.setFileId(fileId);
|
}
|
||||||
writeStatus.setPartitionPath(record.getPartitionPath());
|
|
||||||
writeStatus.getStat().setFileId(fileId);
|
|
||||||
writeStatus.getStat().setPath(relativePath);
|
|
||||||
}
|
|
||||||
keyToNewRecords.put(record.getRecordKey(), record);
|
|
||||||
// update the new location of the record, so we know where to find it next
|
|
||||||
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
|
||||||
}
|
|
||||||
// Create the writer for writing the new version file
|
|
||||||
storageWriter = HoodieStorageWriterFactory
|
|
||||||
.getStorageWriter(commitTime, newFilePath, hoodieTable, config, schema);
|
|
||||||
|
|
||||||
} catch (Exception e) {
|
logger.info(String.format("Merging new data into oldPath %s, as newPath %s",
|
||||||
logger.error("Error in update task at commit " + commitTime, e);
|
oldFilePath.toString(), newFilePath.toString()));
|
||||||
writeStatus.setGlobalError(e);
|
// file name is same for all records, in this bunch
|
||||||
throw new HoodieUpsertException(
|
writeStatus.setFileId(fileId);
|
||||||
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
|
writeStatus.setPartitionPath(record.getPartitionPath());
|
||||||
+ commitTime + " on path " + hoodieTable.getMetaClient().getBasePath(), e);
|
writeStatus.getStat().setFileId(fileId);
|
||||||
|
writeStatus.getStat().setPath(relativePath);
|
||||||
}
|
}
|
||||||
|
keyToNewRecords.put(record.getRecordKey(), record);
|
||||||
|
// update the new location of the record, so we know where to find it next
|
||||||
|
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
||||||
|
}
|
||||||
|
// Create the writer for writing the new version file
|
||||||
|
storageWriter = HoodieStorageWriterFactory
|
||||||
|
.getStorageWriter(commitTime, newFilePath, hoodieTable, 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 path " + hoodieTable.getMetaClient().getBasePath(), e);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, Optional<IndexedRecord> indexedRecord) {
|
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord,
|
||||||
Optional recordMetadata = hoodieRecord.getData().getMetadata();
|
Optional<IndexedRecord> indexedRecord) {
|
||||||
try {
|
Optional recordMetadata = hoodieRecord.getData().getMetadata();
|
||||||
if(indexedRecord.isPresent()) {
|
try {
|
||||||
storageWriter.writeAvroWithMetadata(indexedRecord.get(), hoodieRecord);
|
if (indexedRecord.isPresent()) {
|
||||||
recordsWritten++;
|
storageWriter.writeAvroWithMetadata(indexedRecord.get(), hoodieRecord);
|
||||||
updatedRecordsWritten++;
|
recordsWritten++;
|
||||||
} else {
|
updatedRecordsWritten++;
|
||||||
recordsDeleted++;
|
} else {
|
||||||
}
|
recordsDeleted++;
|
||||||
|
}
|
||||||
|
|
||||||
hoodieRecord.deflate();
|
hoodieRecord.deflate();
|
||||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||||
return true;
|
return true;
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
logger.error("Error writing record "+ hoodieRecord, e);
|
logger.error("Error writing record " + hoodieRecord, e);
|
||||||
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
|
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
|
* Go through an old record. Here if we detect a newer version shows up, we write the new one to
|
||||||
*/
|
* the file.
|
||||||
public void write(GenericRecord oldRecord) {
|
*/
|
||||||
String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
public void write(GenericRecord oldRecord) {
|
||||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||||
boolean copyOldRecord = true;
|
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||||
if (keyToNewRecords.containsKey(key)) {
|
boolean copyOldRecord = true;
|
||||||
try {
|
if (keyToNewRecords.containsKey(key)) {
|
||||||
Optional<IndexedRecord> combinedAvroRecord = hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, schema);
|
try {
|
||||||
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
|
Optional<IndexedRecord> combinedAvroRecord = hoodieRecord.getData()
|
||||||
|
.combineAndGetUpdateValue(oldRecord, schema);
|
||||||
|
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
|
||||||
/* ONLY WHEN
|
/* ONLY WHEN
|
||||||
* 1) we have an update for this key AND
|
* 1) we have an update for this key AND
|
||||||
* 2) We are able to successfully write the the combined new value
|
* 2) We are able to successfully write the the combined new value
|
||||||
*
|
*
|
||||||
* We no longer need to copy the old record over.
|
* We no longer need to copy the old record over.
|
||||||
*/
|
*/
|
||||||
copyOldRecord = false;
|
copyOldRecord = false;
|
||||||
}
|
|
||||||
keyToNewRecords.remove(key);
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new HoodieUpsertException("Failed to combine/merge new record with old value in storage, for new record {"
|
|
||||||
+ keyToNewRecords.get(key) + "}, old value {" + oldRecord + "}", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (copyOldRecord) {
|
|
||||||
// this should work as it is, since this is an existing record
|
|
||||||
String errMsg = "Failed to merge old record into new file for key " + key + " from old file "
|
|
||||||
+ getOldFilePath() + " to new file " + newFilePath;
|
|
||||||
try {
|
|
||||||
storageWriter.writeAvro(key, oldRecord);
|
|
||||||
} catch (ClassCastException e) {
|
|
||||||
logger.error(
|
|
||||||
"Schema mismatch when rewriting old record " + oldRecord + " from file "
|
|
||||||
+ getOldFilePath() + " to file " + newFilePath + " with schema " + schema
|
|
||||||
.toString(true));
|
|
||||||
throw new HoodieUpsertException(errMsg, e);
|
|
||||||
} catch (IOException e) {
|
|
||||||
logger.error("Failed to merge old record into new file for key " + key + " from old file "
|
|
||||||
+ getOldFilePath() + " to new file " + newFilePath, e);
|
|
||||||
throw new HoodieUpsertException(errMsg, e);
|
|
||||||
}
|
|
||||||
recordsWritten ++;
|
|
||||||
}
|
}
|
||||||
|
keyToNewRecords.remove(key);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new HoodieUpsertException(
|
||||||
|
"Failed to combine/merge new record with old value in storage, for new record {"
|
||||||
|
+ keyToNewRecords.get(key) + "}, old value {" + oldRecord + "}", e);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void close() {
|
if (copyOldRecord) {
|
||||||
try {
|
// this should work as it is, since this is an existing record
|
||||||
// write out any pending records (this can happen when inserts are turned into updates)
|
String errMsg = "Failed to merge old record into new file for key " + key + " from old file "
|
||||||
Iterator<String> pendingRecordsItr = keyToNewRecords.keySet().iterator();
|
+ getOldFilePath() + " to new file " + newFilePath;
|
||||||
while (pendingRecordsItr.hasNext()) {
|
try {
|
||||||
String key = pendingRecordsItr.next();
|
storageWriter.writeAvro(key, oldRecord);
|
||||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
} catch (ClassCastException e) {
|
||||||
writeUpdateRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema));
|
logger.error(
|
||||||
}
|
"Schema mismatch when rewriting old record " + oldRecord + " from file "
|
||||||
keyToNewRecords.clear();
|
+ getOldFilePath() + " to file " + newFilePath + " with schema " + schema
|
||||||
|
.toString(true));
|
||||||
if (storageWriter != null) {
|
throw new HoodieUpsertException(errMsg, e);
|
||||||
storageWriter.close();
|
} catch (IOException e) {
|
||||||
}
|
logger.error("Failed to merge old record into new file for key " + key + " from old file "
|
||||||
|
+ getOldFilePath() + " to new file " + newFilePath, e);
|
||||||
writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, newFilePath));
|
throw new HoodieUpsertException(errMsg, e);
|
||||||
writeStatus.getStat().setNumWrites(recordsWritten);
|
}
|
||||||
writeStatus.getStat().setNumDeletes(recordsDeleted);
|
recordsWritten++;
|
||||||
writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten);
|
|
||||||
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public Path getOldFilePath() {
|
public void close() {
|
||||||
return oldFilePath;
|
try {
|
||||||
}
|
// write out any pending records (this can happen when inserts are turned into updates)
|
||||||
|
Iterator<String> pendingRecordsItr = keyToNewRecords.keySet().iterator();
|
||||||
|
while (pendingRecordsItr.hasNext()) {
|
||||||
|
String key = pendingRecordsItr.next();
|
||||||
|
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||||
|
writeUpdateRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema));
|
||||||
|
}
|
||||||
|
keyToNewRecords.clear();
|
||||||
|
|
||||||
public WriteStatus getWriteStatus() {
|
if (storageWriter != null) {
|
||||||
return writeStatus;
|
storageWriter.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, newFilePath));
|
||||||
|
writeStatus.getStat().setNumWrites(recordsWritten);
|
||||||
|
writeStatus.getStat().setNumDeletes(recordsDeleted);
|
||||||
|
writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten);
|
||||||
|
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public Path getOldFilePath() {
|
||||||
|
return oldFilePath;
|
||||||
|
}
|
||||||
|
|
||||||
|
public WriteStatus getWriteStatus() {
|
||||||
|
return writeStatus;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,7 +18,6 @@ package com.uber.hoodie.io.compact;
|
|||||||
|
|
||||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||||
|
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
|
import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
@@ -27,8 +26,8 @@ import java.util.Map;
|
|||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Encapsulates all the needed information about a compaction
|
* Encapsulates all the needed information about a compaction and make a decision whether this
|
||||||
* and make a decision whether this compaction is effective or not
|
* compaction is effective or not
|
||||||
*
|
*
|
||||||
* @see CompactionStrategy
|
* @see CompactionStrategy
|
||||||
*/
|
*/
|
||||||
|
|||||||
@@ -22,29 +22,28 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
|||||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A HoodieCompactor runs compaction on a hoodie table
|
* A HoodieCompactor runs compaction on a hoodie table
|
||||||
*/
|
*/
|
||||||
public interface HoodieCompactor extends Serializable {
|
public interface HoodieCompactor extends Serializable {
|
||||||
/**
|
|
||||||
* Compact the delta files with the data files
|
/**
|
||||||
* @throws Exception
|
* Compact the delta files with the data files
|
||||||
*/
|
*/
|
||||||
HoodieCompactionMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config,
|
HoodieCompactionMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config,
|
||||||
HoodieTable hoodieTable) throws Exception;
|
HoodieTable hoodieTable) throws Exception;
|
||||||
|
|
||||||
|
|
||||||
// Helper methods
|
// Helper methods
|
||||||
default String startCompactionCommit(HoodieTable hoodieTable) {
|
default String startCompactionCommit(HoodieTable hoodieTable) {
|
||||||
String commitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
|
String commitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
|
||||||
HoodieActiveTimeline activeTimeline = hoodieTable.getActiveTimeline();
|
HoodieActiveTimeline activeTimeline = hoodieTable.getActiveTimeline();
|
||||||
activeTimeline
|
activeTimeline
|
||||||
.createInflight(new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, commitTime));
|
.createInflight(new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, commitTime));
|
||||||
return commitTime;
|
return commitTime;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,14 +16,14 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.io.compact;
|
package com.uber.hoodie.io.compact;
|
||||||
|
|
||||||
|
import static java.util.stream.Collectors.toList;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.uber.hoodie.WriteStatus;
|
import com.uber.hoodie.WriteStatus;
|
||||||
import com.uber.hoodie.common.model.CompactionWriteStat;
|
import com.uber.hoodie.common.model.CompactionWriteStat;
|
||||||
import com.uber.hoodie.common.model.HoodieAvroPayload;
|
|
||||||
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
|
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
|
||||||
import com.uber.hoodie.common.model.HoodieTableType;
|
import com.uber.hoodie.common.model.HoodieTableType;
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||||
@@ -36,7 +36,12 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
|||||||
import com.uber.hoodie.exception.HoodieCompactionException;
|
import com.uber.hoodie.exception.HoodieCompactionException;
|
||||||
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
|
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.StreamSupport;
|
import java.util.stream.StreamSupport;
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
@@ -46,18 +51,10 @@ import org.apache.log4j.Logger;
|
|||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.nio.charset.StandardCharsets;
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Optional;
|
|
||||||
|
|
||||||
import static java.util.stream.Collectors.*;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage.
|
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all
|
||||||
* Computes all possible compactions, passes it through a CompactionFilter and executes
|
* possible compactions, passes it through a CompactionFilter and executes all the compactions and
|
||||||
* all the compactions and writes a new version of base files and make a normal commit
|
* writes a new version of base files and make a normal commit
|
||||||
*
|
*
|
||||||
* @see HoodieCompactor
|
* @see HoodieCompactor
|
||||||
*/
|
*/
|
||||||
@@ -80,7 +77,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
|||||||
String compactionCommit = startCompactionCommit(hoodieTable);
|
String compactionCommit = startCompactionCommit(hoodieTable);
|
||||||
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommit);
|
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommit);
|
||||||
List<String> partitionPaths =
|
List<String> partitionPaths =
|
||||||
FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning());
|
FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
|
||||||
|
config.shouldAssumeDatePartitioning());
|
||||||
|
|
||||||
log.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
|
log.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
|
||||||
List<CompactionOperation> operations =
|
List<CompactionOperation> operations =
|
||||||
@@ -89,7 +87,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
|||||||
.getRTFileSystemView()
|
.getRTFileSystemView()
|
||||||
.getLatestFileSlices(partitionPath)
|
.getLatestFileSlices(partitionPath)
|
||||||
.map(s -> new CompactionOperation(s.getDataFile().get(),
|
.map(s -> new CompactionOperation(s.getDataFile().get(),
|
||||||
partitionPath, s.getLogFiles().collect(Collectors.toList()), config))
|
partitionPath, s.getLogFiles().collect(Collectors.toList()), config))
|
||||||
.collect(toList()).iterator()).collect();
|
.collect(toList()).iterator()).collect();
|
||||||
log.info("Total of " + operations.size() + " compactions are retrieved");
|
log.info("Total of " + operations.size() + " compactions are retrieved");
|
||||||
|
|
||||||
@@ -150,14 +148,15 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
|||||||
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
|
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
|
||||||
|
|
||||||
String maxInstantTime = metaClient.getActiveTimeline()
|
String maxInstantTime = metaClient.getActiveTimeline()
|
||||||
.getTimelineOfActions(
|
.getTimelineOfActions(
|
||||||
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
|
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
|
||||||
HoodieTimeline.COMPACTION_ACTION,
|
HoodieTimeline.COMPACTION_ACTION,
|
||||||
HoodieTimeline.DELTA_COMMIT_ACTION))
|
HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||||
|
|
||||||
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs, metaClient.getBasePath(),
|
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs,
|
||||||
operation.getDeltaFilePaths(), readerSchema, maxInstantTime);
|
metaClient.getBasePath(),
|
||||||
|
operation.getDeltaFilePaths(), readerSchema, maxInstantTime);
|
||||||
if (!scanner.iterator().hasNext()) {
|
if (!scanner.iterator().hasNext()) {
|
||||||
return Lists.newArrayList();
|
return Lists.newArrayList();
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -28,8 +28,8 @@ import java.util.Map;
|
|||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* CompactionStrategy which looks at total IO to be done for the compaction (read + write)
|
* CompactionStrategy which looks at total IO to be done for the compaction (read + write) and
|
||||||
* and limits the list of compactions to be under a configured limit on the IO
|
* limits the list of compactions to be under a configured limit on the IO
|
||||||
*
|
*
|
||||||
* @see CompactionStrategy
|
* @see CompactionStrategy
|
||||||
*/
|
*/
|
||||||
@@ -46,7 +46,7 @@ public class BoundedIOCompactionStrategy implements CompactionStrategy {
|
|||||||
// Total size of all the log files
|
// Total size of all the log files
|
||||||
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(
|
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(
|
||||||
Optional::isPresent).map(Optional::get).reduce(
|
Optional::isPresent).map(Optional::get).reduce(
|
||||||
(size1, size2) -> size1 + size2).orElse(0L);
|
(size1, size2) -> size1 + size2).orElse(0L);
|
||||||
// Total read will be the base file + all the log files
|
// Total read will be the base file + all the log files
|
||||||
Long totalIORead = FSUtils.getSizeInMB(dataFile.getFileSize() + totalLogFileSize);
|
Long totalIORead = FSUtils.getSizeInMB(dataFile.getFileSize() + totalLogFileSize);
|
||||||
// Total write will be similar to the size of the base file
|
// Total write will be similar to the size of the base file
|
||||||
@@ -62,7 +62,8 @@ public class BoundedIOCompactionStrategy implements CompactionStrategy {
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig, List<CompactionOperation> operations) {
|
public List<CompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||||
|
List<CompactionOperation> operations) {
|
||||||
// Iterate through the operations in order and accept operations as long as we are within the IO limit
|
// Iterate through the operations in order and accept operations as long as we are within the IO limit
|
||||||
// Preserves the original ordering of compactions
|
// Preserves the original ordering of compactions
|
||||||
List<CompactionOperation> finalOperations = Lists.newArrayList();
|
List<CompactionOperation> finalOperations = Lists.newArrayList();
|
||||||
|
|||||||
@@ -25,12 +25,12 @@ import java.util.List;
|
|||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Strategy for compaction. Pluggable implementation of define how compaction should be done.
|
* Strategy for compaction. Pluggable implementation of define how compaction should be done. The
|
||||||
* The implementations of this interface can capture the relevant metrics to order and filter
|
* implementations of this interface can capture the relevant metrics to order and filter the final
|
||||||
* the final list of compaction operation to run in a single compaction.
|
* list of compaction operation to run in a single compaction.
|
||||||
*
|
*
|
||||||
* Implementation of CompactionStrategy cannot hold any state.
|
* Implementation of CompactionStrategy cannot hold any state. Difference instantiations can be
|
||||||
* Difference instantiations can be passed in every time
|
* passed in every time
|
||||||
*
|
*
|
||||||
* @see com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor
|
* @see com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor
|
||||||
* @see CompactionOperation
|
* @see CompactionOperation
|
||||||
@@ -38,8 +38,8 @@ import java.util.Map;
|
|||||||
public interface CompactionStrategy extends Serializable {
|
public interface CompactionStrategy extends Serializable {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Callback hook when a CompactionOperation is created. Individual strategies can
|
* Callback hook when a CompactionOperation is created. Individual strategies can capture the
|
||||||
* capture the metrics they need to decide on the priority.
|
* metrics they need to decide on the priority.
|
||||||
*
|
*
|
||||||
* @param dataFile - Base file to compact
|
* @param dataFile - Base file to compact
|
||||||
* @param partitionPath - Partition path
|
* @param partitionPath - Partition path
|
||||||
@@ -50,8 +50,8 @@ public interface CompactionStrategy extends Serializable {
|
|||||||
List<HoodieLogFile> logFiles);
|
List<HoodieLogFile> logFiles);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Order and Filter the list of compactions. Use the metrics captured with the
|
* Order and Filter the list of compactions. Use the metrics captured with the captureMetrics to
|
||||||
* captureMetrics to order and filter out compactions
|
* order and filter out compactions
|
||||||
*
|
*
|
||||||
* @param writeConfig - HoodieWriteConfig - config for this compaction is passed in
|
* @param writeConfig - HoodieWriteConfig - config for this compaction is passed in
|
||||||
* @param operations - list of compactions collected
|
* @param operations - list of compactions collected
|
||||||
|
|||||||
@@ -27,8 +27,8 @@ import java.util.Optional;
|
|||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* LogFileSizeBasedCompactionStrategy orders the compactions based on the total log files size
|
* LogFileSizeBasedCompactionStrategy orders the compactions based on the total log files size and
|
||||||
* and limits the compactions within a configured IO bound
|
* limits the compactions within a configured IO bound
|
||||||
*
|
*
|
||||||
* @see BoundedIOCompactionStrategy
|
* @see BoundedIOCompactionStrategy
|
||||||
* @see CompactionStrategy
|
* @see CompactionStrategy
|
||||||
|
|||||||
@@ -25,9 +25,9 @@ import java.util.List;
|
|||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* UnBoundedCompactionStrategy will not change ordering or filter any compaction.
|
* UnBoundedCompactionStrategy will not change ordering or filter any compaction. It is a
|
||||||
* It is a pass-through and will compact all the base files which has a log file.
|
* pass-through and will compact all the base files which has a log file. This usually means
|
||||||
* This usually means no-intelligence on compaction.
|
* no-intelligence on compaction.
|
||||||
*
|
*
|
||||||
* @see CompactionStrategy
|
* @see CompactionStrategy
|
||||||
*/
|
*/
|
||||||
|
|||||||
@@ -17,50 +17,50 @@
|
|||||||
package com.uber.hoodie.io.storage;
|
package com.uber.hoodie.io.storage;
|
||||||
|
|
||||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||||
import org.apache.avro.Schema;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||||
|
|
||||||
public class HoodieParquetConfig {
|
public class HoodieParquetConfig {
|
||||||
private HoodieAvroWriteSupport writeSupport;
|
|
||||||
private CompressionCodecName compressionCodecName;
|
|
||||||
private int blockSize;
|
|
||||||
private int pageSize;
|
|
||||||
private int maxFileSize;
|
|
||||||
private Configuration hadoopConf;
|
|
||||||
|
|
||||||
public HoodieParquetConfig(HoodieAvroWriteSupport writeSupport,
|
private HoodieAvroWriteSupport writeSupport;
|
||||||
CompressionCodecName compressionCodecName, int blockSize, int pageSize, int maxFileSize,
|
private CompressionCodecName compressionCodecName;
|
||||||
Configuration hadoopConf) {
|
private int blockSize;
|
||||||
this.writeSupport = writeSupport;
|
private int pageSize;
|
||||||
this.compressionCodecName = compressionCodecName;
|
private int maxFileSize;
|
||||||
this.blockSize = blockSize;
|
private Configuration hadoopConf;
|
||||||
this.pageSize = pageSize;
|
|
||||||
this.maxFileSize = maxFileSize;
|
|
||||||
this.hadoopConf = hadoopConf;
|
|
||||||
}
|
|
||||||
|
|
||||||
public HoodieAvroWriteSupport getWriteSupport() {
|
public HoodieParquetConfig(HoodieAvroWriteSupport writeSupport,
|
||||||
return writeSupport;
|
CompressionCodecName compressionCodecName, int blockSize, int pageSize, int maxFileSize,
|
||||||
}
|
Configuration hadoopConf) {
|
||||||
|
this.writeSupport = writeSupport;
|
||||||
|
this.compressionCodecName = compressionCodecName;
|
||||||
|
this.blockSize = blockSize;
|
||||||
|
this.pageSize = pageSize;
|
||||||
|
this.maxFileSize = maxFileSize;
|
||||||
|
this.hadoopConf = hadoopConf;
|
||||||
|
}
|
||||||
|
|
||||||
public CompressionCodecName getCompressionCodecName() {
|
public HoodieAvroWriteSupport getWriteSupport() {
|
||||||
return compressionCodecName;
|
return writeSupport;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getBlockSize() {
|
public CompressionCodecName getCompressionCodecName() {
|
||||||
return blockSize;
|
return compressionCodecName;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getPageSize() {
|
public int getBlockSize() {
|
||||||
return pageSize;
|
return blockSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int getMaxFileSize() {
|
public int getPageSize() {
|
||||||
return maxFileSize;
|
return pageSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Configuration getHadoopConf() {
|
public int getMaxFileSize() {
|
||||||
return hadoopConf;
|
return maxFileSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Configuration getHadoopConf() {
|
||||||
|
return hadoopConf;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -20,6 +20,8 @@ import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
|||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
import org.apache.avro.generic.IndexedRecord;
|
import org.apache.avro.generic.IndexedRecord;
|
||||||
@@ -30,79 +32,76 @@ import org.apache.parquet.hadoop.ParquetFileWriter;
|
|||||||
import org.apache.parquet.hadoop.ParquetWriter;
|
import org.apache.parquet.hadoop.ParquetWriter;
|
||||||
import org.apache.spark.TaskContext;
|
import org.apache.spark.TaskContext;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file.
|
* HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file. Provides
|
||||||
* Provides a way to check if the current file can take more records with the <code>canWrite()</code>
|
* a way to check if the current file can take more records with the <code>canWrite()</code>
|
||||||
*
|
|
||||||
* @param <T>
|
|
||||||
*/
|
*/
|
||||||
public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends IndexedRecord>
|
public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends IndexedRecord>
|
||||||
extends ParquetWriter<IndexedRecord> implements HoodieStorageWriter<R> {
|
extends ParquetWriter<IndexedRecord> implements HoodieStorageWriter<R> {
|
||||||
private static double STREAM_COMPRESSION_RATIO = 0.1;
|
|
||||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
private static double STREAM_COMPRESSION_RATIO = 0.1;
|
||||||
|
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||||
|
|
||||||
|
|
||||||
private final Path file;
|
private final Path file;
|
||||||
private final HoodieWrapperFileSystem fs;
|
private final HoodieWrapperFileSystem fs;
|
||||||
private final long maxFileSize;
|
private final long maxFileSize;
|
||||||
private final HoodieAvroWriteSupport writeSupport;
|
private final HoodieAvroWriteSupport writeSupport;
|
||||||
private final String commitTime;
|
private final String commitTime;
|
||||||
private final Schema schema;
|
private final Schema schema;
|
||||||
|
|
||||||
|
|
||||||
private static Configuration registerFileSystem(Configuration conf) {
|
private static Configuration registerFileSystem(Configuration conf) {
|
||||||
Configuration returnConf = new Configuration(conf);
|
Configuration returnConf = new Configuration(conf);
|
||||||
String scheme = FileSystem.getDefaultUri(conf).getScheme();
|
String scheme = FileSystem.getDefaultUri(conf).getScheme();
|
||||||
returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl",
|
returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl",
|
||||||
HoodieWrapperFileSystem.class.getName());
|
HoodieWrapperFileSystem.class.getName());
|
||||||
return returnConf;
|
return returnConf;
|
||||||
}
|
}
|
||||||
|
|
||||||
public HoodieParquetWriter(String commitTime, Path file,
|
public HoodieParquetWriter(String commitTime, Path file,
|
||||||
HoodieParquetConfig parquetConfig, Schema schema) throws IOException {
|
HoodieParquetConfig parquetConfig, Schema schema) throws IOException {
|
||||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||||
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(),
|
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(),
|
||||||
parquetConfig.getCompressionCodecName(), parquetConfig.getBlockSize(),
|
parquetConfig.getCompressionCodecName(), parquetConfig.getBlockSize(),
|
||||||
parquetConfig.getPageSize(), parquetConfig.getPageSize(),
|
parquetConfig.getPageSize(), parquetConfig.getPageSize(),
|
||||||
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED,
|
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED,
|
||||||
ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION,
|
ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION,
|
||||||
registerFileSystem(parquetConfig.getHadoopConf()));
|
registerFileSystem(parquetConfig.getHadoopConf()));
|
||||||
this.file =
|
this.file =
|
||||||
HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
||||||
this.fs = (HoodieWrapperFileSystem) this.file
|
this.fs = (HoodieWrapperFileSystem) this.file
|
||||||
.getFileSystem(registerFileSystem(parquetConfig.getHadoopConf()));
|
.getFileSystem(registerFileSystem(parquetConfig.getHadoopConf()));
|
||||||
// We cannot accurately measure the snappy compressed output file size. We are choosing a conservative 10%
|
// We cannot accurately measure the snappy compressed output file size. We are choosing a conservative 10%
|
||||||
// TODO - compute this compression ratio dynamically by looking at the bytes written to the stream and the actual file size reported by HDFS
|
// TODO - compute this compression ratio dynamically by looking at the bytes written to the stream and the actual file size reported by HDFS
|
||||||
this.maxFileSize = parquetConfig.getMaxFileSize() + Math
|
this.maxFileSize = parquetConfig.getMaxFileSize() + Math
|
||||||
.round(parquetConfig.getMaxFileSize() * STREAM_COMPRESSION_RATIO);
|
.round(parquetConfig.getMaxFileSize() * STREAM_COMPRESSION_RATIO);
|
||||||
this.writeSupport = parquetConfig.getWriteSupport();
|
this.writeSupport = parquetConfig.getWriteSupport();
|
||||||
this.commitTime = commitTime;
|
this.commitTime = commitTime;
|
||||||
this.schema = schema;
|
this.schema = schema;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
|
public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
|
||||||
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
||||||
recordIndex.getAndIncrement());
|
recordIndex.getAndIncrement());
|
||||||
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord,
|
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord,
|
||||||
record.getRecordKey(),
|
record.getRecordKey(),
|
||||||
record.getPartitionPath(),
|
record.getPartitionPath(),
|
||||||
file.getName());
|
file.getName());
|
||||||
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, commitTime, seqId);
|
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, commitTime, seqId);
|
||||||
super.write(avroRecord);
|
super.write(avroRecord);
|
||||||
writeSupport.add(record.getRecordKey());
|
writeSupport.add(record.getRecordKey());
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean canWrite() {
|
public boolean canWrite() {
|
||||||
return fs.getBytesWritten(file) < maxFileSize;
|
return fs.getBytesWritten(file) < maxFileSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override public void writeAvro(String key, IndexedRecord object) throws IOException {
|
@Override
|
||||||
super.write(object);
|
public void writeAvro(String key, IndexedRecord object) throws IOException {
|
||||||
writeSupport.add(key);
|
super.write(object);
|
||||||
}
|
writeSupport.add(key);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,13 +17,16 @@
|
|||||||
package com.uber.hoodie.io.storage;
|
package com.uber.hoodie.io.storage;
|
||||||
|
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
|
import java.io.IOException;
|
||||||
import org.apache.avro.generic.IndexedRecord;
|
import org.apache.avro.generic.IndexedRecord;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
public interface HoodieStorageWriter<R extends IndexedRecord> {
|
public interface HoodieStorageWriter<R extends IndexedRecord> {
|
||||||
void writeAvroWithMetadata(R newRecord, HoodieRecord record) throws IOException;
|
|
||||||
boolean canWrite();
|
void writeAvroWithMetadata(R newRecord, HoodieRecord record) throws IOException;
|
||||||
void close() throws IOException;
|
|
||||||
void writeAvro(String key, R oldRecord) throws IOException;
|
boolean canWrite();
|
||||||
|
|
||||||
|
void close() throws IOException;
|
||||||
|
|
||||||
|
void writeAvro(String key, R oldRecord) throws IOException;
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,42 +16,42 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.io.storage;
|
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.avro.HoodieAvroWriteSupport;
|
||||||
import com.uber.hoodie.common.BloomFilter;
|
import com.uber.hoodie.common.BloomFilter;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
import java.io.IOException;
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.avro.generic.IndexedRecord;
|
import org.apache.avro.generic.IndexedRecord;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
public class HoodieStorageWriterFactory {
|
public class HoodieStorageWriterFactory {
|
||||||
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
|
|
||||||
String commitTime, Path path, HoodieTable<T> hoodieTable, HoodieWriteConfig config, Schema schema)
|
|
||||||
throws IOException {
|
|
||||||
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
|
|
||||||
// Currently only parquet is supported
|
|
||||||
return newParquetStorageWriter(commitTime, path, config, schema);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(
|
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
|
||||||
String commitTime, Path path, HoodieWriteConfig config, Schema schema) throws IOException {
|
String commitTime, Path path, HoodieTable<T> hoodieTable, HoodieWriteConfig config,
|
||||||
BloomFilter filter =
|
Schema schema)
|
||||||
new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP());
|
throws IOException {
|
||||||
HoodieAvroWriteSupport writeSupport =
|
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
|
||||||
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
|
// Currently only parquet is supported
|
||||||
|
return newParquetStorageWriter(commitTime, path, config, schema);
|
||||||
|
}
|
||||||
|
|
||||||
HoodieParquetConfig parquetConfig =
|
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(
|
||||||
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
String commitTime, Path path, HoodieWriteConfig config, Schema schema) throws IOException {
|
||||||
config.getParquetBlockSize(), config.getParquetPageSize(),
|
BloomFilter filter =
|
||||||
config.getParquetMaxFileSize(), FSUtils.getFs().getConf());
|
new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP());
|
||||||
|
HoodieAvroWriteSupport writeSupport =
|
||||||
|
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
|
||||||
|
|
||||||
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
|
HoodieParquetConfig parquetConfig =
|
||||||
}
|
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
||||||
|
config.getParquetBlockSize(), config.getParquetPageSize(),
|
||||||
|
config.getParquetMaxFileSize(), FSUtils.getFs().getConf());
|
||||||
|
|
||||||
|
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
File diff suppressed because it is too large
Load Diff
@@ -16,44 +16,47 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.io.storage;
|
package com.uber.hoodie.io.storage;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Wrapper over <code>FSDataOutputStream</code> to keep track of the size of the written bytes.
|
* Wrapper over <code>FSDataOutputStream</code> to keep track of the size of the written bytes. This
|
||||||
* This gives a cheap way to check on the underlying file size.
|
* gives a cheap way to check on the underlying file size.
|
||||||
*/
|
*/
|
||||||
public class SizeAwareFSDataOutputStream extends FSDataOutputStream {
|
public class SizeAwareFSDataOutputStream extends FSDataOutputStream {
|
||||||
// A callback to call when the output stream is closed.
|
|
||||||
private final Runnable closeCallback;
|
|
||||||
// Keep track of the bytes written
|
|
||||||
private final AtomicLong bytesWritten = new AtomicLong(0L);
|
|
||||||
|
|
||||||
public SizeAwareFSDataOutputStream(FSDataOutputStream out, Runnable closeCallback)
|
// A callback to call when the output stream is closed.
|
||||||
throws IOException {
|
private final Runnable closeCallback;
|
||||||
super(out);
|
// Keep track of the bytes written
|
||||||
this.closeCallback = closeCallback;
|
private final AtomicLong bytesWritten = new AtomicLong(0L);
|
||||||
}
|
|
||||||
|
|
||||||
@Override public synchronized void write(byte[] b, int off, int len) throws IOException {
|
public SizeAwareFSDataOutputStream(FSDataOutputStream out, Runnable closeCallback)
|
||||||
bytesWritten.addAndGet(len);
|
throws IOException {
|
||||||
super.write(b, off, len);
|
super(out);
|
||||||
}
|
this.closeCallback = closeCallback;
|
||||||
|
}
|
||||||
|
|
||||||
@Override public void write(byte[] b) throws IOException {
|
@Override
|
||||||
bytesWritten.addAndGet(b.length);
|
public synchronized void write(byte[] b, int off, int len) throws IOException {
|
||||||
super.write(b);
|
bytesWritten.addAndGet(len);
|
||||||
}
|
super.write(b, off, len);
|
||||||
|
}
|
||||||
|
|
||||||
@Override public void close() throws IOException {
|
@Override
|
||||||
super.close();
|
public void write(byte[] b) throws IOException {
|
||||||
closeCallback.run();
|
bytesWritten.addAndGet(b.length);
|
||||||
}
|
super.write(b);
|
||||||
|
}
|
||||||
|
|
||||||
public long getBytesWritten() {
|
@Override
|
||||||
return bytesWritten.get();
|
public void close() throws IOException {
|
||||||
}
|
super.close();
|
||||||
|
closeCallback.run();
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getBytesWritten() {
|
||||||
|
return bytesWritten.get();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -22,7 +22,6 @@ import com.codahale.metrics.Timer;
|
|||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
@@ -30,119 +29,122 @@ import org.apache.log4j.Logger;
|
|||||||
* Wrapper for metrics-related operations.
|
* Wrapper for metrics-related operations.
|
||||||
*/
|
*/
|
||||||
public class HoodieMetrics {
|
public class HoodieMetrics {
|
||||||
private HoodieWriteConfig config = null;
|
|
||||||
private String tableName = null;
|
|
||||||
private static Logger logger = LogManager.getLogger(HoodieMetrics.class);
|
|
||||||
// Some timers
|
|
||||||
public String rollbackTimerName = null;
|
|
||||||
public String cleanTimerName = null;
|
|
||||||
public String commitTimerName = null;
|
|
||||||
private Timer rollbackTimer = null;
|
|
||||||
private Timer cleanTimer = null;
|
|
||||||
private Timer commitTimer = null;
|
|
||||||
|
|
||||||
public HoodieMetrics(HoodieWriteConfig config, String tableName) {
|
private HoodieWriteConfig config = null;
|
||||||
this.config = config;
|
private String tableName = null;
|
||||||
this.tableName = tableName;
|
private static Logger logger = LogManager.getLogger(HoodieMetrics.class);
|
||||||
if (config.isMetricsOn()) {
|
// Some timers
|
||||||
Metrics.init(config);
|
public String rollbackTimerName = null;
|
||||||
this.rollbackTimerName = getMetricsName("timer", "rollback");
|
public String cleanTimerName = null;
|
||||||
this.cleanTimerName = getMetricsName("timer", "clean");
|
public String commitTimerName = null;
|
||||||
this.commitTimerName = getMetricsName("timer", "commit");
|
private Timer rollbackTimer = null;
|
||||||
|
private Timer cleanTimer = null;
|
||||||
|
private Timer commitTimer = null;
|
||||||
|
|
||||||
|
public HoodieMetrics(HoodieWriteConfig config, String tableName) {
|
||||||
|
this.config = config;
|
||||||
|
this.tableName = tableName;
|
||||||
|
if (config.isMetricsOn()) {
|
||||||
|
Metrics.init(config);
|
||||||
|
this.rollbackTimerName = getMetricsName("timer", "rollback");
|
||||||
|
this.cleanTimerName = getMetricsName("timer", "clean");
|
||||||
|
this.commitTimerName = getMetricsName("timer", "commit");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private Timer createTimer(String name) {
|
||||||
|
return config.isMetricsOn() ? Metrics.getInstance().getRegistry().timer(name) : null;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Timer.Context getRollbackCtx() {
|
||||||
|
if (config.isMetricsOn() && rollbackTimer == null) {
|
||||||
|
rollbackTimer = createTimer(rollbackTimerName);
|
||||||
|
}
|
||||||
|
return rollbackTimer == null ? null : rollbackTimer.time();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Timer.Context getCleanCtx() {
|
||||||
|
if (config.isMetricsOn() && cleanTimer == null) {
|
||||||
|
cleanTimer = createTimer(cleanTimerName);
|
||||||
|
}
|
||||||
|
return cleanTimer == null ? null : cleanTimer.time();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Timer.Context getCommitCtx() {
|
||||||
|
if (config.isMetricsOn() && commitTimer == null) {
|
||||||
|
commitTimer = createTimer(commitTimerName);
|
||||||
|
}
|
||||||
|
return commitTimer == null ? null : commitTimer.time();
|
||||||
|
}
|
||||||
|
|
||||||
|
public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs,
|
||||||
|
HoodieCommitMetadata metadata) {
|
||||||
|
if (config.isMetricsOn()) {
|
||||||
|
long totalPartitionsWritten = metadata.fetchTotalPartitionsWritten();
|
||||||
|
long totalFilesInsert = metadata.fetchTotalFilesInsert();
|
||||||
|
long totalFilesUpdate = metadata.fetchTotalFilesUpdated();
|
||||||
|
long totalRecordsWritten = metadata.fetchTotalRecordsWritten();
|
||||||
|
long totalUpdateRecordsWritten = metadata.fetchTotalUpdateRecordsWritten();
|
||||||
|
long totalInsertRecordsWritten = metadata.fetchTotalInsertRecordsWritten();
|
||||||
|
long totalBytesWritten = metadata.fetchTotalBytesWritten();
|
||||||
|
registerGauge(getMetricsName("commit", "duration"), durationInMs);
|
||||||
|
registerGauge(getMetricsName("commit", "totalPartitionsWritten"), totalPartitionsWritten);
|
||||||
|
registerGauge(getMetricsName("commit", "totalFilesInsert"), totalFilesInsert);
|
||||||
|
registerGauge(getMetricsName("commit", "totalFilesUpdate"), totalFilesUpdate);
|
||||||
|
registerGauge(getMetricsName("commit", "totalRecordsWritten"), totalRecordsWritten);
|
||||||
|
registerGauge(getMetricsName("commit", "totalUpdateRecordsWritten"),
|
||||||
|
totalUpdateRecordsWritten);
|
||||||
|
registerGauge(getMetricsName("commit", "totalInsertRecordsWritten"),
|
||||||
|
totalInsertRecordsWritten);
|
||||||
|
registerGauge(getMetricsName("commit", "totalBytesWritten"), totalBytesWritten);
|
||||||
|
registerGauge(getMetricsName("commit", "commitTime"), commitEpochTimeInMs);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) {
|
||||||
|
if (config.isMetricsOn()) {
|
||||||
|
logger.info(String.format("Sending rollback metrics (duration=%d, numFilesDeleted=$d)",
|
||||||
|
durationInMs, numFilesDeleted));
|
||||||
|
registerGauge(getMetricsName("rollback", "duration"), durationInMs);
|
||||||
|
registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void updateCleanMetrics(long durationInMs, int numFilesDeleted) {
|
||||||
|
if (config.isMetricsOn()) {
|
||||||
|
logger.info(String.format("Sending clean metrics (duration=%d, numFilesDeleted=%d)",
|
||||||
|
durationInMs, numFilesDeleted));
|
||||||
|
registerGauge(getMetricsName("clean", "duration"), durationInMs);
|
||||||
|
registerGauge(getMetricsName("clean", "numFilesDeleted"), numFilesDeleted);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@VisibleForTesting
|
||||||
|
String getMetricsName(String action, String metric) {
|
||||||
|
return config == null ? null :
|
||||||
|
String.format("%s.%s.%s", tableName, action, metric);
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerGauge(String metricName, final long value) {
|
||||||
|
try {
|
||||||
|
MetricRegistry registry = Metrics.getInstance().getRegistry();
|
||||||
|
registry.register(metricName, new Gauge<Long>() {
|
||||||
|
@Override
|
||||||
|
public Long getValue() {
|
||||||
|
return value;
|
||||||
}
|
}
|
||||||
|
});
|
||||||
|
} catch (Exception e) {
|
||||||
|
// Here we catch all exception, so the major upsert pipeline will not be affected if the metrics system
|
||||||
|
// has some issues.
|
||||||
|
logger.error("Failed to send metrics: ", e);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private Timer createTimer(String name) {
|
/**
|
||||||
return config.isMetricsOn() ? Metrics.getInstance().getRegistry().timer(name) : null;
|
* By default, the timer context returns duration with nano seconds. Convert it to millisecond.
|
||||||
}
|
*/
|
||||||
|
public long getDurationInMs(long ctxDuration) {
|
||||||
public Timer.Context getRollbackCtx() {
|
return ctxDuration / 1000000;
|
||||||
if (config.isMetricsOn() && rollbackTimer == null) {
|
}
|
||||||
rollbackTimer = createTimer(rollbackTimerName);
|
|
||||||
}
|
|
||||||
return rollbackTimer == null ? null : rollbackTimer.time();
|
|
||||||
}
|
|
||||||
|
|
||||||
public Timer.Context getCleanCtx() {
|
|
||||||
if (config.isMetricsOn() && cleanTimer == null) {
|
|
||||||
cleanTimer = createTimer(cleanTimerName);
|
|
||||||
}
|
|
||||||
return cleanTimer == null ? null : cleanTimer.time();
|
|
||||||
}
|
|
||||||
|
|
||||||
public Timer.Context getCommitCtx() {
|
|
||||||
if (config.isMetricsOn() && commitTimer == null) {
|
|
||||||
commitTimer = createTimer(commitTimerName);
|
|
||||||
}
|
|
||||||
return commitTimer == null ? null : commitTimer.time();
|
|
||||||
}
|
|
||||||
|
|
||||||
public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs, HoodieCommitMetadata metadata) {
|
|
||||||
if (config.isMetricsOn()) {
|
|
||||||
long totalPartitionsWritten = metadata.fetchTotalPartitionsWritten();
|
|
||||||
long totalFilesInsert = metadata.fetchTotalFilesInsert();
|
|
||||||
long totalFilesUpdate = metadata.fetchTotalFilesUpdated();
|
|
||||||
long totalRecordsWritten = metadata.fetchTotalRecordsWritten();
|
|
||||||
long totalUpdateRecordsWritten = metadata.fetchTotalUpdateRecordsWritten();
|
|
||||||
long totalInsertRecordsWritten = metadata.fetchTotalInsertRecordsWritten();
|
|
||||||
long totalBytesWritten = metadata.fetchTotalBytesWritten();
|
|
||||||
registerGauge(getMetricsName("commit", "duration"), durationInMs);
|
|
||||||
registerGauge(getMetricsName("commit", "totalPartitionsWritten"), totalPartitionsWritten);
|
|
||||||
registerGauge(getMetricsName("commit", "totalFilesInsert"), totalFilesInsert);
|
|
||||||
registerGauge(getMetricsName("commit", "totalFilesUpdate"), totalFilesUpdate);
|
|
||||||
registerGauge(getMetricsName("commit", "totalRecordsWritten"), totalRecordsWritten);
|
|
||||||
registerGauge(getMetricsName("commit", "totalUpdateRecordsWritten"), totalUpdateRecordsWritten);
|
|
||||||
registerGauge(getMetricsName("commit", "totalInsertRecordsWritten"), totalInsertRecordsWritten);
|
|
||||||
registerGauge(getMetricsName("commit", "totalBytesWritten"), totalBytesWritten);
|
|
||||||
registerGauge(getMetricsName("commit", "commitTime"), commitEpochTimeInMs);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) {
|
|
||||||
if (config.isMetricsOn()) {
|
|
||||||
logger.info(String.format("Sending rollback metrics (duration=%d, numFilesDeleted=$d)",
|
|
||||||
durationInMs, numFilesDeleted));
|
|
||||||
registerGauge(getMetricsName("rollback", "duration"), durationInMs);
|
|
||||||
registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public void updateCleanMetrics(long durationInMs, int numFilesDeleted) {
|
|
||||||
if (config.isMetricsOn()) {
|
|
||||||
logger.info(String.format("Sending clean metrics (duration=%d, numFilesDeleted=%d)",
|
|
||||||
durationInMs, numFilesDeleted));
|
|
||||||
registerGauge(getMetricsName("clean", "duration"), durationInMs);
|
|
||||||
registerGauge(getMetricsName("clean", "numFilesDeleted"), numFilesDeleted);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@VisibleForTesting
|
|
||||||
String getMetricsName(String action, String metric) {
|
|
||||||
return config == null ? null :
|
|
||||||
String.format("%s.%s.%s", tableName, action, metric);
|
|
||||||
}
|
|
||||||
|
|
||||||
void registerGauge(String metricName, final long value) {
|
|
||||||
try {
|
|
||||||
MetricRegistry registry = Metrics.getInstance().getRegistry();
|
|
||||||
registry.register(metricName, new Gauge<Long>() {
|
|
||||||
@Override
|
|
||||||
public Long getValue() {
|
|
||||||
return value;
|
|
||||||
}
|
|
||||||
});
|
|
||||||
} catch (Exception e) {
|
|
||||||
// Here we catch all exception, so the major upsert pipeline will not be affected if the metrics system
|
|
||||||
// has some issues.
|
|
||||||
logger.error("Failed to send metrics: ", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* By default, the timer context returns duration with nano seconds.
|
|
||||||
* Convert it to millisecond.
|
|
||||||
*/
|
|
||||||
public long getDurationInMs(long ctxDuration) {
|
|
||||||
return ctxDuration / 1000000;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -22,16 +22,17 @@ import java.io.Closeable;
|
|||||||
* Used for testing.
|
* Used for testing.
|
||||||
*/
|
*/
|
||||||
public class InMemoryMetricsReporter extends MetricsReporter {
|
public class InMemoryMetricsReporter extends MetricsReporter {
|
||||||
@Override
|
|
||||||
public void start() {
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void report() {
|
public void start() {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Closeable getReporter() {
|
public void report() {
|
||||||
return null;
|
}
|
||||||
}
|
|
||||||
|
@Override
|
||||||
|
public Closeable getReporter() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -19,65 +19,64 @@ package com.uber.hoodie.metrics;
|
|||||||
import com.codahale.metrics.MetricRegistry;
|
import com.codahale.metrics.MetricRegistry;
|
||||||
import com.google.common.io.Closeables;
|
import com.google.common.io.Closeables;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.config.HoodieMetricsConfig;
|
|
||||||
import com.uber.hoodie.exception.HoodieException;
|
import com.uber.hoodie.exception.HoodieException;
|
||||||
import org.apache.commons.configuration.ConfigurationException;
|
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
|
import org.apache.commons.configuration.ConfigurationException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This is the main class of the metrics system.
|
* This is the main class of the metrics system.
|
||||||
*/
|
*/
|
||||||
public class Metrics {
|
public class Metrics {
|
||||||
private static volatile boolean initialized = false;
|
|
||||||
private static Metrics metrics = null;
|
|
||||||
private final MetricRegistry registry;
|
|
||||||
private MetricsReporter reporter = null;
|
|
||||||
|
|
||||||
private Metrics(HoodieWriteConfig metricConfig) throws ConfigurationException {
|
private static volatile boolean initialized = false;
|
||||||
registry = new MetricRegistry();
|
private static Metrics metrics = null;
|
||||||
|
private final MetricRegistry registry;
|
||||||
|
private MetricsReporter reporter = null;
|
||||||
|
|
||||||
reporter = MetricsReporterFactory.createReporter(metricConfig, registry);
|
private Metrics(HoodieWriteConfig metricConfig) throws ConfigurationException {
|
||||||
if (reporter == null) {
|
registry = new MetricRegistry();
|
||||||
throw new RuntimeException("Cannot initialize Reporter.");
|
|
||||||
}
|
reporter = MetricsReporterFactory.createReporter(metricConfig, registry);
|
||||||
|
if (reporter == null) {
|
||||||
|
throw new RuntimeException("Cannot initialize Reporter.");
|
||||||
|
}
|
||||||
// reporter.start();
|
// reporter.start();
|
||||||
|
|
||||||
Runtime.getRuntime().addShutdownHook(new Thread() {
|
Runtime.getRuntime().addShutdownHook(new Thread() {
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
try {
|
|
||||||
reporter.report();
|
|
||||||
Closeables.close(reporter.getReporter(), true);
|
|
||||||
} catch (Exception e) {
|
|
||||||
e.printStackTrace();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Metrics getInstance() {
|
|
||||||
assert initialized;
|
|
||||||
return metrics;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static synchronized void init(HoodieWriteConfig metricConfig) {
|
|
||||||
if (initialized) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
try {
|
try {
|
||||||
metrics = new Metrics(metricConfig);
|
reporter.report();
|
||||||
} catch (ConfigurationException e) {
|
Closeables.close(reporter.getReporter(), true);
|
||||||
throw new HoodieException(e);
|
} catch (Exception e) {
|
||||||
|
e.printStackTrace();
|
||||||
}
|
}
|
||||||
initialized = true;
|
}
|
||||||
}
|
});
|
||||||
|
}
|
||||||
|
|
||||||
public MetricRegistry getRegistry() {
|
public static Metrics getInstance() {
|
||||||
return registry;
|
assert initialized;
|
||||||
}
|
return metrics;
|
||||||
|
}
|
||||||
|
|
||||||
public Closeable getReporter() {
|
public static synchronized void init(HoodieWriteConfig metricConfig) {
|
||||||
return reporter.getReporter();
|
if (initialized) {
|
||||||
|
return;
|
||||||
}
|
}
|
||||||
|
try {
|
||||||
|
metrics = new Metrics(metricConfig);
|
||||||
|
} catch (ConfigurationException e) {
|
||||||
|
throw new HoodieException(e);
|
||||||
|
}
|
||||||
|
initialized = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
public MetricRegistry getRegistry() {
|
||||||
|
return registry;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Closeable getReporter() {
|
||||||
|
return reporter.getReporter();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -21,75 +21,74 @@ import com.codahale.metrics.MetricRegistry;
|
|||||||
import com.codahale.metrics.graphite.Graphite;
|
import com.codahale.metrics.graphite.Graphite;
|
||||||
import com.codahale.metrics.graphite.GraphiteReporter;
|
import com.codahale.metrics.graphite.GraphiteReporter;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
|
|
||||||
import org.apache.log4j.LogManager;
|
|
||||||
import org.apache.log4j.Logger;
|
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.net.InetSocketAddress;
|
import java.net.InetSocketAddress;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implementation of Graphite reporter, which connects to the Graphite server,
|
* Implementation of Graphite reporter, which connects to the Graphite server, and send metrics to
|
||||||
* and send metrics to that server.
|
* that server.
|
||||||
*/
|
*/
|
||||||
public class MetricsGraphiteReporter extends MetricsReporter {
|
public class MetricsGraphiteReporter extends MetricsReporter {
|
||||||
private final MetricRegistry registry;
|
|
||||||
private final GraphiteReporter graphiteReporter;
|
|
||||||
private final HoodieWriteConfig config;
|
|
||||||
private String serverHost;
|
|
||||||
private int serverPort;
|
|
||||||
|
|
||||||
private static Logger logger = LogManager.getLogger(MetricsGraphiteReporter.class);
|
private final MetricRegistry registry;
|
||||||
|
private final GraphiteReporter graphiteReporter;
|
||||||
|
private final HoodieWriteConfig config;
|
||||||
|
private String serverHost;
|
||||||
|
private int serverPort;
|
||||||
|
|
||||||
public MetricsGraphiteReporter(HoodieWriteConfig config, MetricRegistry registry) {
|
private static Logger logger = LogManager.getLogger(MetricsGraphiteReporter.class);
|
||||||
this.registry = registry;
|
|
||||||
this.config = config;
|
|
||||||
|
|
||||||
// Check the serverHost and serverPort here
|
public MetricsGraphiteReporter(HoodieWriteConfig config, MetricRegistry registry) {
|
||||||
this.serverHost = config.getGraphiteServerHost();
|
this.registry = registry;
|
||||||
this.serverPort = config.getGraphiteServerPort();
|
this.config = config;
|
||||||
if (serverHost == null || serverPort == 0) {
|
|
||||||
throw new RuntimeException(
|
|
||||||
String.format("Graphite cannot be initialized with serverHost[%s] and serverPort[%s].",
|
|
||||||
serverHost, serverPort));
|
|
||||||
}
|
|
||||||
|
|
||||||
this.graphiteReporter = createGraphiteReport();
|
// Check the serverHost and serverPort here
|
||||||
|
this.serverHost = config.getGraphiteServerHost();
|
||||||
|
this.serverPort = config.getGraphiteServerPort();
|
||||||
|
if (serverHost == null || serverPort == 0) {
|
||||||
|
throw new RuntimeException(
|
||||||
|
String.format("Graphite cannot be initialized with serverHost[%s] and serverPort[%s].",
|
||||||
|
serverHost, serverPort));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
this.graphiteReporter = createGraphiteReport();
|
||||||
public void start() {
|
}
|
||||||
if (graphiteReporter != null) {
|
|
||||||
graphiteReporter.start(30, TimeUnit.SECONDS);
|
|
||||||
} else {
|
|
||||||
logger.error("Cannot start as the graphiteReporter is null.");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void report() {
|
public void start() {
|
||||||
if (graphiteReporter != null) {
|
if (graphiteReporter != null) {
|
||||||
graphiteReporter.report();
|
graphiteReporter.start(30, TimeUnit.SECONDS);
|
||||||
} else {
|
} else {
|
||||||
logger.error("Cannot report metrics as the graphiteReporter is null.");
|
logger.error("Cannot start as the graphiteReporter is null.");
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Closeable getReporter() {
|
public void report() {
|
||||||
return graphiteReporter;
|
if (graphiteReporter != null) {
|
||||||
|
graphiteReporter.report();
|
||||||
|
} else {
|
||||||
|
logger.error("Cannot report metrics as the graphiteReporter is null.");
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private GraphiteReporter createGraphiteReport() {
|
@Override
|
||||||
Graphite graphite = new Graphite(
|
public Closeable getReporter() {
|
||||||
new InetSocketAddress(serverHost, serverPort));
|
return graphiteReporter;
|
||||||
String reporterPrefix = config.getGraphiteMetricPrefix();
|
}
|
||||||
return GraphiteReporter.forRegistry(registry)
|
|
||||||
.prefixedWith(reporterPrefix)
|
private GraphiteReporter createGraphiteReport() {
|
||||||
.convertRatesTo(TimeUnit.SECONDS)
|
Graphite graphite = new Graphite(
|
||||||
.convertDurationsTo(TimeUnit.MILLISECONDS)
|
new InetSocketAddress(serverHost, serverPort));
|
||||||
.filter(MetricFilter.ALL)
|
String reporterPrefix = config.getGraphiteMetricPrefix();
|
||||||
.build(graphite);
|
return GraphiteReporter.forRegistry(registry)
|
||||||
}
|
.prefixedWith(reporterPrefix)
|
||||||
|
.convertRatesTo(TimeUnit.SECONDS)
|
||||||
|
.convertDurationsTo(TimeUnit.MILLISECONDS)
|
||||||
|
.filter(MetricFilter.ALL)
|
||||||
|
.build(graphite);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -22,15 +22,16 @@ import java.io.Closeable;
|
|||||||
* Interface for implementing a Reporter.
|
* Interface for implementing a Reporter.
|
||||||
*/
|
*/
|
||||||
public abstract class MetricsReporter {
|
public abstract class MetricsReporter {
|
||||||
/**
|
|
||||||
* Push out metrics at scheduled intervals
|
|
||||||
*/
|
|
||||||
public abstract void start();
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Deterministically push out metrics
|
* Push out metrics at scheduled intervals
|
||||||
*/
|
*/
|
||||||
public abstract void report();
|
public abstract void start();
|
||||||
|
|
||||||
public abstract Closeable getReporter();
|
/**
|
||||||
|
* Deterministically push out metrics
|
||||||
|
*/
|
||||||
|
public abstract void report();
|
||||||
|
|
||||||
|
public abstract Closeable getReporter();
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,7 +18,6 @@ package com.uber.hoodie.metrics;
|
|||||||
|
|
||||||
import com.codahale.metrics.MetricRegistry;
|
import com.codahale.metrics.MetricRegistry;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
@@ -26,23 +25,24 @@ import org.apache.log4j.Logger;
|
|||||||
* Factory class for creating MetricsReporter.
|
* Factory class for creating MetricsReporter.
|
||||||
*/
|
*/
|
||||||
public class MetricsReporterFactory {
|
public class MetricsReporterFactory {
|
||||||
private static Logger logger = LogManager.getLogger(MetricsReporterFactory.class);
|
|
||||||
|
|
||||||
public static MetricsReporter createReporter(HoodieWriteConfig config,
|
private static Logger logger = LogManager.getLogger(MetricsReporterFactory.class);
|
||||||
MetricRegistry registry) {
|
|
||||||
MetricsReporterType type = config.getMetricsReporterType();
|
public static MetricsReporter createReporter(HoodieWriteConfig config,
|
||||||
MetricsReporter reporter = null;
|
MetricRegistry registry) {
|
||||||
switch (type) {
|
MetricsReporterType type = config.getMetricsReporterType();
|
||||||
case GRAPHITE:
|
MetricsReporter reporter = null;
|
||||||
reporter = new MetricsGraphiteReporter(config, registry);
|
switch (type) {
|
||||||
break;
|
case GRAPHITE:
|
||||||
case INMEMORY:
|
reporter = new MetricsGraphiteReporter(config, registry);
|
||||||
reporter = new InMemoryMetricsReporter();
|
break;
|
||||||
break;
|
case INMEMORY:
|
||||||
default:
|
reporter = new InMemoryMetricsReporter();
|
||||||
logger.error("Reporter type[" + type + "] is not supported.");
|
break;
|
||||||
break;
|
default:
|
||||||
}
|
logger.error("Reporter type[" + type + "] is not supported.");
|
||||||
return reporter;
|
break;
|
||||||
}
|
}
|
||||||
|
return reporter;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,10 +17,10 @@
|
|||||||
package com.uber.hoodie.metrics;
|
package com.uber.hoodie.metrics;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Types of the reporter. Right now we only support Graphite.
|
* Types of the reporter. Right now we only support Graphite. We can include JMX and CSV in the
|
||||||
* We can include JMX and CSV in the future.
|
* future.
|
||||||
*/
|
*/
|
||||||
public enum MetricsReporterType {
|
public enum MetricsReporterType {
|
||||||
GRAPHITE,
|
GRAPHITE,
|
||||||
INMEMORY
|
INMEMORY
|
||||||
}
|
}
|
||||||
|
|||||||
File diff suppressed because it is too large
Load Diff
@@ -39,13 +39,6 @@ import com.uber.hoodie.exception.HoodieCompactionException;
|
|||||||
import com.uber.hoodie.exception.HoodieRollbackException;
|
import com.uber.hoodie.exception.HoodieRollbackException;
|
||||||
import com.uber.hoodie.io.HoodieAppendHandle;
|
import com.uber.hoodie.io.HoodieAppendHandle;
|
||||||
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.log4j.LogManager;
|
|
||||||
import org.apache.log4j.Logger;
|
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
|
||||||
import org.apache.spark.api.java.function.Function;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.UncheckedIOException;
|
import java.io.UncheckedIOException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
@@ -56,179 +49,209 @@ import java.util.List;
|
|||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.log4j.LogManager;
|
||||||
|
import org.apache.log4j.Logger;
|
||||||
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
|
import org.apache.spark.api.java.function.Function;
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implementation of a more real-time read-optimized Hoodie Table where
|
* Implementation of a more real-time read-optimized Hoodie Table where
|
||||||
*
|
*
|
||||||
* INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or)
|
* INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or)
|
||||||
* Merge with the smallest existing file, to expand it
|
* Merge with the smallest existing file, to expand it
|
||||||
*
|
*
|
||||||
* UPDATES - Appends the changes to a rolling log file maintained per file Id.
|
* UPDATES - Appends the changes to a rolling log file maintained per file Id. Compaction merges the
|
||||||
* Compaction merges the log file into the base file.
|
* log file into the base file.
|
||||||
*
|
*
|
||||||
* WARNING - MOR table type does not support nested rollbacks, every rollback
|
* WARNING - MOR table type does not support nested rollbacks, every rollback must be followed by an
|
||||||
* must be followed by an attempted commit action
|
* attempted commit action
|
||||||
*/
|
*/
|
||||||
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends HoodieCopyOnWriteTable<T> {
|
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||||
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
|
HoodieCopyOnWriteTable<T> {
|
||||||
|
|
||||||
public HoodieMergeOnReadTable(HoodieWriteConfig config,
|
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
|
||||||
HoodieTableMetaClient metaClient) {
|
|
||||||
super(config, metaClient);
|
public HoodieMergeOnReadTable(HoodieWriteConfig config,
|
||||||
|
HoodieTableMetaClient metaClient) {
|
||||||
|
super(config, metaClient);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
|
||||||
|
Iterator<HoodieRecord<T>> recordItr) throws IOException {
|
||||||
|
logger.info("Merging updates for commit " + commitTime + " for file " + fileId);
|
||||||
|
HoodieAppendHandle<T> appendHandle =
|
||||||
|
new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr);
|
||||||
|
appendHandle.doAppend();
|
||||||
|
appendHandle.close();
|
||||||
|
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus()))
|
||||||
|
.iterator();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc) {
|
||||||
|
logger.info("Checking if compaction needs to be run on " + config.getBasePath());
|
||||||
|
Optional<HoodieInstant> lastCompaction = getActiveTimeline().getCompactionTimeline()
|
||||||
|
.filterCompletedInstants().lastInstant();
|
||||||
|
String deltaCommitsSinceTs = "0";
|
||||||
|
if (lastCompaction.isPresent()) {
|
||||||
|
deltaCommitsSinceTs = lastCompaction.get().getTimestamp();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
int deltaCommitsSinceLastCompaction = getActiveTimeline().getDeltaCommitTimeline()
|
||||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
|
.findInstantsAfter(deltaCommitsSinceTs, Integer.MAX_VALUE).countInstants();
|
||||||
Iterator<HoodieRecord<T>> recordItr) throws IOException {
|
if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
|
||||||
logger.info("Merging updates for commit " + commitTime + " for file " + fileId);
|
logger.info("Not running compaction as only " + deltaCommitsSinceLastCompaction
|
||||||
HoodieAppendHandle<T> appendHandle =
|
+ " delta commits was found since last compaction " + deltaCommitsSinceTs
|
||||||
new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr);
|
+ ". Waiting for " + config.getInlineCompactDeltaCommitMax());
|
||||||
appendHandle.doAppend();
|
return Optional.empty();
|
||||||
appendHandle.close();
|
|
||||||
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus()))
|
|
||||||
.iterator();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
logger.info("Compacting merge on read table " + config.getBasePath());
|
||||||
public Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc) {
|
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
|
||||||
logger.info("Checking if compaction needs to be run on " + config.getBasePath());
|
try {
|
||||||
Optional<HoodieInstant> lastCompaction = getActiveTimeline().getCompactionTimeline()
|
return Optional.of(compactor.compact(jsc, config, this));
|
||||||
.filterCompletedInstants().lastInstant();
|
} catch (IOException e) {
|
||||||
String deltaCommitsSinceTs = "0";
|
throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e);
|
||||||
if (lastCompaction.isPresent()) {
|
|
||||||
deltaCommitsSinceTs = lastCompaction.get().getTimestamp();
|
|
||||||
}
|
|
||||||
|
|
||||||
int deltaCommitsSinceLastCompaction = getActiveTimeline().getDeltaCommitTimeline()
|
|
||||||
.findInstantsAfter(deltaCommitsSinceTs, Integer.MAX_VALUE).countInstants();
|
|
||||||
if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
|
|
||||||
logger.info("Not running compaction as only " + deltaCommitsSinceLastCompaction
|
|
||||||
+ " delta commits was found since last compaction " + deltaCommitsSinceTs
|
|
||||||
+ ". Waiting for " + config.getInlineCompactDeltaCommitMax());
|
|
||||||
return Optional.empty();
|
|
||||||
}
|
|
||||||
|
|
||||||
logger.info("Compacting merge on read table " + config.getBasePath());
|
|
||||||
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
|
|
||||||
try {
|
|
||||||
return Optional.of(compactor.compact(jsc, config, this));
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits) throws IOException {
|
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits)
|
||||||
|
throws IOException {
|
||||||
|
|
||||||
//At the moment, MOR table type does not support nested rollbacks
|
//At the moment, MOR table type does not support nested rollbacks
|
||||||
if(commits.size() > 1) {
|
if (commits.size() > 1) {
|
||||||
throw new UnsupportedOperationException("Nested Rollbacks are not supported");
|
throw new UnsupportedOperationException("Nested Rollbacks are not supported");
|
||||||
}
|
|
||||||
Map<String, HoodieInstant> commitsAndCompactions =
|
|
||||||
this.getActiveTimeline()
|
|
||||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION))
|
|
||||||
.getInstants()
|
|
||||||
.filter(i -> commits.contains(i.getTimestamp()))
|
|
||||||
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
|
|
||||||
|
|
||||||
// Atomically un-publish all non-inflight commits
|
|
||||||
commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue())
|
|
||||||
.filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight);
|
|
||||||
|
|
||||||
logger.info("Unpublished " + commits);
|
|
||||||
|
|
||||||
Long startTime = System.currentTimeMillis();
|
|
||||||
|
|
||||||
List<HoodieRollbackStat> allRollbackStats = commits.stream().map(commit -> {
|
|
||||||
HoodieInstant instant = commitsAndCompactions.get(commit);
|
|
||||||
List<HoodieRollbackStat> stats = null;
|
|
||||||
switch (instant.getAction()) {
|
|
||||||
case HoodieTimeline.COMMIT_ACTION:
|
|
||||||
case HoodieTimeline.COMPACTION_ACTION:
|
|
||||||
try {
|
|
||||||
logger.info("Starting to rollback Commit/Compaction " + instant);
|
|
||||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
|
||||||
.fromBytes(this.getCommitTimeline().getInstantDetails(new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
|
||||||
|
|
||||||
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream().collect(Collectors.toList()))
|
|
||||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
|
||||||
Map<FileStatus, Boolean> results = super.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
|
||||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
|
||||||
.withDeletedFileResults(results).build();
|
|
||||||
}).collect();
|
|
||||||
logger.info("Finished rollback of Commit/Compaction " + instant);
|
|
||||||
break;
|
|
||||||
} catch (IOException io) {
|
|
||||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
|
||||||
}
|
|
||||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
|
||||||
try {
|
|
||||||
logger.info("Starting to rollback delta commit " + instant);
|
|
||||||
|
|
||||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
|
||||||
.fromBytes(this.getCommitTimeline().getInstantDetails(new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
|
||||||
|
|
||||||
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream().collect(Collectors.toList()))
|
|
||||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
|
||||||
// read commit file and (either append delete blocks or delete file)
|
|
||||||
Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
|
||||||
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
|
|
||||||
|
|
||||||
// we do not know fileIds for inserts (first inserts are parquet files), delete all parquet files for the corresponding failed commit, if present (same as COW)
|
|
||||||
filesToDeletedStatus = super.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
|
||||||
|
|
||||||
// append rollback blocks for updates
|
|
||||||
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream().filter(wStat -> wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT).forEach(wStat -> {
|
|
||||||
HoodieLogFormat.Writer writer = null;
|
|
||||||
try {
|
|
||||||
writer = HoodieLogFormat.newWriterBuilder()
|
|
||||||
.onParentPath(new Path(this.getMetaClient().getBasePath(), partitionPath))
|
|
||||||
.withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit())
|
|
||||||
.withFs(FSUtils.getFs()).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
|
||||||
Long numRollbackBlocks = 0L;
|
|
||||||
// generate metadata
|
|
||||||
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
|
|
||||||
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
|
|
||||||
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, commit);
|
|
||||||
// if update belongs to an existing log file
|
|
||||||
writer.appendBlock(new HoodieCommandBlock(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata));
|
|
||||||
numRollbackBlocks++;
|
|
||||||
if(wStat.getNumDeletes() > 0) {
|
|
||||||
writer.appendBlock(new HoodieCommandBlock(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata));
|
|
||||||
numRollbackBlocks++;
|
|
||||||
}
|
|
||||||
filesToNumBlocksRollback.put(FSUtils.getFs().getFileStatus(writer.getLogFile().getPath()), numRollbackBlocks);
|
|
||||||
} catch (IOException | InterruptedException io) {
|
|
||||||
throw new HoodieRollbackException("Failed to rollback for commit " + commit, io);
|
|
||||||
} finally {
|
|
||||||
try {
|
|
||||||
writer.close();
|
|
||||||
} catch (IOException io) {
|
|
||||||
throw new UncheckedIOException(io);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
});
|
|
||||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
|
||||||
.withDeletedFileResults(filesToDeletedStatus)
|
|
||||||
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build();
|
|
||||||
}).collect();
|
|
||||||
logger.info("Fnished rollback of delta commit " + instant);
|
|
||||||
break;
|
|
||||||
} catch (IOException io) {
|
|
||||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return stats;
|
|
||||||
}).flatMap(x -> x.stream()).collect(Collectors.toList());
|
|
||||||
|
|
||||||
commitsAndCompactions.entrySet().stream()
|
|
||||||
.map(entry -> new HoodieInstant(true, entry.getValue().getAction(), entry.getValue().getTimestamp()))
|
|
||||||
.forEach(this.getActiveTimeline()::deleteInflight);
|
|
||||||
|
|
||||||
logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
|
||||||
|
|
||||||
return allRollbackStats;
|
|
||||||
}
|
}
|
||||||
|
Map<String, HoodieInstant> commitsAndCompactions =
|
||||||
|
this.getActiveTimeline()
|
||||||
|
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
||||||
|
HoodieActiveTimeline.COMPACTION_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION))
|
||||||
|
.getInstants()
|
||||||
|
.filter(i -> commits.contains(i.getTimestamp()))
|
||||||
|
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
|
||||||
|
|
||||||
|
// Atomically un-publish all non-inflight commits
|
||||||
|
commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue())
|
||||||
|
.filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight);
|
||||||
|
|
||||||
|
logger.info("Unpublished " + commits);
|
||||||
|
|
||||||
|
Long startTime = System.currentTimeMillis();
|
||||||
|
|
||||||
|
List<HoodieRollbackStat> allRollbackStats = commits.stream().map(commit -> {
|
||||||
|
HoodieInstant instant = commitsAndCompactions.get(commit);
|
||||||
|
List<HoodieRollbackStat> stats = null;
|
||||||
|
switch (instant.getAction()) {
|
||||||
|
case HoodieTimeline.COMMIT_ACTION:
|
||||||
|
case HoodieTimeline.COMPACTION_ACTION:
|
||||||
|
try {
|
||||||
|
logger.info("Starting to rollback Commit/Compaction " + instant);
|
||||||
|
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||||
|
.fromBytes(this.getCommitTimeline().getInstantDetails(
|
||||||
|
new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
||||||
|
|
||||||
|
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream()
|
||||||
|
.collect(Collectors.toList()))
|
||||||
|
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||||
|
Map<FileStatus, Boolean> results = super
|
||||||
|
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||||
|
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||||
|
.withDeletedFileResults(results).build();
|
||||||
|
}).collect();
|
||||||
|
logger.info("Finished rollback of Commit/Compaction " + instant);
|
||||||
|
break;
|
||||||
|
} catch (IOException io) {
|
||||||
|
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||||
|
}
|
||||||
|
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||||
|
try {
|
||||||
|
logger.info("Starting to rollback delta commit " + instant);
|
||||||
|
|
||||||
|
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||||
|
.fromBytes(this.getCommitTimeline().getInstantDetails(
|
||||||
|
new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
||||||
|
|
||||||
|
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream()
|
||||||
|
.collect(Collectors.toList()))
|
||||||
|
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||||
|
// read commit file and (either append delete blocks or delete file)
|
||||||
|
Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
||||||
|
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
|
||||||
|
|
||||||
|
// we do not know fileIds for inserts (first inserts are parquet files), delete all parquet files for the corresponding failed commit, if present (same as COW)
|
||||||
|
filesToDeletedStatus = super
|
||||||
|
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||||
|
|
||||||
|
// append rollback blocks for updates
|
||||||
|
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
|
||||||
|
.filter(wStat -> wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT)
|
||||||
|
.forEach(wStat -> {
|
||||||
|
HoodieLogFormat.Writer writer = null;
|
||||||
|
try {
|
||||||
|
writer = HoodieLogFormat.newWriterBuilder()
|
||||||
|
.onParentPath(
|
||||||
|
new Path(this.getMetaClient().getBasePath(), partitionPath))
|
||||||
|
.withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit())
|
||||||
|
.withFs(FSUtils.getFs())
|
||||||
|
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||||
|
Long numRollbackBlocks = 0L;
|
||||||
|
// generate metadata
|
||||||
|
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
|
||||||
|
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME,
|
||||||
|
metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
|
||||||
|
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, commit);
|
||||||
|
// if update belongs to an existing log file
|
||||||
|
writer.appendBlock(new HoodieCommandBlock(
|
||||||
|
HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK,
|
||||||
|
metadata));
|
||||||
|
numRollbackBlocks++;
|
||||||
|
if (wStat.getNumDeletes() > 0) {
|
||||||
|
writer.appendBlock(new HoodieCommandBlock(
|
||||||
|
HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK,
|
||||||
|
metadata));
|
||||||
|
numRollbackBlocks++;
|
||||||
|
}
|
||||||
|
filesToNumBlocksRollback
|
||||||
|
.put(FSUtils.getFs().getFileStatus(writer.getLogFile().getPath()),
|
||||||
|
numRollbackBlocks);
|
||||||
|
} catch (IOException | InterruptedException io) {
|
||||||
|
throw new HoodieRollbackException(
|
||||||
|
"Failed to rollback for commit " + commit, io);
|
||||||
|
} finally {
|
||||||
|
try {
|
||||||
|
writer.close();
|
||||||
|
} catch (IOException io) {
|
||||||
|
throw new UncheckedIOException(io);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
});
|
||||||
|
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||||
|
.withDeletedFileResults(filesToDeletedStatus)
|
||||||
|
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build();
|
||||||
|
}).collect();
|
||||||
|
logger.info("Fnished rollback of delta commit " + instant);
|
||||||
|
break;
|
||||||
|
} catch (IOException io) {
|
||||||
|
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return stats;
|
||||||
|
}).flatMap(x -> x.stream()).collect(Collectors.toList());
|
||||||
|
|
||||||
|
commitsAndCompactions.entrySet().stream()
|
||||||
|
.map(entry -> new HoodieInstant(true, entry.getValue().getAction(),
|
||||||
|
entry.getValue().getTimestamp()))
|
||||||
|
.forEach(this.getActiveTimeline()::deleteInflight);
|
||||||
|
|
||||||
|
logger
|
||||||
|
.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
||||||
|
|
||||||
|
return allRollbackStats;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -34,7 +34,6 @@ import com.uber.hoodie.common.util.AvroUtils;
|
|||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.exception.HoodieCommitException;
|
import com.uber.hoodie.exception.HoodieCommitException;
|
||||||
import com.uber.hoodie.exception.HoodieException;
|
import com.uber.hoodie.exception.HoodieException;
|
||||||
import com.uber.hoodie.exception.HoodieRollbackException;
|
|
||||||
import com.uber.hoodie.exception.HoodieSavepointException;
|
import com.uber.hoodie.exception.HoodieSavepointException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
@@ -43,8 +42,6 @@ import java.util.List;
|
|||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
@@ -55,291 +52,245 @@ import org.apache.spark.api.java.JavaSparkContext;
|
|||||||
* Abstract implementation of a HoodieTable
|
* Abstract implementation of a HoodieTable
|
||||||
*/
|
*/
|
||||||
public abstract class HoodieTable<T extends HoodieRecordPayload> implements Serializable {
|
public abstract class HoodieTable<T extends HoodieRecordPayload> implements Serializable {
|
||||||
protected final HoodieWriteConfig config;
|
|
||||||
protected final HoodieTableMetaClient metaClient;
|
|
||||||
private static Logger logger = LogManager.getLogger(HoodieTable.class);
|
|
||||||
|
|
||||||
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
protected final HoodieWriteConfig config;
|
||||||
this.config = config;
|
protected final HoodieTableMetaClient metaClient;
|
||||||
this.metaClient = metaClient;
|
private static Logger logger = LogManager.getLogger(HoodieTable.class);
|
||||||
|
|
||||||
|
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||||
|
this.config = config;
|
||||||
|
this.metaClient = metaClient;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Provides a partitioner to perform the upsert operation, based on the workload profile
|
||||||
|
*/
|
||||||
|
public abstract Partitioner getUpsertPartitioner(WorkloadProfile profile);
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Provides a partitioner to perform the insert operation, based on the workload profile
|
||||||
|
*/
|
||||||
|
public abstract Partitioner getInsertPartitioner(WorkloadProfile profile);
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return whether this HoodieTable implementation can benefit from workload profiling
|
||||||
|
*/
|
||||||
|
public abstract boolean isWorkloadProfileNeeded();
|
||||||
|
|
||||||
|
public HoodieWriteConfig getConfig() {
|
||||||
|
return config;
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieTableMetaClient getMetaClient() {
|
||||||
|
return metaClient;
|
||||||
|
}
|
||||||
|
|
||||||
|
public FileSystem getFs() {
|
||||||
|
return metaClient.getFs();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the view of the file system for this table
|
||||||
|
*/
|
||||||
|
public TableFileSystemView getFileSystemView() {
|
||||||
|
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the read optimized view of the file system for this table
|
||||||
|
*/
|
||||||
|
public TableFileSystemView.ReadOptimizedView getROFileSystemView() {
|
||||||
|
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the real time view of the file system for this table
|
||||||
|
*/
|
||||||
|
public TableFileSystemView.RealtimeView getRTFileSystemView() {
|
||||||
|
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the completed (commit + compaction) view of the file system for this table
|
||||||
|
*/
|
||||||
|
public TableFileSystemView getCompletedFileSystemView() {
|
||||||
|
return new HoodieTableFileSystemView(metaClient, getCommitTimeline());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get only the completed (no-inflights) commit timeline
|
||||||
|
*/
|
||||||
|
public HoodieTimeline getCompletedCommitTimeline() {
|
||||||
|
return getCommitTimeline().filterCompletedInstants();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get only the inflights (no-completed) commit timeline
|
||||||
|
*/
|
||||||
|
public HoodieTimeline getInflightCommitTimeline() {
|
||||||
|
return getCommitTimeline().filterInflights();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get only the completed (no-inflights) clean timeline
|
||||||
|
*/
|
||||||
|
public HoodieTimeline getCompletedCleanTimeline() {
|
||||||
|
return getActiveTimeline().getCleanerTimeline().filterCompletedInstants();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get only the completed (no-inflights) savepoint timeline
|
||||||
|
*/
|
||||||
|
public HoodieTimeline getCompletedSavepointTimeline() {
|
||||||
|
return getActiveTimeline().getSavePointTimeline().filterCompletedInstants();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the list of savepoints in this table
|
||||||
|
*/
|
||||||
|
public List<String> getSavepoints() {
|
||||||
|
return getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the list of data file names savepointed
|
||||||
|
*/
|
||||||
|
public Stream<String> getSavepointedDataFiles(String savepointTime) {
|
||||||
|
if (!getSavepoints().contains(savepointTime)) {
|
||||||
|
throw new HoodieSavepointException(
|
||||||
|
"Could not get data files for savepoint " + savepointTime + ". No such savepoint.");
|
||||||
}
|
}
|
||||||
|
HoodieInstant instant =
|
||||||
/**
|
new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime);
|
||||||
* Provides a partitioner to perform the upsert operation, based on the
|
HoodieSavepointMetadata metadata = null;
|
||||||
* workload profile
|
try {
|
||||||
*
|
metadata = AvroUtils.deserializeHoodieSavepointMetadata(
|
||||||
* @return
|
getActiveTimeline().getInstantDetails(instant).get());
|
||||||
*/
|
} catch (IOException e) {
|
||||||
public abstract Partitioner getUpsertPartitioner(WorkloadProfile profile);
|
throw new HoodieSavepointException(
|
||||||
|
"Could not get savepointed data files for savepoint " + savepointTime, e);
|
||||||
|
|
||||||
/**
|
|
||||||
* Provides a partitioner to perform the insert operation, based on the workload profile
|
|
||||||
*
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
public abstract Partitioner getInsertPartitioner(WorkloadProfile profile);
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Return whether this HoodieTable implementation can benefit from workload
|
|
||||||
* profiling
|
|
||||||
*
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
public abstract boolean isWorkloadProfileNeeded();
|
|
||||||
|
|
||||||
public HoodieWriteConfig getConfig() {
|
|
||||||
return config;
|
|
||||||
}
|
}
|
||||||
|
return metadata.getPartitionMetadata().values().stream()
|
||||||
|
.flatMap(s -> s.getSavepointDataFile().stream());
|
||||||
|
}
|
||||||
|
|
||||||
public HoodieTableMetaClient getMetaClient() {
|
public HoodieActiveTimeline getActiveTimeline() {
|
||||||
return metaClient;
|
return metaClient.getActiveTimeline();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the commit timeline visible for this table
|
||||||
|
*/
|
||||||
|
public HoodieTimeline getCommitTimeline() {
|
||||||
|
switch (metaClient.getTableType()) {
|
||||||
|
case COPY_ON_WRITE:
|
||||||
|
return getActiveTimeline().getCommitTimeline();
|
||||||
|
case MERGE_ON_READ:
|
||||||
|
// We need to include the parquet files written out in delta commits
|
||||||
|
// Include commit action to be able to start doing a MOR over a COW dataset - no migration required
|
||||||
|
return getActiveTimeline().getCommitsAndCompactionsTimeline();
|
||||||
|
default:
|
||||||
|
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public FileSystem getFs() {
|
/**
|
||||||
return metaClient.getFs();
|
* Get only the completed (no-inflights) compaction commit timeline
|
||||||
|
*/
|
||||||
|
public HoodieTimeline getCompletedCompactionCommitTimeline() {
|
||||||
|
return getCompactionCommitTimeline().filterCompletedInstants();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the compacted commit timeline visible for this table
|
||||||
|
*/
|
||||||
|
public HoodieTimeline getCompactionCommitTimeline() {
|
||||||
|
switch (metaClient.getTableType()) {
|
||||||
|
case COPY_ON_WRITE:
|
||||||
|
return getActiveTimeline().getCommitsAndCompactionsTimeline();
|
||||||
|
case MERGE_ON_READ:
|
||||||
|
// We need to include the parquet files written out in delta commits in tagging
|
||||||
|
return getActiveTimeline().getTimelineOfActions(
|
||||||
|
Sets.newHashSet(HoodieActiveTimeline.COMPACTION_ACTION));
|
||||||
|
default:
|
||||||
|
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the view of the file system for this table
|
* Gets the commit action type
|
||||||
*
|
*/
|
||||||
* @return
|
public String getCommitActionType() {
|
||||||
*/
|
switch (metaClient.getTableType()) {
|
||||||
public TableFileSystemView getFileSystemView() {
|
case COPY_ON_WRITE:
|
||||||
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
|
return HoodieActiveTimeline.COMMIT_ACTION;
|
||||||
|
case MERGE_ON_READ:
|
||||||
|
return HoodieActiveTimeline.DELTA_COMMIT_ACTION;
|
||||||
}
|
}
|
||||||
|
throw new HoodieCommitException(
|
||||||
|
"Could not commit on unknown storage type " + metaClient.getTableType());
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the read optimized view of the file system for this table
|
* Gets the action type for a compaction commit
|
||||||
*
|
*/
|
||||||
* @return
|
public String getCompactedCommitActionType() {
|
||||||
*/
|
switch (metaClient.getTableType()) {
|
||||||
public TableFileSystemView.ReadOptimizedView getROFileSystemView() {
|
case COPY_ON_WRITE:
|
||||||
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
|
return HoodieTimeline.COMMIT_ACTION;
|
||||||
|
case MERGE_ON_READ:
|
||||||
|
return HoodieTimeline.COMPACTION_ACTION;
|
||||||
}
|
}
|
||||||
|
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the real time view of the file system for this table
|
/**
|
||||||
*
|
* Perform the ultimate IO for a given upserted (RDD) partition
|
||||||
* @return
|
*/
|
||||||
*/
|
public abstract Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime,
|
||||||
public TableFileSystemView.RealtimeView getRTFileSystemView() {
|
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
||||||
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
|
|
||||||
|
/**
|
||||||
|
* Perform the ultimate IO for a given inserted (RDD) partition
|
||||||
|
*/
|
||||||
|
public abstract Iterator<List<WriteStatus>> handleInsertPartition(String commitTime,
|
||||||
|
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
||||||
|
|
||||||
|
|
||||||
|
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
|
||||||
|
HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||||
|
switch (metaClient.getTableType()) {
|
||||||
|
case COPY_ON_WRITE:
|
||||||
|
return new HoodieCopyOnWriteTable<>(config, metaClient);
|
||||||
|
case MERGE_ON_READ:
|
||||||
|
return new HoodieMergeOnReadTable<>(config, metaClient);
|
||||||
|
default:
|
||||||
|
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the completed (commit + compaction) view of the file system for this table
|
* Run Compaction on the table. Compaction arranges the data so that it is optimized for data
|
||||||
*
|
* access
|
||||||
* @return
|
*/
|
||||||
*/
|
public abstract Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc);
|
||||||
public TableFileSystemView getCompletedFileSystemView() {
|
|
||||||
return new HoodieTableFileSystemView(metaClient, getCommitTimeline());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get only the completed (no-inflights) commit timeline
|
* Clean partition paths according to cleaning policy and returns the number of files cleaned.
|
||||||
* @return
|
*/
|
||||||
*/
|
public abstract List<HoodieCleanStat> clean(JavaSparkContext jsc);
|
||||||
public HoodieTimeline getCompletedCommitTimeline() {
|
|
||||||
return getCommitTimeline().filterCompletedInstants();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get only the inflights (no-completed) commit timeline
|
* Rollback the (inflight/committed) record changes with the given commit time. Four steps: (1)
|
||||||
* @return
|
* Atomically unpublish this commit (2) clean indexing data (3) clean new generated parquet files
|
||||||
*/
|
* / log blocks (4) Finally, delete .<action>.commit or .<action>.inflight file
|
||||||
public HoodieTimeline getInflightCommitTimeline() {
|
*/
|
||||||
return getCommitTimeline().filterInflights();
|
public abstract List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits)
|
||||||
}
|
throws IOException;
|
||||||
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get only the completed (no-inflights) clean timeline
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
public HoodieTimeline getCompletedCleanTimeline() {
|
|
||||||
return getActiveTimeline().getCleanerTimeline().filterCompletedInstants();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get only the completed (no-inflights) savepoint timeline
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
public HoodieTimeline getCompletedSavepointTimeline() {
|
|
||||||
return getActiveTimeline().getSavePointTimeline().filterCompletedInstants();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the list of savepoints in this table
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
public List<String> getSavepoints() {
|
|
||||||
return getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp)
|
|
||||||
.collect(Collectors.toList());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the list of data file names savepointed
|
|
||||||
*
|
|
||||||
* @param savepointTime
|
|
||||||
* @return
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
public Stream<String> getSavepointedDataFiles(String savepointTime) {
|
|
||||||
if (!getSavepoints().contains(savepointTime)) {
|
|
||||||
throw new HoodieSavepointException(
|
|
||||||
"Could not get data files for savepoint " + savepointTime + ". No such savepoint.");
|
|
||||||
}
|
|
||||||
HoodieInstant instant =
|
|
||||||
new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime);
|
|
||||||
HoodieSavepointMetadata metadata = null;
|
|
||||||
try {
|
|
||||||
metadata = AvroUtils.deserializeHoodieSavepointMetadata(
|
|
||||||
getActiveTimeline().getInstantDetails(instant).get());
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieSavepointException(
|
|
||||||
"Could not get savepointed data files for savepoint " + savepointTime, e);
|
|
||||||
}
|
|
||||||
return metadata.getPartitionMetadata().values().stream()
|
|
||||||
.flatMap(s -> s.getSavepointDataFile().stream());
|
|
||||||
}
|
|
||||||
|
|
||||||
public HoodieActiveTimeline getActiveTimeline() {
|
|
||||||
return metaClient.getActiveTimeline();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the commit timeline visible for this table
|
|
||||||
*
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
public HoodieTimeline getCommitTimeline() {
|
|
||||||
switch (metaClient.getTableType()) {
|
|
||||||
case COPY_ON_WRITE:
|
|
||||||
return getActiveTimeline().getCommitTimeline();
|
|
||||||
case MERGE_ON_READ:
|
|
||||||
// We need to include the parquet files written out in delta commits
|
|
||||||
// Include commit action to be able to start doing a MOR over a COW dataset - no migration required
|
|
||||||
return getActiveTimeline().getCommitsAndCompactionsTimeline();
|
|
||||||
default:
|
|
||||||
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get only the completed (no-inflights) compaction commit timeline
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
public HoodieTimeline getCompletedCompactionCommitTimeline() {
|
|
||||||
return getCompactionCommitTimeline().filterCompletedInstants();
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the compacted commit timeline visible for this table
|
|
||||||
*
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
public HoodieTimeline getCompactionCommitTimeline() {
|
|
||||||
switch (metaClient.getTableType()) {
|
|
||||||
case COPY_ON_WRITE:
|
|
||||||
return getActiveTimeline().getCommitsAndCompactionsTimeline();
|
|
||||||
case MERGE_ON_READ:
|
|
||||||
// We need to include the parquet files written out in delta commits in tagging
|
|
||||||
return getActiveTimeline().getTimelineOfActions(
|
|
||||||
Sets.newHashSet(HoodieActiveTimeline.COMPACTION_ACTION));
|
|
||||||
default:
|
|
||||||
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Gets the commit action type
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
public String getCommitActionType() {
|
|
||||||
switch (metaClient.getTableType()) {
|
|
||||||
case COPY_ON_WRITE:
|
|
||||||
return HoodieActiveTimeline.COMMIT_ACTION;
|
|
||||||
case MERGE_ON_READ:
|
|
||||||
return HoodieActiveTimeline.DELTA_COMMIT_ACTION;
|
|
||||||
}
|
|
||||||
throw new HoodieCommitException(
|
|
||||||
"Could not commit on unknown storage type " + metaClient.getTableType());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Gets the action type for a compaction commit
|
|
||||||
* @return
|
|
||||||
*/
|
|
||||||
public String getCompactedCommitActionType() {
|
|
||||||
switch (metaClient.getTableType()) {
|
|
||||||
case COPY_ON_WRITE:
|
|
||||||
return HoodieTimeline.COMMIT_ACTION;
|
|
||||||
case MERGE_ON_READ:
|
|
||||||
return HoodieTimeline.COMPACTION_ACTION;
|
|
||||||
}
|
|
||||||
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Perform the ultimate IO for a given upserted (RDD) partition
|
|
||||||
*
|
|
||||||
* @param partition
|
|
||||||
* @param recordIterator
|
|
||||||
* @param partitioner
|
|
||||||
*/
|
|
||||||
public abstract Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime,
|
|
||||||
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Perform the ultimate IO for a given inserted (RDD) partition
|
|
||||||
*
|
|
||||||
* @param partition
|
|
||||||
* @param recordIterator
|
|
||||||
* @param partitioner
|
|
||||||
*/
|
|
||||||
public abstract Iterator<List<WriteStatus>> handleInsertPartition(String commitTime,
|
|
||||||
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
|
||||||
|
|
||||||
|
|
||||||
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
|
|
||||||
HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
|
||||||
switch (metaClient.getTableType()) {
|
|
||||||
case COPY_ON_WRITE:
|
|
||||||
return new HoodieCopyOnWriteTable<>(config, metaClient);
|
|
||||||
case MERGE_ON_READ:
|
|
||||||
return new HoodieMergeOnReadTable<>(config, metaClient);
|
|
||||||
default:
|
|
||||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Run Compaction on the table.
|
|
||||||
* Compaction arranges the data so that it is optimized for data access
|
|
||||||
*/
|
|
||||||
public abstract Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc);
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Clean partition paths according to cleaning policy and returns the number
|
|
||||||
* of files cleaned.
|
|
||||||
*/
|
|
||||||
public abstract List<HoodieCleanStat> clean(JavaSparkContext jsc);
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Rollback the (inflight/committed) record changes with the given commit time.
|
|
||||||
* Four steps:
|
|
||||||
* (1) Atomically unpublish this commit
|
|
||||||
* (2) clean indexing data
|
|
||||||
* (3) clean new generated parquet files / log blocks
|
|
||||||
* (4) Finally, delete .<action>.commit or .<action>.inflight file
|
|
||||||
* @param commits
|
|
||||||
* @return
|
|
||||||
* @throws HoodieRollbackException
|
|
||||||
*/
|
|
||||||
public abstract List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits) throws IOException;
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -20,13 +20,13 @@ import com.uber.hoodie.common.model.HoodieRecordPayload;
|
|||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Repartition input records into at least expected number of output spark partitions. It should give
|
* Repartition input records into at least expected number of output spark partitions. It should
|
||||||
* below guarantees
|
* give below guarantees - Output spark partition will have records from only one hoodie partition.
|
||||||
* - Output spark partition will have records from only one hoodie partition.
|
* - Average records per output spark partitions should be almost equal to (#inputRecords /
|
||||||
* - Average records per output spark partitions should be almost equal to (#inputRecords / #outputSparkPartitions)
|
* #outputSparkPartitions) to avoid possible skews.
|
||||||
* to avoid possible skews.
|
|
||||||
*/
|
*/
|
||||||
public interface UserDefinedBulkInsertPartitioner<T extends HoodieRecordPayload> {
|
public interface UserDefinedBulkInsertPartitioner<T extends HoodieRecordPayload> {
|
||||||
|
|
||||||
JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions);
|
JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records,
|
||||||
|
int outputSparkPartitions);
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -20,15 +20,11 @@ package com.uber.hoodie.table;
|
|||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||||
|
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
|
||||||
import org.apache.spark.api.java.function.PairFunction;
|
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import scala.Option;
|
import scala.Option;
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
@@ -40,73 +36,76 @@ import scala.Tuple2;
|
|||||||
*/
|
*/
|
||||||
public class WorkloadProfile<T extends HoodieRecordPayload> implements Serializable {
|
public class WorkloadProfile<T extends HoodieRecordPayload> implements Serializable {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Input workload
|
* Input workload
|
||||||
*/
|
*/
|
||||||
private final JavaRDD<HoodieRecord<T>> taggedRecords;
|
private final JavaRDD<HoodieRecord<T>> taggedRecords;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Computed workload profile
|
* Computed workload profile
|
||||||
*/
|
*/
|
||||||
private final HashMap<String, WorkloadStat> partitionPathStatMap;
|
private final HashMap<String, WorkloadStat> partitionPathStatMap;
|
||||||
|
|
||||||
|
|
||||||
private final WorkloadStat globalStat;
|
private final WorkloadStat globalStat;
|
||||||
|
|
||||||
|
|
||||||
public WorkloadProfile(JavaRDD<HoodieRecord<T>> taggedRecords) {
|
public WorkloadProfile(JavaRDD<HoodieRecord<T>> taggedRecords) {
|
||||||
this.taggedRecords = taggedRecords;
|
this.taggedRecords = taggedRecords;
|
||||||
this.partitionPathStatMap = new HashMap<>();
|
this.partitionPathStatMap = new HashMap<>();
|
||||||
this.globalStat = new WorkloadStat();
|
this.globalStat = new WorkloadStat();
|
||||||
buildProfile();
|
buildProfile();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void buildProfile() {
|
||||||
|
|
||||||
|
Map<Tuple2<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = taggedRecords
|
||||||
|
.mapToPair(record ->
|
||||||
|
new Tuple2<>(
|
||||||
|
new Tuple2<>(record.getPartitionPath(), Option.apply(record.getCurrentLocation())),
|
||||||
|
record))
|
||||||
|
.countByKey();
|
||||||
|
|
||||||
|
for (Map.Entry<Tuple2<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts
|
||||||
|
.entrySet()) {
|
||||||
|
String partitionPath = e.getKey()._1();
|
||||||
|
Long count = e.getValue();
|
||||||
|
Option<HoodieRecordLocation> locOption = e.getKey()._2();
|
||||||
|
|
||||||
|
if (!partitionPathStatMap.containsKey(partitionPath)) {
|
||||||
|
partitionPathStatMap.put(partitionPath, new WorkloadStat());
|
||||||
|
}
|
||||||
|
|
||||||
|
if (locOption.isDefined()) {
|
||||||
|
// update
|
||||||
|
partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count);
|
||||||
|
globalStat.addUpdates(locOption.get(), count);
|
||||||
|
} else {
|
||||||
|
// insert
|
||||||
|
partitionPathStatMap.get(partitionPath).addInserts(count);
|
||||||
|
globalStat.addInserts(count);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private void buildProfile() {
|
public WorkloadStat getGlobalStat() {
|
||||||
|
return globalStat;
|
||||||
|
}
|
||||||
|
|
||||||
Map<Tuple2<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = taggedRecords
|
public Set<String> getPartitionPaths() {
|
||||||
.mapToPair(record ->
|
return partitionPathStatMap.keySet();
|
||||||
new Tuple2<>(new Tuple2<>(record.getPartitionPath(), Option.apply(record.getCurrentLocation())), record))
|
}
|
||||||
.countByKey();
|
|
||||||
|
|
||||||
for (Map.Entry<Tuple2<String, Option<HoodieRecordLocation>>, Long> e: partitionLocationCounts.entrySet()) {
|
public WorkloadStat getWorkloadStat(String partitionPath) {
|
||||||
String partitionPath = e.getKey()._1();
|
return partitionPathStatMap.get(partitionPath);
|
||||||
Long count = e.getValue();
|
}
|
||||||
Option<HoodieRecordLocation> locOption = e.getKey()._2();
|
|
||||||
|
|
||||||
if (!partitionPathStatMap.containsKey(partitionPath)){
|
@Override
|
||||||
partitionPathStatMap.put(partitionPath, new WorkloadStat());
|
public String toString() {
|
||||||
}
|
final StringBuilder sb = new StringBuilder("WorkloadProfile {");
|
||||||
|
sb.append("globalStat=").append(globalStat).append(", ");
|
||||||
if (locOption.isDefined()) {
|
sb.append("partitionStat=").append(partitionPathStatMap);
|
||||||
// update
|
sb.append('}');
|
||||||
partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count);
|
return sb.toString();
|
||||||
globalStat.addUpdates(locOption.get(), count);
|
}
|
||||||
} else {
|
|
||||||
// insert
|
|
||||||
partitionPathStatMap.get(partitionPath).addInserts(count);
|
|
||||||
globalStat.addInserts(count);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public WorkloadStat getGlobalStat() {
|
|
||||||
return globalStat;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Set<String> getPartitionPaths() {
|
|
||||||
return partitionPathStatMap.keySet();
|
|
||||||
}
|
|
||||||
|
|
||||||
public WorkloadStat getWorkloadStat(String partitionPath){
|
|
||||||
return partitionPathStatMap.get(partitionPath);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
final StringBuilder sb = new StringBuilder("WorkloadProfile {");
|
|
||||||
sb.append("globalStat=").append(globalStat).append(", ");
|
|
||||||
sb.append("partitionStat=").append(partitionPathStatMap);
|
|
||||||
sb.append('}');
|
|
||||||
return sb.toString();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,7 +17,6 @@
|
|||||||
package com.uber.hoodie.table;
|
package com.uber.hoodie.table;
|
||||||
|
|
||||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||||
|
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
|
||||||
@@ -25,43 +24,44 @@ import java.util.HashMap;
|
|||||||
* Wraps stats about a single partition path.
|
* Wraps stats about a single partition path.
|
||||||
*/
|
*/
|
||||||
public class WorkloadStat implements Serializable {
|
public class WorkloadStat implements Serializable {
|
||||||
private long numInserts = 0L;
|
|
||||||
|
|
||||||
private long numUpdates = 0L;
|
private long numInserts = 0L;
|
||||||
|
|
||||||
private HashMap<String, Long> updateLocationToCount;
|
private long numUpdates = 0L;
|
||||||
|
|
||||||
public WorkloadStat() {
|
private HashMap<String, Long> updateLocationToCount;
|
||||||
updateLocationToCount = new HashMap<>();
|
|
||||||
}
|
|
||||||
|
|
||||||
long addInserts(long numInserts) {
|
public WorkloadStat() {
|
||||||
return this.numInserts += numInserts;
|
updateLocationToCount = new HashMap<>();
|
||||||
}
|
}
|
||||||
|
|
||||||
long addUpdates(HoodieRecordLocation location, long numUpdates) {
|
long addInserts(long numInserts) {
|
||||||
updateLocationToCount.put(location.getFileId(), numUpdates);
|
return this.numInserts += numInserts;
|
||||||
return this.numUpdates += numUpdates;
|
}
|
||||||
}
|
|
||||||
|
|
||||||
public long getNumUpdates() {
|
long addUpdates(HoodieRecordLocation location, long numUpdates) {
|
||||||
return numUpdates;
|
updateLocationToCount.put(location.getFileId(), numUpdates);
|
||||||
}
|
return this.numUpdates += numUpdates;
|
||||||
|
}
|
||||||
|
|
||||||
public long getNumInserts() {
|
public long getNumUpdates() {
|
||||||
return numInserts;
|
return numUpdates;
|
||||||
}
|
}
|
||||||
|
|
||||||
public HashMap<String, Long> getUpdateLocationToCount() {
|
public long getNumInserts() {
|
||||||
return updateLocationToCount;
|
return numInserts;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
public HashMap<String, Long> getUpdateLocationToCount() {
|
||||||
public String toString() {
|
return updateLocationToCount;
|
||||||
final StringBuilder sb = new StringBuilder("WorkloadStat {");
|
}
|
||||||
sb.append("numInserts=").append(numInserts).append(", ");
|
|
||||||
sb.append("numUpdates=").append(numUpdates);
|
@Override
|
||||||
sb.append('}');
|
public String toString() {
|
||||||
return sb.toString();
|
final StringBuilder sb = new StringBuilder("WorkloadStat {");
|
||||||
}
|
sb.append("numInserts=").append(numInserts).append(", ");
|
||||||
|
sb.append("numUpdates=").append(numUpdates);
|
||||||
|
sb.append('}');
|
||||||
|
return sb.toString();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -13,7 +13,6 @@
|
|||||||
# See the License for the specific language governing permissions and
|
# See the License for the specific language governing permissions and
|
||||||
# limitations under the License.
|
# limitations under the License.
|
||||||
#
|
#
|
||||||
|
|
||||||
# Set root logger level to DEBUG and its only appender to A1.
|
# Set root logger level to DEBUG and its only appender to A1.
|
||||||
log4j.rootLogger=INFO, A1
|
log4j.rootLogger=INFO, A1
|
||||||
# A1 is set to be a ConsoleAppender.
|
# A1 is set to be a ConsoleAppender.
|
||||||
|
|||||||
@@ -22,13 +22,12 @@ import com.uber.hoodie.common.HoodieTestDataGenerator;
|
|||||||
import com.uber.hoodie.common.model.HoodieAvroPayload;
|
import com.uber.hoodie.common.model.HoodieAvroPayload;
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
import com.uber.hoodie.common.model.HoodieTableType;
|
import com.uber.hoodie.common.model.HoodieTableType;
|
||||||
import com.uber.hoodie.common.table.HoodieTableConfig;
|
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.index.HoodieIndex;
|
import com.uber.hoodie.index.HoodieIndex;
|
||||||
|
import java.util.List;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
@@ -36,7 +35,6 @@ import org.apache.log4j.Logger;
|
|||||||
import org.apache.spark.SparkConf;
|
import org.apache.spark.SparkConf;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Driver program that uses the Hoodie client with synthetic workload, and performs basic
|
* Driver program that uses the Hoodie client with synthetic workload, and performs basic
|
||||||
@@ -44,75 +42,77 @@ import java.util.List;
|
|||||||
*/
|
*/
|
||||||
public class HoodieClientExample {
|
public class HoodieClientExample {
|
||||||
|
|
||||||
@Parameter(names={"--table-path", "-p"}, description = "path for Hoodie sample table")
|
@Parameter(names = {"--table-path", "-p"}, description = "path for Hoodie sample table")
|
||||||
private String tablePath = "file:///tmp/hoodie/sample-table";
|
private String tablePath = "file:///tmp/hoodie/sample-table";
|
||||||
|
|
||||||
@Parameter(names={"--table-name", "-n"}, description = "table name for Hoodie sample table")
|
@Parameter(names = {"--table-name", "-n"}, description = "table name for Hoodie sample table")
|
||||||
private String tableName = "hoodie_rt";
|
private String tableName = "hoodie_rt";
|
||||||
|
|
||||||
@Parameter(names={"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ")
|
@Parameter(names = {"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ")
|
||||||
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||||
|
|
||||||
@Parameter(names = {"--help", "-h"}, help = true)
|
@Parameter(names = {"--help", "-h"}, help = true)
|
||||||
public Boolean help = false;
|
public Boolean help = false;
|
||||||
|
|
||||||
private static Logger logger = LogManager.getLogger(HoodieClientExample.class);
|
private static Logger logger = LogManager.getLogger(HoodieClientExample.class);
|
||||||
|
|
||||||
public static void main(String[] args) throws Exception {
|
public static void main(String[] args) throws Exception {
|
||||||
HoodieClientExample cli = new HoodieClientExample();
|
HoodieClientExample cli = new HoodieClientExample();
|
||||||
JCommander cmd = new JCommander(cli, args);
|
JCommander cmd = new JCommander(cli, args);
|
||||||
|
|
||||||
if (cli.help) {
|
if (cli.help) {
|
||||||
cmd.usage();
|
cmd.usage();
|
||||||
System.exit(1);
|
System.exit(1);
|
||||||
}
|
}
|
||||||
cli.run();
|
cli.run();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
public void run() throws Exception {
|
||||||
|
|
||||||
|
SparkConf sparkConf = new SparkConf().setAppName("hoodie-client-example");
|
||||||
|
sparkConf.setMaster("local[1]");
|
||||||
|
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||||
|
sparkConf.set("spark.kryoserializer.buffer.max", "512m");
|
||||||
|
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
|
||||||
|
|
||||||
|
// Generator of some records to be loaded in.
|
||||||
|
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
||||||
|
|
||||||
|
// initialize the table, if not done already
|
||||||
|
Path path = new Path(tablePath);
|
||||||
|
FileSystem fs = FSUtils.getFs();
|
||||||
|
if (!fs.exists(path)) {
|
||||||
|
HoodieTableMetaClient
|
||||||
|
.initTableType(fs, tablePath, HoodieTableType.valueOf(tableType), tableName,
|
||||||
|
HoodieAvroPayload.class.getName());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Create the write client to write some records in
|
||||||
|
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
|
||||||
|
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||||
|
.forTable(tableName).withIndexConfig(
|
||||||
|
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||||
|
.build();
|
||||||
|
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
||||||
|
|
||||||
public void run() throws Exception {
|
/**
|
||||||
|
* Write 1 (only inserts)
|
||||||
|
*/
|
||||||
|
String newCommitTime = client.startCommit();
|
||||||
|
logger.info("Starting commit " + newCommitTime);
|
||||||
|
|
||||||
SparkConf sparkConf = new SparkConf().setAppName("hoodie-client-example");
|
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||||
sparkConf.setMaster("local[1]");
|
JavaRDD<HoodieRecord> writeRecords = jsc.<HoodieRecord>parallelize(records, 1);
|
||||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
client.upsert(writeRecords, newCommitTime);
|
||||||
sparkConf.set("spark.kryoserializer.buffer.max", "512m");
|
|
||||||
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
|
|
||||||
|
|
||||||
// Generator of some records to be loaded in.
|
/**
|
||||||
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
|
* Write 2 (updates)
|
||||||
|
*/
|
||||||
// initialize the table, if not done already
|
newCommitTime = client.startCommit();
|
||||||
Path path = new Path(tablePath);
|
logger.info("Starting commit " + newCommitTime);
|
||||||
FileSystem fs = FSUtils.getFs();
|
records.addAll(dataGen.generateUpdates(newCommitTime, 100));
|
||||||
if (!fs.exists(path)) {
|
writeRecords = jsc.<HoodieRecord>parallelize(records, 1);
|
||||||
HoodieTableMetaClient.initTableType(fs, tablePath, HoodieTableType.valueOf(tableType), tableName, HoodieAvroPayload.class.getName());
|
client.upsert(writeRecords, newCommitTime);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Create the write client to write some records in
|
|
||||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
|
|
||||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
|
||||||
.forTable(tableName).withIndexConfig(
|
|
||||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
|
||||||
.build();
|
|
||||||
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Write 1 (only inserts)
|
|
||||||
*/
|
|
||||||
String newCommitTime = client.startCommit();
|
|
||||||
logger.info("Starting commit " + newCommitTime);
|
|
||||||
|
|
||||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
|
||||||
JavaRDD<HoodieRecord> writeRecords = jsc.<HoodieRecord>parallelize(records, 1);
|
|
||||||
client.upsert(writeRecords, newCommitTime);
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Write 2 (updates)
|
|
||||||
*/
|
|
||||||
newCommitTime = client.startCommit();
|
|
||||||
logger.info("Starting commit " + newCommitTime);
|
|
||||||
records.addAll(dataGen.generateUpdates(newCommitTime, 100));
|
|
||||||
writeRecords = jsc.<HoodieRecord>parallelize(records, 1);
|
|
||||||
client.upsert(writeRecords, newCommitTime);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|||||||
File diff suppressed because it is too large
Load Diff
@@ -29,15 +29,6 @@ import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
|
|||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
import com.uber.hoodie.exception.HoodieException;
|
import com.uber.hoodie.exception.HoodieException;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
|
||||||
import org.apache.spark.sql.Dataset;
|
|
||||||
import org.apache.spark.sql.Row;
|
|
||||||
import org.apache.spark.sql.SQLContext;
|
|
||||||
|
|
||||||
import org.apache.spark.SparkConf;
|
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.RandomAccessFile;
|
import java.io.RandomAccessFile;
|
||||||
@@ -49,6 +40,12 @@ import java.util.Iterator;
|
|||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.spark.SparkConf;
|
||||||
|
import org.apache.spark.sql.Dataset;
|
||||||
|
import org.apache.spark.sql.Row;
|
||||||
|
import org.apache.spark.sql.SQLContext;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Utility methods to aid testing inside the HoodieClient module.
|
* Utility methods to aid testing inside the HoodieClient module.
|
||||||
@@ -56,133 +53,142 @@ import java.util.stream.Collectors;
|
|||||||
public class HoodieClientTestUtils {
|
public class HoodieClientTestUtils {
|
||||||
|
|
||||||
|
|
||||||
public static List<WriteStatus> collectStatuses(Iterator<List<WriteStatus>> statusListItr) {
|
public static List<WriteStatus> collectStatuses(Iterator<List<WriteStatus>> statusListItr) {
|
||||||
List<WriteStatus> statuses = new ArrayList<>();
|
List<WriteStatus> statuses = new ArrayList<>();
|
||||||
while (statusListItr.hasNext()) {
|
while (statusListItr.hasNext()) {
|
||||||
statuses.addAll(statusListItr.next());
|
statuses.addAll(statusListItr.next());
|
||||||
}
|
}
|
||||||
return statuses;
|
return statuses;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Set<String> getRecordKeys(List<HoodieRecord> hoodieRecords) {
|
public static Set<String> getRecordKeys(List<HoodieRecord> hoodieRecords) {
|
||||||
Set<String> keys = new HashSet<>();
|
Set<String> keys = new HashSet<>();
|
||||||
for (HoodieRecord rec: hoodieRecords) {
|
for (HoodieRecord rec : hoodieRecords) {
|
||||||
keys.add(rec.getRecordKey());
|
keys.add(rec.getRecordKey());
|
||||||
}
|
}
|
||||||
return keys;
|
return keys;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void fakeMetaFile(String basePath, String commitTime, String suffix) throws IOException {
|
private static void fakeMetaFile(String basePath, String commitTime, String suffix)
|
||||||
String parentPath = basePath + "/"+ HoodieTableMetaClient.METAFOLDER_NAME;
|
throws IOException {
|
||||||
new File(parentPath).mkdirs();
|
String parentPath = basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME;
|
||||||
new File(parentPath + "/" + commitTime + suffix).createNewFile();
|
new File(parentPath).mkdirs();
|
||||||
}
|
new File(parentPath + "/" + commitTime + suffix).createNewFile();
|
||||||
|
}
|
||||||
|
|
||||||
public static void fakeCommitFile(String basePath, String commitTime) throws IOException {
|
|
||||||
fakeMetaFile(basePath, commitTime, HoodieTimeline.COMMIT_EXTENSION);
|
public static void fakeCommitFile(String basePath, String commitTime) throws IOException {
|
||||||
}
|
fakeMetaFile(basePath, commitTime, HoodieTimeline.COMMIT_EXTENSION);
|
||||||
|
}
|
||||||
public static void fakeInFlightFile(String basePath, String commitTime) throws IOException {
|
|
||||||
fakeMetaFile(basePath, commitTime, HoodieTimeline.INFLIGHT_EXTENSION);
|
public static void fakeInFlightFile(String basePath, String commitTime) throws IOException {
|
||||||
}
|
fakeMetaFile(basePath, commitTime, HoodieTimeline.INFLIGHT_EXTENSION);
|
||||||
|
}
|
||||||
public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId) throws Exception {
|
|
||||||
fakeDataFile(basePath, partitionPath, commitTime, fileId, 0);
|
public static void fakeDataFile(String basePath, String partitionPath, String commitTime,
|
||||||
}
|
String fileId) throws Exception {
|
||||||
|
fakeDataFile(basePath, partitionPath, commitTime, fileId, 0);
|
||||||
public static void fakeDataFile(String basePath, String partitionPath, String commitTime, String fileId, long length) throws Exception {
|
}
|
||||||
String parentPath = String.format("%s/%s", basePath, partitionPath);
|
|
||||||
new File(parentPath).mkdirs();
|
public static void fakeDataFile(String basePath, String partitionPath, String commitTime,
|
||||||
String path = String.format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, 0, fileId));
|
String fileId, long length) throws Exception {
|
||||||
new File(path).createNewFile();
|
String parentPath = String.format("%s/%s", basePath, partitionPath);
|
||||||
new RandomAccessFile(path, "rw").setLength(length);
|
new File(parentPath).mkdirs();
|
||||||
}
|
String path = String
|
||||||
|
.format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, 0, fileId));
|
||||||
public static SparkConf getSparkConfForTest(String appName) {
|
new File(path).createNewFile();
|
||||||
SparkConf sparkConf = new SparkConf()
|
new RandomAccessFile(path, "rw").setLength(length);
|
||||||
.setAppName(appName)
|
}
|
||||||
.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
|
||||||
.setMaster("local[1]");
|
public static SparkConf getSparkConfForTest(String appName) {
|
||||||
return HoodieReadClient.addHoodieSupport(sparkConf);
|
SparkConf sparkConf = new SparkConf()
|
||||||
}
|
.setAppName(appName)
|
||||||
|
.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
|
||||||
public static HashMap<String, String> getLatestFileIDsToFullPath(String basePath,
|
.setMaster("local[1]");
|
||||||
HoodieTimeline commitTimeline,
|
return HoodieReadClient.addHoodieSupport(sparkConf);
|
||||||
List<HoodieInstant> commitsToReturn) throws IOException {
|
}
|
||||||
HashMap<String, String> fileIdToFullPath = new HashMap<>();
|
|
||||||
for (HoodieInstant commit : commitsToReturn) {
|
public static HashMap<String, String> getLatestFileIDsToFullPath(String basePath,
|
||||||
HoodieCommitMetadata metadata =
|
HoodieTimeline commitTimeline,
|
||||||
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get());
|
List<HoodieInstant> commitsToReturn) throws IOException {
|
||||||
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath));
|
HashMap<String, String> fileIdToFullPath = new HashMap<>();
|
||||||
}
|
for (HoodieInstant commit : commitsToReturn) {
|
||||||
return fileIdToFullPath;
|
HoodieCommitMetadata metadata =
|
||||||
}
|
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get());
|
||||||
|
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath));
|
||||||
public static Dataset<Row> readCommit(String basePath,
|
}
|
||||||
SQLContext sqlContext,
|
return fileIdToFullPath;
|
||||||
HoodieTimeline commitTimeline,
|
}
|
||||||
String commitTime) {
|
|
||||||
HoodieInstant commitInstant =
|
public static Dataset<Row> readCommit(String basePath,
|
||||||
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
SQLContext sqlContext,
|
||||||
if (!commitTimeline.containsInstant(commitInstant)) {
|
HoodieTimeline commitTimeline,
|
||||||
new HoodieException("No commit exists at " + commitTime);
|
String commitTime) {
|
||||||
}
|
HoodieInstant commitInstant =
|
||||||
try {
|
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||||
HashMap<String, String> paths = getLatestFileIDsToFullPath(basePath, commitTimeline, Arrays.asList(commitInstant));
|
if (!commitTimeline.containsInstant(commitInstant)) {
|
||||||
return sqlContext.read()
|
new HoodieException("No commit exists at " + commitTime);
|
||||||
.parquet(paths.values().toArray(new String[paths.size()]))
|
}
|
||||||
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime));
|
try {
|
||||||
} catch (Exception e) {
|
HashMap<String, String> paths = getLatestFileIDsToFullPath(basePath, commitTimeline,
|
||||||
throw new HoodieException("Error reading commit " + commitTime, e);
|
Arrays.asList(commitInstant));
|
||||||
}
|
return sqlContext.read()
|
||||||
}
|
.parquet(paths.values().toArray(new String[paths.size()]))
|
||||||
|
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime));
|
||||||
/**
|
} catch (Exception e) {
|
||||||
* Obtain all new data written into the Hoodie dataset since the given timestamp.
|
throw new HoodieException("Error reading commit " + commitTime, e);
|
||||||
*/
|
}
|
||||||
public static Dataset<Row> readSince(String basePath,
|
}
|
||||||
SQLContext sqlContext,
|
|
||||||
HoodieTimeline commitTimeline,
|
/**
|
||||||
String lastCommitTime) {
|
* Obtain all new data written into the Hoodie dataset since the given timestamp.
|
||||||
List<HoodieInstant> commitsToReturn =
|
*/
|
||||||
commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE)
|
public static Dataset<Row> readSince(String basePath,
|
||||||
.getInstants().collect(Collectors.toList());
|
SQLContext sqlContext,
|
||||||
try {
|
HoodieTimeline commitTimeline,
|
||||||
// Go over the commit metadata, and obtain the new files that need to be read.
|
String lastCommitTime) {
|
||||||
HashMap<String, String> fileIdToFullPath = getLatestFileIDsToFullPath(basePath, commitTimeline, commitsToReturn);
|
List<HoodieInstant> commitsToReturn =
|
||||||
return sqlContext.read()
|
commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE)
|
||||||
.parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()]))
|
.getInstants().collect(Collectors.toList());
|
||||||
.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime));
|
try {
|
||||||
} catch (IOException e) {
|
// Go over the commit metadata, and obtain the new files that need to be read.
|
||||||
throw new HoodieException("Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e);
|
HashMap<String, String> fileIdToFullPath = getLatestFileIDsToFullPath(basePath,
|
||||||
}
|
commitTimeline, commitsToReturn);
|
||||||
}
|
return sqlContext.read()
|
||||||
|
.parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()]))
|
||||||
/**
|
.filter(
|
||||||
* Reads the paths under the a hoodie dataset out as a DataFrame
|
String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime));
|
||||||
*/
|
} catch (IOException e) {
|
||||||
public static Dataset<Row> read(String basePath,
|
throw new HoodieException(
|
||||||
SQLContext sqlContext,
|
"Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e);
|
||||||
FileSystem fs,
|
}
|
||||||
String... paths) {
|
}
|
||||||
List<String> filteredPaths = new ArrayList<>();
|
|
||||||
try {
|
/**
|
||||||
HoodieTable hoodieTable = HoodieTable
|
* Reads the paths under the a hoodie dataset out as a DataFrame
|
||||||
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
|
*/
|
||||||
for (String path : paths) {
|
public static Dataset<Row> read(String basePath,
|
||||||
TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(hoodieTable.getMetaClient(),
|
SQLContext sqlContext,
|
||||||
hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path)));
|
FileSystem fs,
|
||||||
List<HoodieDataFile> latestFiles = fileSystemView.getLatestDataFiles().collect(
|
String... paths) {
|
||||||
Collectors.toList());
|
List<String> filteredPaths = new ArrayList<>();
|
||||||
for (HoodieDataFile file : latestFiles) {
|
try {
|
||||||
filteredPaths.add(file.getPath());
|
HoodieTable hoodieTable = HoodieTable
|
||||||
}
|
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
|
||||||
}
|
for (String path : paths) {
|
||||||
return sqlContext.read()
|
TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(
|
||||||
.parquet(filteredPaths.toArray(new String[filteredPaths.size()]));
|
hoodieTable.getMetaClient(),
|
||||||
} catch (Exception e) {
|
hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path)));
|
||||||
throw new HoodieException("Error reading hoodie dataset as a dataframe", e);
|
List<HoodieDataFile> latestFiles = fileSystemView.getLatestDataFiles().collect(
|
||||||
|
Collectors.toList());
|
||||||
|
for (HoodieDataFile file : latestFiles) {
|
||||||
|
filteredPaths.add(file.getPath());
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
return sqlContext.read()
|
||||||
|
.parquet(filteredPaths.toArray(new String[filteredPaths.size()]));
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new HoodieException("Error reading hoodie dataset as a dataframe", e);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,9 +16,16 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.common;
|
package com.uber.hoodie.common;
|
||||||
|
|
||||||
|
import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||||
|
|
||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||||
import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat;
|
import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
import org.apache.avro.generic.GenericRecordBuilder;
|
import org.apache.avro.generic.GenericRecordBuilder;
|
||||||
@@ -30,69 +37,64 @@ import org.apache.hadoop.mapred.InputSplit;
|
|||||||
import org.apache.hadoop.mapred.JobConf;
|
import org.apache.hadoop.mapred.JobConf;
|
||||||
import org.apache.hadoop.mapred.RecordReader;
|
import org.apache.hadoop.mapred.RecordReader;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Utility methods to aid in testing MergeOnRead (workaround for HoodieReadClient for MOR)
|
* Utility methods to aid in testing MergeOnRead (workaround for HoodieReadClient for MOR)
|
||||||
*/
|
*/
|
||||||
public class HoodieMergeOnReadTestUtils {
|
public class HoodieMergeOnReadTestUtils {
|
||||||
|
|
||||||
public static List<GenericRecord> getRecordsUsingInputFormat(List<String> inputPaths) throws IOException {
|
public static List<GenericRecord> getRecordsUsingInputFormat(List<String> inputPaths)
|
||||||
JobConf jobConf = new JobConf();
|
throws IOException {
|
||||||
Schema schema = HoodieAvroUtils.addMetadataFields(Schema.parse(TRIP_EXAMPLE_SCHEMA));
|
JobConf jobConf = new JobConf();
|
||||||
HoodieRealtimeInputFormat inputFormat = new HoodieRealtimeInputFormat();
|
Schema schema = HoodieAvroUtils.addMetadataFields(Schema.parse(TRIP_EXAMPLE_SCHEMA));
|
||||||
setPropsForInputFormat(inputFormat, jobConf, schema);
|
HoodieRealtimeInputFormat inputFormat = new HoodieRealtimeInputFormat();
|
||||||
return inputPaths.stream().map(path -> {
|
setPropsForInputFormat(inputFormat, jobConf, schema);
|
||||||
setInputPath(jobConf, path);
|
return inputPaths.stream().map(path -> {
|
||||||
List<GenericRecord> records = new ArrayList<>();
|
setInputPath(jobConf, path);
|
||||||
try {
|
List<GenericRecord> records = new ArrayList<>();
|
||||||
List<InputSplit> splits = Arrays.asList(inputFormat.getSplits(jobConf, 1));
|
try {
|
||||||
RecordReader recordReader = inputFormat.getRecordReader(splits.get(0), jobConf, null);
|
List<InputSplit> splits = Arrays.asList(inputFormat.getSplits(jobConf, 1));
|
||||||
Void key = (Void) recordReader.createKey();
|
RecordReader recordReader = inputFormat.getRecordReader(splits.get(0), jobConf, null);
|
||||||
ArrayWritable writable = (ArrayWritable) recordReader.createValue();
|
Void key = (Void) recordReader.createKey();
|
||||||
while (recordReader.next(key, writable)) {
|
ArrayWritable writable = (ArrayWritable) recordReader.createValue();
|
||||||
GenericRecordBuilder newRecord = new GenericRecordBuilder(schema);
|
while (recordReader.next(key, writable)) {
|
||||||
// writable returns an array with [field1, field2, _hoodie_commit_time, _hoodie_commit_seqno]
|
GenericRecordBuilder newRecord = new GenericRecordBuilder(schema);
|
||||||
Writable[] values = writable.get();
|
// writable returns an array with [field1, field2, _hoodie_commit_time, _hoodie_commit_seqno]
|
||||||
schema.getFields().forEach(field -> {
|
Writable[] values = writable.get();
|
||||||
newRecord.set(field, values[2]);
|
schema.getFields().forEach(field -> {
|
||||||
});
|
newRecord.set(field, values[2]);
|
||||||
records.add(newRecord.build());
|
});
|
||||||
}
|
records.add(newRecord.build());
|
||||||
} catch (IOException ie) {
|
}
|
||||||
ie.printStackTrace();
|
} catch (IOException ie) {
|
||||||
}
|
ie.printStackTrace();
|
||||||
return records;
|
}
|
||||||
}).reduce((a, b) -> {
|
return records;
|
||||||
a.addAll(b);
|
}).reduce((a, b) -> {
|
||||||
return a;
|
a.addAll(b);
|
||||||
}).get();
|
return a;
|
||||||
}
|
}).get();
|
||||||
|
}
|
||||||
|
|
||||||
private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf, Schema schema) {
|
private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf,
|
||||||
List<Schema.Field> fields = schema.getFields();
|
Schema schema) {
|
||||||
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
|
List<Schema.Field> fields = schema.getFields();
|
||||||
String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
|
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
|
||||||
Configuration conf = FSUtils.getFs().getConf();
|
String postions = fields.stream().map(f -> String.valueOf(f.pos()))
|
||||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
.collect(Collectors.joining(","));
|
||||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
|
Configuration conf = FSUtils.getFs().getConf();
|
||||||
jobConf.set("partition_columns", "datestr");
|
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||||
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
|
||||||
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
|
jobConf.set("partition_columns", "datestr");
|
||||||
conf.set("partition_columns", "datestr");
|
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||||
inputFormat.setConf(conf);
|
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
|
||||||
jobConf.addResource(conf);
|
conf.set("partition_columns", "datestr");
|
||||||
}
|
inputFormat.setConf(conf);
|
||||||
|
jobConf.addResource(conf);
|
||||||
|
}
|
||||||
|
|
||||||
private static void setInputPath(JobConf jobConf, String inputPath) {
|
private static void setInputPath(JobConf jobConf, String inputPath) {
|
||||||
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
|
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
|
||||||
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
|
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
|
||||||
jobConf.set("map.input.dir", inputPath);
|
jobConf.set("map.input.dir", inputPath);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,17 +16,21 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.common;
|
package com.uber.hoodie.common;
|
||||||
|
|
||||||
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
|
|
||||||
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
|
|
||||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||||
import com.uber.hoodie.common.model.HoodieKey;
|
import com.uber.hoodie.common.model.HoodieKey;
|
||||||
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||||
import com.uber.hoodie.common.util.AvroUtils;
|
|
||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.charset.StandardCharsets;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Random;
|
||||||
|
import java.util.UUID;
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.avro.generic.GenericData;
|
import org.apache.avro.generic.GenericData;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
@@ -34,15 +38,6 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.nio.charset.StandardCharsets;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.Random;
|
|
||||||
import java.util.UUID;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Class to be used in tests to keep generating test inserts and updates against a corpus.
|
* Class to be used in tests to keep generating test inserts and updates against a corpus.
|
||||||
*
|
*
|
||||||
@@ -51,153 +46,164 @@ import java.util.UUID;
|
|||||||
public class HoodieTestDataGenerator {
|
public class HoodieTestDataGenerator {
|
||||||
|
|
||||||
static class KeyPartition {
|
static class KeyPartition {
|
||||||
HoodieKey key;
|
|
||||||
String partitionPath;
|
HoodieKey key;
|
||||||
|
String partitionPath;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\","
|
||||||
|
+ "\"name\": \"triprec\","
|
||||||
|
+ "\"fields\": [ "
|
||||||
|
+ "{\"name\": \"timestamp\",\"type\": \"double\"},"
|
||||||
|
+ "{\"name\": \"_row_key\", \"type\": \"string\"},"
|
||||||
|
+ "{\"name\": \"rider\", \"type\": \"string\"},"
|
||||||
|
+ "{\"name\": \"driver\", \"type\": \"string\"},"
|
||||||
|
+ "{\"name\": \"begin_lat\", \"type\": \"double\"},"
|
||||||
|
+ "{\"name\": \"begin_lon\", \"type\": \"double\"},"
|
||||||
|
+ "{\"name\": \"end_lat\", \"type\": \"double\"},"
|
||||||
|
+ "{\"name\": \"end_lon\", \"type\": \"double\"},"
|
||||||
|
+ "{\"name\":\"fare\",\"type\": \"double\"}]}";
|
||||||
|
|
||||||
|
// based on examination of sample file, the schema produces the following per record size
|
||||||
|
public static final int SIZE_PER_RECORD = 50 * 1024;
|
||||||
|
|
||||||
|
public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"};
|
||||||
|
|
||||||
|
|
||||||
|
public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths,
|
||||||
|
String basePath) {
|
||||||
|
for (String partitionPath : partitionPaths) {
|
||||||
|
new HoodiePartitionMetadata(fs, "000", new Path(basePath), new Path(basePath, partitionPath))
|
||||||
|
.trySave(0);
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public static String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\","
|
private List<KeyPartition> existingKeysList = new ArrayList<>();
|
||||||
+ "\"name\": \"triprec\","
|
public static Schema avroSchema = HoodieAvroUtils
|
||||||
+ "\"fields\": [ "
|
.addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA));
|
||||||
+ "{\"name\": \"timestamp\",\"type\": \"double\"},"
|
private static Random rand = new Random(46474747);
|
||||||
+ "{\"name\": \"_row_key\", \"type\": \"string\"},"
|
private String[] partitionPaths = DEFAULT_PARTITION_PATHS;
|
||||||
+ "{\"name\": \"rider\", \"type\": \"string\"},"
|
|
||||||
+ "{\"name\": \"driver\", \"type\": \"string\"},"
|
|
||||||
+ "{\"name\": \"begin_lat\", \"type\": \"double\"},"
|
|
||||||
+ "{\"name\": \"begin_lon\", \"type\": \"double\"},"
|
|
||||||
+ "{\"name\": \"end_lat\", \"type\": \"double\"},"
|
|
||||||
+ "{\"name\": \"end_lon\", \"type\": \"double\"},"
|
|
||||||
+ "{\"name\":\"fare\",\"type\": \"double\"}]}";
|
|
||||||
|
|
||||||
// based on examination of sample file, the schema produces the following per record size
|
public HoodieTestDataGenerator(String[] partitionPaths) {
|
||||||
public static final int SIZE_PER_RECORD = 50 * 1024;
|
this.partitionPaths = partitionPaths;
|
||||||
|
}
|
||||||
|
|
||||||
public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"};
|
public HoodieTestDataGenerator() {
|
||||||
|
this(new String[]{"2016/03/15", "2015/03/16", "2015/03/17"});
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
public static void writePartitionMetadata(FileSystem fs, String[] partitionPaths, String basePath) {
|
/**
|
||||||
for (String partitionPath: partitionPaths) {
|
* Generates new inserts, uniformly across the partition paths above. It also updates the list of
|
||||||
new HoodiePartitionMetadata(fs, "000", new Path(basePath), new Path(basePath, partitionPath)).trySave(0);
|
* existing keys.
|
||||||
}
|
*/
|
||||||
|
public List<HoodieRecord> generateInserts(String commitTime, int n) throws IOException {
|
||||||
|
List<HoodieRecord> inserts = new ArrayList<>();
|
||||||
|
for (int i = 0; i < n; i++) {
|
||||||
|
String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
|
||||||
|
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
|
||||||
|
HoodieRecord record = new HoodieRecord(key, generateRandomValue(key, commitTime));
|
||||||
|
inserts.add(record);
|
||||||
|
|
||||||
|
KeyPartition kp = new KeyPartition();
|
||||||
|
kp.key = key;
|
||||||
|
kp.partitionPath = partitionPath;
|
||||||
|
existingKeysList.add(kp);
|
||||||
}
|
}
|
||||||
|
return inserts;
|
||||||
|
}
|
||||||
|
|
||||||
private List<KeyPartition> existingKeysList = new ArrayList<>();
|
public List<HoodieRecord> generateDeletes(String commitTime, int n) throws IOException {
|
||||||
public static Schema avroSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA));
|
List<HoodieRecord> inserts = generateInserts(commitTime, n);
|
||||||
private static Random rand = new Random(46474747);
|
return generateDeletesFromExistingRecords(inserts);
|
||||||
private String[] partitionPaths = DEFAULT_PARTITION_PATHS;
|
}
|
||||||
|
|
||||||
|
public List<HoodieRecord> generateDeletesFromExistingRecords(List<HoodieRecord> existingRecords)
|
||||||
|
throws IOException {
|
||||||
|
List<HoodieRecord> deletes = new ArrayList<>();
|
||||||
|
for (HoodieRecord existingRecord : existingRecords) {
|
||||||
|
HoodieRecord record = generateDeleteRecord(existingRecord);
|
||||||
|
deletes.add(record);
|
||||||
|
|
||||||
public HoodieTestDataGenerator(String[] partitionPaths) {
|
|
||||||
this.partitionPaths = partitionPaths;
|
|
||||||
}
|
}
|
||||||
|
return deletes;
|
||||||
|
}
|
||||||
|
|
||||||
public HoodieTestDataGenerator() {
|
public HoodieRecord generateDeleteRecord(HoodieRecord existingRecord) throws IOException {
|
||||||
this(new String[]{"2016/03/15", "2015/03/16", "2015/03/17"});
|
HoodieKey key = existingRecord.getKey();
|
||||||
|
TestRawTripPayload payload = new TestRawTripPayload(Optional.empty(), key.getRecordKey(),
|
||||||
|
key.getPartitionPath(), null, true);
|
||||||
|
return new HoodieRecord(key, payload);
|
||||||
|
}
|
||||||
|
|
||||||
|
public List<HoodieRecord> generateUpdates(String commitTime, List<HoodieRecord> baseRecords)
|
||||||
|
throws IOException {
|
||||||
|
List<HoodieRecord> updates = new ArrayList<>();
|
||||||
|
for (HoodieRecord baseRecord : baseRecords) {
|
||||||
|
HoodieRecord record = new HoodieRecord(baseRecord.getKey(),
|
||||||
|
generateRandomValue(baseRecord.getKey(), commitTime));
|
||||||
|
updates.add(record);
|
||||||
}
|
}
|
||||||
|
return updates;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
/**
|
* Generates new updates, randomly distributed across the keys above.
|
||||||
* Generates new inserts, uniformly across the partition paths above. It also updates the list
|
*/
|
||||||
* of existing keys.
|
public List<HoodieRecord> generateUpdates(String commitTime, int n) throws IOException {
|
||||||
*/
|
List<HoodieRecord> updates = new ArrayList<>();
|
||||||
public List<HoodieRecord> generateInserts(String commitTime, int n) throws IOException {
|
for (int i = 0; i < n; i++) {
|
||||||
List<HoodieRecord> inserts = new ArrayList<>();
|
KeyPartition kp = existingKeysList.get(rand.nextInt(existingKeysList.size() - 1));
|
||||||
for (int i = 0; i < n; i++) {
|
HoodieRecord record = new HoodieRecord(kp.key, generateRandomValue(kp.key, commitTime));
|
||||||
String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
|
updates.add(record);
|
||||||
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
|
|
||||||
HoodieRecord record = new HoodieRecord(key, generateRandomValue(key, commitTime));
|
|
||||||
inserts.add(record);
|
|
||||||
|
|
||||||
KeyPartition kp = new KeyPartition();
|
|
||||||
kp.key = key;
|
|
||||||
kp.partitionPath = partitionPath;
|
|
||||||
existingKeysList.add(kp);
|
|
||||||
}
|
|
||||||
return inserts;
|
|
||||||
}
|
}
|
||||||
|
return updates;
|
||||||
|
}
|
||||||
|
|
||||||
public List<HoodieRecord> generateDeletes(String commitTime, int n) throws IOException {
|
|
||||||
List<HoodieRecord> inserts = generateInserts(commitTime, n);
|
/**
|
||||||
return generateDeletesFromExistingRecords(inserts);
|
* Generates a new avro record of the above schema format, retaining the key if optionally
|
||||||
}
|
* provided.
|
||||||
|
*/
|
||||||
public List<HoodieRecord> generateDeletesFromExistingRecords(List<HoodieRecord> existingRecords) throws IOException {
|
public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime)
|
||||||
List<HoodieRecord> deletes = new ArrayList<>();
|
throws IOException {
|
||||||
for (HoodieRecord existingRecord: existingRecords) {
|
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime,
|
||||||
HoodieRecord record = generateDeleteRecord(existingRecord);
|
"driver-" + commitTime, 0.0);
|
||||||
deletes.add(record);
|
HoodieAvroUtils.addCommitMetadataToRecord(rec, commitTime, "-1");
|
||||||
|
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(),
|
||||||
}
|
TRIP_EXAMPLE_SCHEMA);
|
||||||
return deletes;
|
}
|
||||||
}
|
|
||||||
|
public static GenericRecord generateGenericRecord(String rowKey, String riderName,
|
||||||
public HoodieRecord generateDeleteRecord(HoodieRecord existingRecord) throws IOException {
|
String driverName, double timestamp) {
|
||||||
HoodieKey key = existingRecord.getKey();
|
GenericRecord rec = new GenericData.Record(avroSchema);
|
||||||
TestRawTripPayload payload = new TestRawTripPayload(Optional.empty(), key.getRecordKey(), key.getPartitionPath(), null, true);
|
rec.put("_row_key", rowKey);
|
||||||
return new HoodieRecord(key, payload);
|
rec.put("timestamp", timestamp);
|
||||||
}
|
rec.put("rider", riderName);
|
||||||
|
rec.put("driver", driverName);
|
||||||
public List<HoodieRecord> generateUpdates(String commitTime, List<HoodieRecord> baseRecords) throws IOException {
|
rec.put("begin_lat", rand.nextDouble());
|
||||||
List<HoodieRecord> updates = new ArrayList<>();
|
rec.put("begin_lon", rand.nextDouble());
|
||||||
for (HoodieRecord baseRecord: baseRecords) {
|
rec.put("end_lat", rand.nextDouble());
|
||||||
HoodieRecord record = new HoodieRecord(baseRecord.getKey(), generateRandomValue(baseRecord.getKey(), commitTime));
|
rec.put("end_lon", rand.nextDouble());
|
||||||
updates.add(record);
|
rec.put("fare", rand.nextDouble() * 100);
|
||||||
}
|
return rec;
|
||||||
return updates;
|
}
|
||||||
}
|
|
||||||
|
public static void createCommitFile(String basePath, String commitTime) throws IOException {
|
||||||
/**
|
Path commitFile =
|
||||||
* Generates new updates, randomly distributed across the keys above.
|
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
|
||||||
*/
|
.makeCommitFileName(commitTime));
|
||||||
public List<HoodieRecord> generateUpdates(String commitTime, int n) throws IOException {
|
FileSystem fs = FSUtils.getFs();
|
||||||
List<HoodieRecord> updates = new ArrayList<>();
|
FSDataOutputStream os = fs.create(commitFile, true);
|
||||||
for (int i = 0; i < n; i++) {
|
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||||
KeyPartition kp = existingKeysList.get(rand.nextInt(existingKeysList.size() - 1));
|
try {
|
||||||
HoodieRecord record = new HoodieRecord(kp.key, generateRandomValue(kp.key, commitTime));
|
// Write empty commit metadata
|
||||||
updates.add(record);
|
os.writeBytes(new String(commitMetadata.toJsonString().getBytes(
|
||||||
}
|
StandardCharsets.UTF_8)));
|
||||||
return updates;
|
} finally {
|
||||||
}
|
os.close();
|
||||||
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Generates a new avro record of the above schema format, retaining the key if optionally
|
|
||||||
* provided.
|
|
||||||
*/
|
|
||||||
public static TestRawTripPayload generateRandomValue(HoodieKey key, String commitTime) throws IOException {
|
|
||||||
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime,
|
|
||||||
"driver-" + commitTime, 0.0);
|
|
||||||
HoodieAvroUtils.addCommitMetadataToRecord(rec, commitTime, "-1");
|
|
||||||
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static GenericRecord generateGenericRecord(String rowKey, String riderName,
|
|
||||||
String driverName, double timestamp) {
|
|
||||||
GenericRecord rec = new GenericData.Record(avroSchema);
|
|
||||||
rec.put("_row_key", rowKey);
|
|
||||||
rec.put("timestamp", timestamp);
|
|
||||||
rec.put("rider", riderName);
|
|
||||||
rec.put("driver", driverName);
|
|
||||||
rec.put("begin_lat", rand.nextDouble());
|
|
||||||
rec.put("begin_lon", rand.nextDouble());
|
|
||||||
rec.put("end_lat", rand.nextDouble());
|
|
||||||
rec.put("end_lon", rand.nextDouble());
|
|
||||||
rec.put("fare", rand.nextDouble() * 100);
|
|
||||||
return rec;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void createCommitFile(String basePath, String commitTime) throws IOException {
|
|
||||||
Path commitFile =
|
|
||||||
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime));
|
|
||||||
FileSystem fs = FSUtils.getFs();
|
|
||||||
FSDataOutputStream os = fs.create(commitFile, true);
|
|
||||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
|
||||||
try {
|
|
||||||
// Write empty commit metadata
|
|
||||||
os.writeBytes(new String(commitMetadata.toJsonString().getBytes(
|
|
||||||
StandardCharsets.UTF_8)));
|
|
||||||
} finally {
|
|
||||||
os.close();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public static void createSavepointFile(String basePath, String commitTime) throws IOException {
|
public static void createSavepointFile(String basePath, String commitTime) throws IOException {
|
||||||
Path commitFile =
|
Path commitFile =
|
||||||
@@ -215,7 +221,7 @@ public class HoodieTestDataGenerator {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public String[] getPartitionPaths() {
|
public String[] getPartitionPaths() {
|
||||||
return partitionPaths;
|
return partitionPaths;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,174 +17,182 @@
|
|||||||
package com.uber.hoodie.common;
|
package com.uber.hoodie.common;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
|
||||||
import com.uber.hoodie.WriteStatus;
|
import com.uber.hoodie.WriteStatus;
|
||||||
import com.uber.hoodie.avro.MercifulJsonConverter;
|
import com.uber.hoodie.avro.MercifulJsonConverter;
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||||
|
import java.io.ByteArrayInputStream;
|
||||||
|
import java.io.ByteArrayOutputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.StringWriter;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map.Entry;
|
|
||||||
import org.apache.avro.Schema;
|
|
||||||
import org.apache.avro.generic.IndexedRecord;
|
|
||||||
import org.apache.commons.io.IOUtils;
|
|
||||||
|
|
||||||
import java.io.*;
|
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Map.Entry;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.zip.Deflater;
|
import java.util.zip.Deflater;
|
||||||
import java.util.zip.DeflaterOutputStream;
|
import java.util.zip.DeflaterOutputStream;
|
||||||
import java.util.zip.InflaterInputStream;
|
import java.util.zip.InflaterInputStream;
|
||||||
|
import org.apache.avro.Schema;
|
||||||
|
import org.apache.avro.generic.IndexedRecord;
|
||||||
|
import org.apache.commons.io.IOUtils;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Example row change event based on some example data used by testcases. The data avro schema is
|
* Example row change event based on some example data used by testcases. The data avro schema is
|
||||||
* src/test/resources/schema1.
|
* src/test/resources/schema1.
|
||||||
*/
|
*/
|
||||||
public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayload> {
|
public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayload> {
|
||||||
private transient static final ObjectMapper mapper = new ObjectMapper();
|
|
||||||
private String partitionPath;
|
|
||||||
private String rowKey;
|
|
||||||
private byte[] jsonDataCompressed;
|
|
||||||
private int dataSize;
|
|
||||||
private boolean isDeleted;
|
|
||||||
|
|
||||||
public TestRawTripPayload(Optional<String> jsonData, String rowKey, String partitionPath,
|
private transient static final ObjectMapper mapper = new ObjectMapper();
|
||||||
String schemaStr, Boolean isDeleted) throws IOException {
|
private String partitionPath;
|
||||||
if(jsonData.isPresent()) {
|
private String rowKey;
|
||||||
this.jsonDataCompressed = compressData(jsonData.get());
|
private byte[] jsonDataCompressed;
|
||||||
this.dataSize = jsonData.get().length();
|
private int dataSize;
|
||||||
}
|
private boolean isDeleted;
|
||||||
this.rowKey = rowKey;
|
|
||||||
this.partitionPath = partitionPath;
|
public TestRawTripPayload(Optional<String> jsonData, String rowKey, String partitionPath,
|
||||||
this.isDeleted = isDeleted;
|
String schemaStr, Boolean isDeleted) throws IOException {
|
||||||
|
if (jsonData.isPresent()) {
|
||||||
|
this.jsonDataCompressed = compressData(jsonData.get());
|
||||||
|
this.dataSize = jsonData.get().length();
|
||||||
}
|
}
|
||||||
|
this.rowKey = rowKey;
|
||||||
|
this.partitionPath = partitionPath;
|
||||||
|
this.isDeleted = isDeleted;
|
||||||
|
}
|
||||||
|
|
||||||
public TestRawTripPayload(String jsonData, String rowKey, String partitionPath,
|
public TestRawTripPayload(String jsonData, String rowKey, String partitionPath,
|
||||||
String schemaStr)throws IOException {
|
String schemaStr) throws IOException {
|
||||||
this(Optional.of(jsonData), rowKey, partitionPath, schemaStr, false);
|
this(Optional.of(jsonData), rowKey, partitionPath, schemaStr, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
public TestRawTripPayload(String jsonData) throws IOException {
|
||||||
|
this.jsonDataCompressed = compressData(jsonData);
|
||||||
|
this.dataSize = jsonData.length();
|
||||||
|
Map<String, Object> jsonRecordMap = mapper.readValue(jsonData, Map.class);
|
||||||
|
this.rowKey = jsonRecordMap.get("_row_key").toString();
|
||||||
|
this.partitionPath = jsonRecordMap.get("time").toString().split("T")[0].replace("-", "/");
|
||||||
|
this.isDeleted = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getPartitionPath() {
|
||||||
|
return partitionPath;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TestRawTripPayload preCombine(TestRawTripPayload another) {
|
||||||
|
return another;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema)
|
||||||
|
throws IOException {
|
||||||
|
return this.getInsertValue(schema);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
|
||||||
|
if (isDeleted) {
|
||||||
|
return Optional.empty();
|
||||||
|
} else {
|
||||||
|
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
|
||||||
|
return Optional.of(jsonConverter.convert(getJsonData()));
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public TestRawTripPayload(String jsonData) throws IOException {
|
@Override
|
||||||
this.jsonDataCompressed = compressData(jsonData);
|
public Optional<Map<String, String>> getMetadata() {
|
||||||
this.dataSize = jsonData.length();
|
// Let's assume we want to count the number of input row change events
|
||||||
Map<String, Object> jsonRecordMap = mapper.readValue(jsonData, Map.class);
|
// that are processed. Let the time-bucket for this row change event be 1506582000.
|
||||||
this.rowKey = jsonRecordMap.get("_row_key").toString();
|
Map<String, String> metadataMap = new HashMap<>();
|
||||||
this.partitionPath = jsonRecordMap.get("time").toString().split("T")[0].replace("-", "/");
|
metadataMap.put("InputRecordCount_1506582000", "2");
|
||||||
this.isDeleted = false;
|
return Optional.of(metadataMap);
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getPartitionPath() {
|
public String getRowKey() {
|
||||||
return partitionPath;
|
return rowKey;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String getJsonData() throws IOException {
|
||||||
|
return unCompressData(jsonDataCompressed);
|
||||||
|
}
|
||||||
|
|
||||||
|
private byte[] compressData(String jsonData) throws IOException {
|
||||||
|
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||||
|
DeflaterOutputStream dos =
|
||||||
|
new DeflaterOutputStream(baos, new Deflater(Deflater.BEST_COMPRESSION), true);
|
||||||
|
try {
|
||||||
|
dos.write(jsonData.getBytes());
|
||||||
|
} finally {
|
||||||
|
dos.flush();
|
||||||
|
dos.close();
|
||||||
}
|
}
|
||||||
|
return baos.toByteArray();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override public TestRawTripPayload preCombine(TestRawTripPayload another) {
|
private String unCompressData(byte[] data) throws IOException {
|
||||||
return another;
|
InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data));
|
||||||
}
|
StringWriter sw = new StringWriter(dataSize);
|
||||||
|
IOUtils.copy(iis, sw);
|
||||||
|
return sw.toString();
|
||||||
|
}
|
||||||
|
|
||||||
@Override public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException {
|
/**
|
||||||
return this.getInsertValue(schema);
|
* A custom {@link WriteStatus} that merges passed metadata key value map to {@code
|
||||||
}
|
* WriteStatus.markSuccess()} and {@code WriteStatus.markFailure()}.
|
||||||
|
*/
|
||||||
|
public static class MetadataMergeWriteStatus extends WriteStatus {
|
||||||
|
|
||||||
@Override public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
|
private Map<String, String> mergedMetadataMap = new HashMap<>();
|
||||||
if(isDeleted){
|
|
||||||
return Optional.empty();
|
@Override
|
||||||
} else {
|
public void markSuccess(HoodieRecord record, Optional<Map<String, String>> recordMetadata) {
|
||||||
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
|
super.markSuccess(record, recordMetadata);
|
||||||
return Optional.of(jsonConverter.convert(getJsonData()));
|
if (recordMetadata.isPresent()) {
|
||||||
}
|
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Optional<Map<String, String>> getMetadata() {
|
public void markFailure(HoodieRecord record, Throwable t,
|
||||||
// Let's assume we want to count the number of input row change events
|
Optional<Map<String, String>> recordMetadata) {
|
||||||
// that are processed. Let the time-bucket for this row change event be 1506582000.
|
super.markFailure(record, t, recordMetadata);
|
||||||
Map<String, String> metadataMap = new HashMap<>();
|
if (recordMetadata.isPresent()) {
|
||||||
metadataMap.put("InputRecordCount_1506582000", "2");
|
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
|
||||||
return Optional.of(metadataMap);
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getRowKey() {
|
public static Map<String, String> mergeMetadataForWriteStatuses(
|
||||||
return rowKey;
|
List<WriteStatus> writeStatuses) {
|
||||||
|
Map<String, String> allWriteStatusMergedMetadataMap = new HashMap<>();
|
||||||
|
for (WriteStatus writeStatus : writeStatuses) {
|
||||||
|
MetadataMergeWriteStatus.mergeMetadataMaps(
|
||||||
|
((MetadataMergeWriteStatus) writeStatus).getMergedMetadataMap(),
|
||||||
|
allWriteStatusMergedMetadataMap);
|
||||||
|
}
|
||||||
|
return allWriteStatusMergedMetadataMap;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getJsonData() throws IOException {
|
private static void mergeMetadataMaps(Map<String, String> mergeFromMap,
|
||||||
return unCompressData(jsonDataCompressed);
|
Map<String, String> mergeToMap) {
|
||||||
|
for (Entry<String, String> entry : mergeFromMap.entrySet()) {
|
||||||
|
String key = entry.getKey();
|
||||||
|
if (!mergeToMap.containsKey(key)) {
|
||||||
|
mergeToMap.put(key, "0");
|
||||||
|
}
|
||||||
|
mergeToMap
|
||||||
|
.put(key, addStrsAsInt(entry.getValue(), mergeToMap.get(key)));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private byte[] compressData(String jsonData) throws IOException {
|
private Map<String, String> getMergedMetadataMap() {
|
||||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
return mergedMetadataMap;
|
||||||
DeflaterOutputStream dos =
|
|
||||||
new DeflaterOutputStream(baos, new Deflater(Deflater.BEST_COMPRESSION), true);
|
|
||||||
try {
|
|
||||||
dos.write(jsonData.getBytes());
|
|
||||||
} finally {
|
|
||||||
dos.flush();
|
|
||||||
dos.close();
|
|
||||||
}
|
|
||||||
return baos.toByteArray();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static String addStrsAsInt(String a, String b) {
|
||||||
private String unCompressData(byte[] data) throws IOException {
|
return String.valueOf(Integer.parseInt(a) + Integer.parseInt(b));
|
||||||
InflaterInputStream iis = new InflaterInputStream(new ByteArrayInputStream(data));
|
|
||||||
StringWriter sw = new StringWriter(dataSize);
|
|
||||||
IOUtils.copy(iis, sw);
|
|
||||||
return sw.toString();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* A custom {@link WriteStatus} that merges passed metadata key value map
|
|
||||||
* to {@code WriteStatus.markSuccess()} and {@code WriteStatus.markFailure()}.
|
|
||||||
*/
|
|
||||||
public static class MetadataMergeWriteStatus extends WriteStatus {
|
|
||||||
private Map<String, String> mergedMetadataMap = new HashMap<>();
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void markSuccess(HoodieRecord record, Optional<Map<String, String>> recordMetadata) {
|
|
||||||
super.markSuccess(record, recordMetadata);
|
|
||||||
if(recordMetadata.isPresent()) {
|
|
||||||
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void markFailure(HoodieRecord record, Throwable t,
|
|
||||||
Optional<Map<String, String>> recordMetadata) {
|
|
||||||
super.markFailure(record, t, recordMetadata);
|
|
||||||
if(recordMetadata.isPresent()) {
|
|
||||||
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static Map<String, String> mergeMetadataForWriteStatuses(List<WriteStatus> writeStatuses) {
|
|
||||||
Map<String, String> allWriteStatusMergedMetadataMap = new HashMap<>();
|
|
||||||
for (WriteStatus writeStatus : writeStatuses) {
|
|
||||||
MetadataMergeWriteStatus.mergeMetadataMaps(
|
|
||||||
((MetadataMergeWriteStatus)writeStatus).getMergedMetadataMap(),
|
|
||||||
allWriteStatusMergedMetadataMap);
|
|
||||||
}
|
|
||||||
return allWriteStatusMergedMetadataMap;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void mergeMetadataMaps(Map<String, String> mergeFromMap, Map<String, String> mergeToMap) {
|
|
||||||
for (Entry<String, String> entry : mergeFromMap.entrySet()) {
|
|
||||||
String key = entry.getKey();
|
|
||||||
if(!mergeToMap.containsKey(key)) {
|
|
||||||
mergeToMap.put(key, "0");
|
|
||||||
}
|
|
||||||
mergeToMap
|
|
||||||
.put(key, addStrsAsInt(entry.getValue(), mergeToMap.get(key)));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private Map<String, String> getMergedMetadataMap() {
|
|
||||||
return mergedMetadataMap;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static String addStrsAsInt(String a, String b) {
|
|
||||||
return String.valueOf(Integer.parseInt(a) + Integer.parseInt(b));
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,7 +16,7 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.config;
|
package com.uber.hoodie.config;
|
||||||
|
|
||||||
import static org.junit.Assert.*;
|
import static org.junit.Assert.assertEquals;
|
||||||
|
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig.Builder;
|
import com.uber.hoodie.config.HoodieWriteConfig.Builder;
|
||||||
@@ -29,6 +29,7 @@ import java.util.Properties;
|
|||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
public class HoodieWriteConfigTest {
|
public class HoodieWriteConfigTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testPropertyLoading() throws IOException {
|
public void testPropertyLoading() throws IOException {
|
||||||
Builder builder = HoodieWriteConfig.newBuilder().withPath("/tmp");
|
Builder builder = HoodieWriteConfig.newBuilder().withPath("/tmp");
|
||||||
@@ -46,9 +47,10 @@ public class HoodieWriteConfigTest {
|
|||||||
HoodieWriteConfig config = builder.build();
|
HoodieWriteConfig config = builder.build();
|
||||||
assertEquals(config.getMaxCommitsToKeep(), 5);
|
assertEquals(config.getMaxCommitsToKeep(), 5);
|
||||||
assertEquals(config.getMinCommitsToKeep(), 2);
|
assertEquals(config.getMinCommitsToKeep(), 2);
|
||||||
}
|
}
|
||||||
|
|
||||||
private ByteArrayOutputStream saveParamsIntoOutputStream(Map<String, String> params) throws IOException {
|
private ByteArrayOutputStream saveParamsIntoOutputStream(Map<String, String> params)
|
||||||
|
throws IOException {
|
||||||
Properties properties = new Properties();
|
Properties properties = new Properties();
|
||||||
properties.putAll(params);
|
properties.putAll(params);
|
||||||
ByteArrayOutputStream outStream = new ByteArrayOutputStream();
|
ByteArrayOutputStream outStream = new ByteArrayOutputStream();
|
||||||
|
|||||||
@@ -16,105 +16,103 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.func;
|
package com.uber.hoodie.func;
|
||||||
|
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import static org.junit.Assert.fail;
|
||||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
|
||||||
import com.uber.hoodie.WriteStatus;
|
import com.uber.hoodie.WriteStatus;
|
||||||
import com.uber.hoodie.common.TestRawTripPayload;
|
import com.uber.hoodie.common.TestRawTripPayload;
|
||||||
import com.uber.hoodie.common.model.HoodieKey;
|
import com.uber.hoodie.common.model.HoodieKey;
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
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.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
|
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.rules.TemporaryFolder;
|
import org.junit.rules.TemporaryFolder;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import static org.junit.Assert.fail;
|
|
||||||
|
|
||||||
public class TestUpdateMapFunction {
|
public class TestUpdateMapFunction {
|
||||||
private String basePath = null;
|
|
||||||
|
|
||||||
@Before
|
private String basePath = null;
|
||||||
public void init() throws Exception {
|
|
||||||
// Create a temp folder as the base path
|
@Before
|
||||||
TemporaryFolder folder = new TemporaryFolder();
|
public void init() throws Exception {
|
||||||
folder.create();
|
// Create a temp folder as the base path
|
||||||
this.basePath = folder.getRoot().getAbsolutePath();
|
TemporaryFolder folder = new TemporaryFolder();
|
||||||
HoodieTestUtils.init(basePath);
|
folder.create();
|
||||||
|
this.basePath = folder.getRoot().getAbsolutePath();
|
||||||
|
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");
|
||||||
|
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||||
|
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
|
||||||
|
|
||||||
|
String recordStr1 =
|
||||||
|
"{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||||
|
String recordStr2 =
|
||||||
|
"{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||||
|
String recordStr3 =
|
||||||
|
"{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||||
|
List<HoodieRecord> records = new ArrayList<>();
|
||||||
|
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
||||||
|
records.add(
|
||||||
|
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
|
||||||
|
rowChange1));
|
||||||
|
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
|
||||||
|
records.add(
|
||||||
|
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()),
|
||||||
|
rowChange2));
|
||||||
|
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
||||||
|
records.add(
|
||||||
|
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()),
|
||||||
|
rowChange3));
|
||||||
|
Iterator<List<WriteStatus>> insertResult = table.handleInsert("100", records.iterator());
|
||||||
|
Path commitFile =
|
||||||
|
new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.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 HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||||
|
String fileId = insertResult.next().get(0).getFileId();
|
||||||
|
System.out.println(fileId);
|
||||||
|
|
||||||
|
table = new HoodieCopyOnWriteTable(config, metadata);
|
||||||
|
// New content with values for the newly added field
|
||||||
|
recordStr1 =
|
||||||
|
"{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";
|
||||||
|
records = new ArrayList<>();
|
||||||
|
rowChange1 = new TestRawTripPayload(recordStr1);
|
||||||
|
HoodieRecord record1 =
|
||||||
|
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
|
||||||
|
rowChange1);
|
||||||
|
record1.setCurrentLocation(new HoodieRecordLocation("100", fileId));
|
||||||
|
records.add(record1);
|
||||||
|
|
||||||
|
try {
|
||||||
|
table.handleUpdate("101", fileId, records.iterator());
|
||||||
|
} catch (ClassCastException e) {
|
||||||
|
fail(
|
||||||
|
"UpdateFunction could not read records written with exampleSchema.txt using the exampleEvolvedSchema.txt");
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
private HoodieWriteConfig makeHoodieClientConfig(String schema) throws Exception {
|
||||||
public void testSchemaEvolutionOnUpdate() throws Exception {
|
// Prepare the AvroParquetIO
|
||||||
// Create a bunch of records with a old version of schema
|
String schemaStr = IOUtils.toString(getClass().getResourceAsStream(schema), "UTF-8");
|
||||||
HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
|
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr).build();
|
||||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
}
|
||||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
|
|
||||||
|
|
||||||
String recordStr1 =
|
|
||||||
"{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
|
||||||
String recordStr2 =
|
|
||||||
"{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
|
||||||
String recordStr3 =
|
|
||||||
"{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
|
||||||
List<HoodieRecord> records = new ArrayList<>();
|
|
||||||
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
|
||||||
records.add(
|
|
||||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
|
|
||||||
rowChange1));
|
|
||||||
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
|
|
||||||
records.add(
|
|
||||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()),
|
|
||||||
rowChange2));
|
|
||||||
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
|
||||||
records.add(
|
|
||||||
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()),
|
|
||||||
rowChange3));
|
|
||||||
Iterator<List<WriteStatus>> insertResult = table.handleInsert("100", records.iterator());
|
|
||||||
Path commitFile =
|
|
||||||
new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.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 HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
|
||||||
String fileId = insertResult.next().get(0).getFileId();
|
|
||||||
System.out.println(fileId);
|
|
||||||
|
|
||||||
|
|
||||||
table = new HoodieCopyOnWriteTable(config, metadata);
|
|
||||||
// New content with values for the newly added field
|
|
||||||
recordStr1 =
|
|
||||||
"{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";
|
|
||||||
records = new ArrayList<>();
|
|
||||||
rowChange1 = new TestRawTripPayload(recordStr1);
|
|
||||||
HoodieRecord record1 =
|
|
||||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
|
|
||||||
rowChange1);
|
|
||||||
record1.setCurrentLocation(new HoodieRecordLocation("100", fileId));
|
|
||||||
records.add(record1);
|
|
||||||
|
|
||||||
try {
|
|
||||||
table.handleUpdate("101", fileId, records.iterator());
|
|
||||||
} catch (ClassCastException e) {
|
|
||||||
fail(
|
|
||||||
"UpdateFunction could not read records written with exampleSchema.txt using the exampleEvolvedSchema.txt");
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private HoodieWriteConfig makeHoodieClientConfig(String schema) throws Exception {
|
|
||||||
// Prepare the AvroParquetIO
|
|
||||||
String schemaStr = IOUtils.toString(getClass().getResourceAsStream(schema), "UTF-8");
|
|
||||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr).build();
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,32 +16,31 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.index;
|
package com.uber.hoodie.index;
|
||||||
|
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||||
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
|
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
|
||||||
import com.uber.hoodie.index.hbase.HBaseIndex;
|
import com.uber.hoodie.index.hbase.HBaseIndex;
|
||||||
|
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import static org.junit.Assert.*;
|
|
||||||
|
|
||||||
public class TestHoodieIndex {
|
public class TestHoodieIndex {
|
||||||
@Test
|
|
||||||
public void testCreateIndex() throws Exception {
|
@Test
|
||||||
HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
|
public void testCreateIndex() throws Exception {
|
||||||
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
|
HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
|
||||||
// Different types
|
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
|
||||||
HoodieWriteConfig config = clientConfigBuilder.withPath("")
|
// Different types
|
||||||
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build())
|
HoodieWriteConfig config = clientConfigBuilder.withPath("")
|
||||||
.build();
|
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build())
|
||||||
assertTrue(HoodieIndex.createIndex(config, null) instanceof HBaseIndex);
|
.build();
|
||||||
config = clientConfigBuilder.withPath("").withIndexConfig(
|
assertTrue(HoodieIndex.createIndex(config, null) instanceof HBaseIndex);
|
||||||
indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
config = clientConfigBuilder.withPath("").withIndexConfig(
|
||||||
assertTrue(HoodieIndex.createIndex(config, null) instanceof InMemoryHashIndex);
|
indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||||
config = clientConfigBuilder.withPath("")
|
assertTrue(HoodieIndex.createIndex(config, null) instanceof InMemoryHashIndex);
|
||||||
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
config = clientConfigBuilder.withPath("")
|
||||||
.build();
|
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||||
assertTrue(HoodieIndex.createIndex(config, null) instanceof HoodieBloomIndex);
|
.build();
|
||||||
}
|
assertTrue(HoodieIndex.createIndex(config, null) instanceof HoodieBloomIndex);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -18,28 +18,39 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.index.bloom;
|
package com.uber.hoodie.index.bloom;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertFalse;
|
||||||
|
import static org.junit.Assert.assertNotNull;
|
||||||
|
import static org.junit.Assert.assertNull;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.junit.Assert.fail;
|
||||||
|
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
|
|
||||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
|
||||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
|
||||||
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
|
||||||
import com.uber.hoodie.common.BloomFilter;
|
import com.uber.hoodie.common.BloomFilter;
|
||||||
|
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||||
import com.uber.hoodie.common.TestRawTripPayload;
|
import com.uber.hoodie.common.TestRawTripPayload;
|
||||||
import com.uber.hoodie.common.model.HoodieKey;
|
import com.uber.hoodie.common.model.HoodieKey;
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||||
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||||
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.index.bloom.BloomIndexFileInfo;
|
|
||||||
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
|
|
||||||
import com.uber.hoodie.index.bloom.HoodieBloomIndexCheckFunction;
|
|
||||||
import com.uber.hoodie.io.storage.HoodieParquetConfig;
|
import com.uber.hoodie.io.storage.HoodieParquetConfig;
|
||||||
import com.uber.hoodie.io.storage.HoodieParquetWriter;
|
import com.uber.hoodie.io.storage.HoodieParquetWriter;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.text.SimpleDateFormat;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Date;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.UUID;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
import org.apache.commons.io.IOUtils;
|
import org.apache.commons.io.IOUtils;
|
||||||
@@ -47,11 +58,8 @@ import org.apache.hadoop.conf.Configuration;
|
|||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||||
import org.apache.parquet.avro.AvroWriteSupport;
|
|
||||||
import org.apache.parquet.hadoop.ParquetWriter;
|
import org.apache.parquet.hadoop.ParquetWriter;
|
||||||
import org.apache.parquet.hadoop.api.WriteSupport;
|
|
||||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||||
import org.apache.spark.SparkConf;
|
|
||||||
import org.apache.spark.api.java.JavaPairRDD;
|
import org.apache.spark.api.java.JavaPairRDD;
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
@@ -59,464 +67,489 @@ import org.junit.After;
|
|||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.rules.TemporaryFolder;
|
import org.junit.rules.TemporaryFolder;
|
||||||
import org.mockito.Mockito;
|
|
||||||
|
|
||||||
import scala.Tuple2;
|
import scala.Tuple2;
|
||||||
|
|
||||||
import java.io.File;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.text.SimpleDateFormat;
|
|
||||||
import java.util.*;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
import java.util.stream.Stream;
|
|
||||||
|
|
||||||
import static org.junit.Assert.*;
|
|
||||||
|
|
||||||
public class TestHoodieBloomIndex {
|
public class TestHoodieBloomIndex {
|
||||||
private JavaSparkContext jsc = null;
|
|
||||||
private String basePath = null;
|
|
||||||
private transient final FileSystem fs;
|
|
||||||
private String schemaStr;
|
|
||||||
private Schema schema;
|
|
||||||
|
|
||||||
public TestHoodieBloomIndex() throws Exception {
|
private JavaSparkContext jsc = null;
|
||||||
fs = FSUtils.getFs();
|
private String basePath = null;
|
||||||
}
|
private transient final FileSystem fs;
|
||||||
|
private String schemaStr;
|
||||||
|
private Schema schema;
|
||||||
|
|
||||||
@Before
|
public TestHoodieBloomIndex() throws Exception {
|
||||||
public void init() throws IOException {
|
fs = FSUtils.getFs();
|
||||||
// Initialize a local spark env
|
}
|
||||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieBloomIndex"));
|
|
||||||
// Create a temp folder as the base path
|
|
||||||
TemporaryFolder folder = new TemporaryFolder();
|
|
||||||
folder.create();
|
|
||||||
basePath = folder.getRoot().getAbsolutePath();
|
|
||||||
HoodieTestUtils.init(basePath);
|
|
||||||
// We have some records to be tagged (two different partitions)
|
|
||||||
schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
|
|
||||||
schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Before
|
||||||
public void testLoadUUIDsInMemory() throws IOException {
|
public void init() throws IOException {
|
||||||
// Create one RDD of hoodie record
|
// Initialize a local spark env
|
||||||
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieBloomIndex"));
|
||||||
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
// Create a temp folder as the base path
|
||||||
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
TemporaryFolder folder = new TemporaryFolder();
|
||||||
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
|
folder.create();
|
||||||
|
basePath = folder.getRoot().getAbsolutePath();
|
||||||
|
HoodieTestUtils.init(basePath);
|
||||||
|
// We have some records to be tagged (two different partitions)
|
||||||
|
schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
|
||||||
|
schema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(schemaStr));
|
||||||
|
}
|
||||||
|
|
||||||
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
@Test
|
||||||
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
public void testLoadUUIDsInMemory() throws IOException {
|
||||||
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
|
// Create one RDD of hoodie record
|
||||||
HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||||
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||||
HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||||
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
|
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
|
||||||
HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
|
||||||
|
|
||||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
|
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
||||||
|
HoodieRecord record1 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||||
|
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
|
||||||
|
HoodieRecord record2 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||||
|
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
||||||
|
HoodieRecord record3 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||||
|
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
|
||||||
|
HoodieRecord record4 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||||
|
|
||||||
// Load to memory
|
JavaRDD<HoodieRecord> recordRDD = jsc
|
||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
.parallelize(Arrays.asList(record1, record2, record3, record4));
|
||||||
|
|
||||||
Map<String, Iterable<String>> map = recordRDD
|
// Load to memory
|
||||||
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()))
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||||
.groupByKey().collectAsMap();
|
|
||||||
assertEquals(map.size(), 2);
|
|
||||||
List<String> list1 = Lists.newArrayList(map.get("2016/01/31"));
|
|
||||||
List<String> list2 = Lists.newArrayList(map.get("2015/01/31"));
|
|
||||||
assertEquals(list1.size(), 3);
|
|
||||||
assertEquals(list2.size(), 1);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
Map<String, Iterable<String>> map = recordRDD
|
||||||
public void testLoadInvolvedFiles() throws IOException {
|
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()))
|
||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
|
.groupByKey().collectAsMap();
|
||||||
.withPath(basePath)
|
assertEquals(map.size(), 2);
|
||||||
.build();
|
List<String> list1 = Lists.newArrayList(map.get("2016/01/31"));
|
||||||
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
|
List<String> list2 = Lists.newArrayList(map.get("2015/01/31"));
|
||||||
|
assertEquals(list1.size(), 3);
|
||||||
|
assertEquals(list2.size(), 1);
|
||||||
|
}
|
||||||
|
|
||||||
// Create some partitions, and put some files
|
@Test
|
||||||
// "2016/01/21": 0 file
|
public void testLoadInvolvedFiles() throws IOException {
|
||||||
// "2016/04/01": 1 file (2_0_20160401010101.parquet)
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
|
||||||
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet)
|
.withPath(basePath)
|
||||||
new File(basePath + "/2016/01/21").mkdirs();
|
.build();
|
||||||
new File(basePath + "/2016/04/01").mkdirs();
|
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
|
||||||
new File(basePath + "/2015/03/12").mkdirs();
|
|
||||||
|
|
||||||
TestRawTripPayload rowChange1 = new TestRawTripPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
// Create some partitions, and put some files
|
||||||
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
// "2016/01/21": 0 file
|
||||||
TestRawTripPayload rowChange2 = new TestRawTripPayload("{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
// "2016/04/01": 1 file (2_0_20160401010101.parquet)
|
||||||
HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet)
|
||||||
TestRawTripPayload rowChange3 = new TestRawTripPayload("{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
new File(basePath + "/2016/01/21").mkdirs();
|
||||||
HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
new File(basePath + "/2016/04/01").mkdirs();
|
||||||
TestRawTripPayload rowChange4 = new TestRawTripPayload("{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
new File(basePath + "/2015/03/12").mkdirs();
|
||||||
HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
|
||||||
|
|
||||||
|
TestRawTripPayload rowChange1 = new TestRawTripPayload(
|
||||||
|
"{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||||
|
HoodieRecord record1 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||||
|
TestRawTripPayload rowChange2 = new TestRawTripPayload(
|
||||||
|
"{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||||
|
HoodieRecord record2 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||||
|
TestRawTripPayload rowChange3 = new TestRawTripPayload(
|
||||||
|
"{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||||
|
HoodieRecord record3 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||||
|
TestRawTripPayload rowChange4 = new TestRawTripPayload(
|
||||||
|
"{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||||
|
HoodieRecord record4 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||||
|
|
||||||
writeParquetFile("2016/04/01","2_0_20160401010101.parquet", Lists.newArrayList(), schema, null, false);
|
writeParquetFile("2016/04/01", "2_0_20160401010101.parquet", Lists.newArrayList(), schema, null,
|
||||||
writeParquetFile("2015/03/12","1_0_20150312101010.parquet", Lists.newArrayList(), schema, null, false);
|
false);
|
||||||
writeParquetFile("2015/03/12","3_0_20150312101010.parquet", Arrays.asList(record1), schema, null, false);
|
writeParquetFile("2015/03/12", "1_0_20150312101010.parquet", Lists.newArrayList(), schema, null,
|
||||||
writeParquetFile("2015/03/12","4_0_20150312101010.parquet", Arrays.asList(record2, record3, record4), schema, null, false);
|
false);
|
||||||
|
writeParquetFile("2015/03/12", "3_0_20150312101010.parquet", Arrays.asList(record1), schema,
|
||||||
|
null, false);
|
||||||
|
writeParquetFile("2015/03/12", "4_0_20150312101010.parquet",
|
||||||
|
Arrays.asList(record2, record3, record4), schema, null, false);
|
||||||
|
|
||||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
|
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
|
||||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
||||||
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, table);
|
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, table);
|
||||||
// Still 0, as no valid commit
|
// Still 0, as no valid commit
|
||||||
assertEquals(filesList.size(), 0);
|
assertEquals(filesList.size(), 0);
|
||||||
|
|
||||||
// Add some commits
|
// Add some commits
|
||||||
new File(basePath + "/.hoodie").mkdirs();
|
new File(basePath + "/.hoodie").mkdirs();
|
||||||
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
|
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
|
||||||
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
|
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
|
||||||
|
|
||||||
filesList = index.loadInvolvedFiles(partitions, table);
|
filesList = index.loadInvolvedFiles(partitions, table);
|
||||||
assertEquals(filesList.size(), 4);
|
assertEquals(filesList.size(), 4);
|
||||||
// these files will not have the key ranges
|
// these files will not have the key ranges
|
||||||
assertNull(filesList.get(0)._2().getMaxRecordKey());
|
assertNull(filesList.get(0)._2().getMaxRecordKey());
|
||||||
assertNull(filesList.get(0)._2().getMinRecordKey());
|
assertNull(filesList.get(0)._2().getMinRecordKey());
|
||||||
assertFalse(filesList.get(1)._2().hasKeyRanges());
|
assertFalse(filesList.get(1)._2().hasKeyRanges());
|
||||||
assertNotNull(filesList.get(2)._2().getMaxRecordKey());
|
assertNotNull(filesList.get(2)._2().getMaxRecordKey());
|
||||||
assertNotNull(filesList.get(2)._2().getMinRecordKey());
|
assertNotNull(filesList.get(2)._2().getMinRecordKey());
|
||||||
assertTrue(filesList.get(3)._2().hasKeyRanges());
|
assertTrue(filesList.get(3)._2().hasKeyRanges());
|
||||||
|
|
||||||
// no longer sorted, but should have same files.
|
// no longer sorted, but should have same files.
|
||||||
|
|
||||||
List<Tuple2<String, BloomIndexFileInfo>> expected = Arrays.asList(
|
List<Tuple2<String, BloomIndexFileInfo>> expected = Arrays.asList(
|
||||||
new Tuple2<>("2016/04/01", new BloomIndexFileInfo("2_0_20160401010101.parquet")),
|
new Tuple2<>("2016/04/01", new BloomIndexFileInfo("2_0_20160401010101.parquet")),
|
||||||
new Tuple2<>("2015/03/12",new BloomIndexFileInfo("1_0_20150312101010.parquet")),
|
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("1_0_20150312101010.parquet")),
|
||||||
new Tuple2<>("2015/03/12",new BloomIndexFileInfo("3_0_20150312101010.parquet", "000", "000")),
|
new Tuple2<>("2015/03/12",
|
||||||
new Tuple2<>("2015/03/12",new BloomIndexFileInfo("4_0_20150312101010.parquet", "001", "003"))
|
new BloomIndexFileInfo("3_0_20150312101010.parquet", "000", "000")),
|
||||||
);
|
new Tuple2<>("2015/03/12",
|
||||||
assertEquals(expected, filesList);
|
new BloomIndexFileInfo("4_0_20150312101010.parquet", "001", "003"))
|
||||||
}
|
);
|
||||||
|
assertEquals(expected, filesList);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRangePruning() {
|
public void testRangePruning() {
|
||||||
|
|
||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
|
||||||
.withPath(basePath)
|
.withPath(basePath)
|
||||||
.build();
|
.build();
|
||||||
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
|
HoodieBloomIndex index = new HoodieBloomIndex(config, jsc);
|
||||||
|
|
||||||
|
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
|
||||||
|
partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(
|
||||||
|
new BloomIndexFileInfo("f1"),
|
||||||
|
new BloomIndexFileInfo("f2", "000", "000"),
|
||||||
|
new BloomIndexFileInfo("f3", "001", "003"),
|
||||||
|
new BloomIndexFileInfo("f4", "002", "007"),
|
||||||
|
new BloomIndexFileInfo("f5", "009", "010")
|
||||||
|
));
|
||||||
|
|
||||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
|
JavaPairRDD<String, String> partitionRecordKeyPairRDD = jsc
|
||||||
partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(
|
.parallelize(Arrays.asList(
|
||||||
new BloomIndexFileInfo("f1"),
|
new Tuple2<>("2017/10/22", "003"),
|
||||||
new BloomIndexFileInfo("f2", "000", "000"),
|
new Tuple2<>("2017/10/22", "002"),
|
||||||
new BloomIndexFileInfo("f3", "001", "003"),
|
new Tuple2<>("2017/10/22", "005"),
|
||||||
new BloomIndexFileInfo("f4", "002", "007"),
|
new Tuple2<>("2017/10/22", "004")
|
||||||
new BloomIndexFileInfo("f5", "009", "010")
|
))
|
||||||
|
.mapToPair(t -> t);
|
||||||
|
|
||||||
|
List<Tuple2<String, Tuple2<String, HoodieKey>>> comparisonKeyList = index
|
||||||
|
.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD)
|
||||||
|
.collect();
|
||||||
|
|
||||||
|
assertEquals(10, comparisonKeyList.size());
|
||||||
|
Map<String, List<String>> recordKeyToFileComps = comparisonKeyList.stream()
|
||||||
|
.collect(Collectors.groupingBy(
|
||||||
|
t -> t._2()._2().getRecordKey(),
|
||||||
|
Collectors.mapping(t -> t._2()._1().split("#")[0], Collectors.toList()
|
||||||
|
)
|
||||||
));
|
));
|
||||||
|
|
||||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD = jsc
|
assertEquals(4, recordKeyToFileComps.size());
|
||||||
.parallelize(Arrays.asList(
|
assertEquals(Arrays.asList("f1", "f3", "f4"), recordKeyToFileComps.get("002"));
|
||||||
new Tuple2<>("2017/10/22","003"),
|
assertEquals(Arrays.asList("f1", "f3", "f4"), recordKeyToFileComps.get("003"));
|
||||||
new Tuple2<>("2017/10/22","002"),
|
assertEquals(Arrays.asList("f1", "f4"), recordKeyToFileComps.get("004"));
|
||||||
new Tuple2<>("2017/10/22","005"),
|
assertEquals(Arrays.asList("f1", "f4"), recordKeyToFileComps.get("005"));
|
||||||
new Tuple2<>("2017/10/22","004")
|
}
|
||||||
))
|
|
||||||
.mapToPair(t -> t);
|
@Test
|
||||||
|
public void testCheckUUIDsAgainstOneFile()
|
||||||
|
throws IOException, InterruptedException, ClassNotFoundException {
|
||||||
|
|
||||||
|
// Create some records to use
|
||||||
|
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||||
|
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||||
|
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||||
|
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":32}";
|
||||||
|
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
||||||
|
HoodieRecord record1 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||||
|
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
|
||||||
|
HoodieRecord record2 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||||
|
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
||||||
|
HoodieRecord record3 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||||
|
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
|
||||||
|
HoodieRecord record4 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||||
|
|
||||||
|
// We write record1, record2 to a parquet file, but the bloom filter contains (record1, record2, record3).
|
||||||
|
BloomFilter filter = new BloomFilter(10000, 0.0000001);
|
||||||
|
filter.add(record3.getRecordKey());
|
||||||
|
String filename = writeParquetFile("2016/01/31", Arrays.asList(record1, record2), schema,
|
||||||
|
filter, true);
|
||||||
|
|
||||||
|
// The bloom filter contains 3 records
|
||||||
|
assertTrue(filter.mightContain(record1.getRecordKey()));
|
||||||
|
assertTrue(filter.mightContain(record2.getRecordKey()));
|
||||||
|
assertTrue(filter.mightContain(record3.getRecordKey()));
|
||||||
|
assertFalse(filter.mightContain(record4.getRecordKey()));
|
||||||
|
|
||||||
|
// Compare with file
|
||||||
|
List<String> uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(),
|
||||||
|
record3.getRecordKey(), record4.getRecordKey());
|
||||||
|
|
||||||
|
List<String> results = HoodieBloomIndexCheckFunction.checkCandidatesAgainstFile(uuids,
|
||||||
|
new Path(basePath + "/2016/01/31/" + filename));
|
||||||
|
assertEquals(results.size(), 2);
|
||||||
|
assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")
|
||||||
|
|| results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0"));
|
||||||
|
assertTrue(results.get(0).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")
|
||||||
|
|| results.get(1).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0"));
|
||||||
|
// TODO(vc): Need more coverage on actual filenames
|
||||||
|
//assertTrue(results.get(0)._2().equals(filename));
|
||||||
|
//assertTrue(results.get(1)._2().equals(filename));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTagLocationWithEmptyRDD() throws Exception {
|
||||||
|
// We have some records to be tagged (two different partitions)
|
||||||
|
JavaRDD<HoodieRecord> recordRDD = jsc.emptyRDD();
|
||||||
|
// Also create the metadata and config
|
||||||
|
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||||
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||||
|
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
||||||
|
|
||||||
|
// Let's tag
|
||||||
|
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
|
||||||
|
|
||||||
|
try {
|
||||||
|
bloomIndex.tagLocation(recordRDD, table);
|
||||||
|
} catch (IllegalArgumentException e) {
|
||||||
|
fail(
|
||||||
|
"EmptyRDD should not result in IllegalArgumentException: Positive number of slices required");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
List<Tuple2<String, Tuple2<String, HoodieKey>>> comparisonKeyList = index
|
@Test
|
||||||
.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD)
|
public void testTagLocation() throws Exception {
|
||||||
.collect();
|
// We have some records to be tagged (two different partitions)
|
||||||
|
|
||||||
assertEquals(10, comparisonKeyList.size());
|
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||||
Map<String, List<String>> recordKeyToFileComps = comparisonKeyList.stream()
|
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||||
.collect(Collectors.groupingBy(
|
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||||
t -> t._2()._2().getRecordKey(),
|
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
|
||||||
Collectors.mapping(t -> t._2()._1().split("#")[0], Collectors.toList()
|
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
||||||
)
|
HoodieRecord record1 = new HoodieRecord(
|
||||||
));
|
new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||||
|
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
|
||||||
|
HoodieRecord record2 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||||
|
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
||||||
|
HoodieRecord record3 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||||
|
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
|
||||||
|
HoodieRecord record4 = new HoodieRecord(
|
||||||
|
new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||||
|
JavaRDD<HoodieRecord> recordRDD = jsc
|
||||||
|
.parallelize(Arrays.asList(record1, record2, record3, record4));
|
||||||
|
|
||||||
assertEquals(4, recordKeyToFileComps.size());
|
// Also create the metadata and config
|
||||||
assertEquals(Arrays.asList("f1", "f3", "f4"), recordKeyToFileComps.get("002"));
|
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||||
assertEquals(Arrays.asList("f1", "f3", "f4"), recordKeyToFileComps.get("003"));
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||||
assertEquals(Arrays.asList("f1", "f4"), recordKeyToFileComps.get("004"));
|
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
||||||
assertEquals(Arrays.asList("f1", "f4"), recordKeyToFileComps.get("005"));
|
|
||||||
|
// Let's tag
|
||||||
|
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
|
||||||
|
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
|
||||||
|
|
||||||
|
// Should not find any files
|
||||||
|
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||||
|
assertTrue(!record.isCurrentLocationKnown());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
// We create three parquet file, each having one record. (two different partitions)
|
||||||
public void testCheckUUIDsAgainstOneFile() throws IOException, InterruptedException, ClassNotFoundException {
|
String filename1 = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, null, true);
|
||||||
|
String filename2 = writeParquetFile("2016/01/31", Arrays.asList(record2), schema, null, true);
|
||||||
|
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
|
||||||
|
|
||||||
// Create some records to use
|
// We do the tag again
|
||||||
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
metadata = new HoodieTableMetaClient(fs, basePath);
|
||||||
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
table = HoodieTable.getHoodieTable(metadata, config);
|
||||||
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
|
||||||
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":32}";
|
|
||||||
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
|
||||||
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
|
||||||
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
|
|
||||||
HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
|
||||||
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
|
||||||
HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
|
||||||
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
|
|
||||||
HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
|
||||||
|
|
||||||
|
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
|
||||||
|
|
||||||
// We write record1, record2 to a parquet file, but the bloom filter contains (record1, record2, record3).
|
// Check results
|
||||||
BloomFilter filter = new BloomFilter(10000, 0.0000001);
|
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||||
filter.add(record3.getRecordKey());
|
if (record.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
||||||
String filename = writeParquetFile("2016/01/31", Arrays.asList(record1, record2), schema, filter, true);
|
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename1)));
|
||||||
|
} else if (record.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
|
||||||
|
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename2)));
|
||||||
|
} else if (record.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
||||||
|
assertTrue(!record.isCurrentLocationKnown());
|
||||||
|
} else if (record.getRecordKey().equals("4eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
||||||
|
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename3)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// The bloom filter contains 3 records
|
@Test
|
||||||
assertTrue(filter.mightContain(record1.getRecordKey()));
|
public void testCheckExists() throws Exception {
|
||||||
assertTrue(filter.mightContain(record2.getRecordKey()));
|
// We have some records to be tagged (two different partitions)
|
||||||
assertTrue(filter.mightContain(record3.getRecordKey()));
|
|
||||||
assertFalse(filter.mightContain(record4.getRecordKey()));
|
|
||||||
|
|
||||||
// Compare with file
|
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||||
List<String> uuids = Arrays.asList(record1.getRecordKey(), record2.getRecordKey(),
|
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||||
record3.getRecordKey(), record4.getRecordKey());
|
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||||
|
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
|
||||||
|
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
||||||
|
HoodieKey key1 = new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath());
|
||||||
|
HoodieRecord record1 = new HoodieRecord(key1, rowChange1);
|
||||||
|
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
|
||||||
|
HoodieKey key2 = new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath());
|
||||||
|
HoodieRecord record2 = new HoodieRecord(key2, rowChange2);
|
||||||
|
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
||||||
|
HoodieKey key3 = new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath());
|
||||||
|
HoodieRecord record3 = new HoodieRecord(key3, rowChange3);
|
||||||
|
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
|
||||||
|
HoodieKey key4 = new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath());
|
||||||
|
HoodieRecord record4 = new HoodieRecord(key4, rowChange4);
|
||||||
|
JavaRDD<HoodieKey> keysRDD = jsc.parallelize(Arrays.asList(key1, key2, key3, key4));
|
||||||
|
|
||||||
List<String> results = HoodieBloomIndexCheckFunction.checkCandidatesAgainstFile(uuids,
|
// Also create the metadata and config
|
||||||
new Path(basePath + "/2016/01/31/" + filename));
|
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||||
assertEquals(results.size(), 2);
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||||
assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")
|
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
||||||
|| results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0"));
|
|
||||||
assertTrue(results.get(0).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")
|
// Let's tag
|
||||||
|| results.get(1).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0"));
|
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
|
||||||
// TODO(vc): Need more coverage on actual filenames
|
JavaPairRDD<HoodieKey, Optional<String>> taggedRecordRDD = bloomIndex
|
||||||
//assertTrue(results.get(0)._2().equals(filename));
|
.fetchRecordLocation(keysRDD, table);
|
||||||
//assertTrue(results.get(1)._2().equals(filename));
|
|
||||||
|
// Should not find any files
|
||||||
|
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
|
||||||
|
assertTrue(!record._2.isPresent());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
// We create three parquet file, each having one record. (two different partitions)
|
||||||
public void testTagLocationWithEmptyRDD() throws Exception {
|
String filename1 = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, null, true);
|
||||||
// We have some records to be tagged (two different partitions)
|
String filename2 = writeParquetFile("2016/01/31", Arrays.asList(record2), schema, null, true);
|
||||||
JavaRDD<HoodieRecord> recordRDD = jsc.emptyRDD();
|
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
|
||||||
// Also create the metadata and config
|
|
||||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
|
||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
|
||||||
|
|
||||||
// Let's tag
|
// We do the tag again
|
||||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
|
metadata = new HoodieTableMetaClient(fs, basePath);
|
||||||
|
table = HoodieTable.getHoodieTable(metadata, config);
|
||||||
|
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table);
|
||||||
|
|
||||||
try {
|
// Check results
|
||||||
bloomIndex.tagLocation(recordRDD, table);
|
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
|
||||||
} catch (IllegalArgumentException e) {
|
if (record._1.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
||||||
fail("EmptyRDD should not result in IllegalArgumentException: Positive number of slices required");
|
assertTrue(record._2.isPresent());
|
||||||
}
|
Path path1 = new Path(record._2.get());
|
||||||
|
assertEquals(FSUtils.getFileId(filename1), FSUtils.getFileId(path1.getName()));
|
||||||
|
} else if (record._1.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
|
||||||
|
assertTrue(record._2.isPresent());
|
||||||
|
Path path2 = new Path(record._2.get());
|
||||||
|
assertEquals(FSUtils.getFileId(filename2), FSUtils.getFileId(path2.getName()));
|
||||||
|
} else if (record._1.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
||||||
|
assertTrue(!record._2.isPresent());
|
||||||
|
} else if (record._1.getRecordKey().equals("4eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
||||||
|
assertTrue(record._2.isPresent());
|
||||||
|
Path path3 = new Path(record._2.get());
|
||||||
|
assertEquals(FSUtils.getFileId(filename3), FSUtils.getFileId(path3.getName()));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testTagLocation() throws Exception {
|
public void testBloomFilterFalseError() throws IOException, InterruptedException {
|
||||||
// We have some records to be tagged (two different partitions)
|
// We have two hoodie records
|
||||||
|
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||||
|
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||||
|
|
||||||
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
// We write record1 to a parquet file, using a bloom filter having both records
|
||||||
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
||||||
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
HoodieRecord record1 = new HoodieRecord(
|
||||||
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
|
new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||||
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
|
||||||
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
HoodieRecord record2 = new HoodieRecord(
|
||||||
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
|
new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||||
HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
|
||||||
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
|
||||||
HoodieRecord record3 = new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
|
||||||
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
|
|
||||||
HoodieRecord record4 = new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
|
||||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
|
|
||||||
|
|
||||||
// Also create the metadata and config
|
BloomFilter filter = new BloomFilter(10000, 0.0000001);
|
||||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
filter.add(record2.getRecordKey());
|
||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
String filename = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, filter, true);
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
assertTrue(filter.mightContain(record1.getRecordKey()));
|
||||||
|
assertTrue(filter.mightContain(record2.getRecordKey()));
|
||||||
|
|
||||||
// Let's tag
|
// We do the tag
|
||||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
|
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
|
||||||
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
|
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||||
|
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||||
|
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
||||||
|
|
||||||
// Should not find any files
|
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
|
||||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
|
||||||
assertTrue(!record.isCurrentLocationKnown());
|
|
||||||
}
|
|
||||||
|
|
||||||
// We create three parquet file, each having one record. (two different partitions)
|
// Check results
|
||||||
String filename1 = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, null, true);
|
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||||
String filename2 = writeParquetFile("2016/01/31", Arrays.asList(record2), schema, null, true);
|
if (record.getKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
||||||
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
|
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename)));
|
||||||
|
} else if (record.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
|
||||||
// We do the tag again
|
assertFalse(record.isCurrentLocationKnown());
|
||||||
metadata = new HoodieTableMetaClient(fs, basePath);
|
}
|
||||||
table = HoodieTable.getHoodieTable(metadata, config);
|
|
||||||
|
|
||||||
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
|
|
||||||
|
|
||||||
// Check results
|
|
||||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
|
||||||
if (record.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
|
||||||
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename1)));
|
|
||||||
} else if (record.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
|
|
||||||
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename2)));
|
|
||||||
} else if (record.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
|
||||||
assertTrue(!record.isCurrentLocationKnown());
|
|
||||||
} else if (record.getRecordKey().equals("4eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
|
||||||
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename3)));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
private String writeParquetFile(String partitionPath, List<HoodieRecord> records, Schema schema,
|
||||||
public void testCheckExists() throws Exception {
|
BloomFilter filter, boolean createCommitTime) throws IOException, InterruptedException {
|
||||||
// We have some records to be tagged (two different partitions)
|
Thread.sleep(1000);
|
||||||
|
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
||||||
|
String fileId = UUID.randomUUID().toString();
|
||||||
|
String filename = FSUtils.makeDataFileName(commitTime, 1, fileId);
|
||||||
|
|
||||||
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
return writeParquetFile(partitionPath, filename, records, schema, filter, createCommitTime);
|
||||||
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
}
|
||||||
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
|
||||||
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
|
|
||||||
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
|
||||||
HoodieKey key1 = new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath());
|
|
||||||
HoodieRecord record1 = new HoodieRecord(key1, rowChange1);
|
|
||||||
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
|
|
||||||
HoodieKey key2 = new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath());
|
|
||||||
HoodieRecord record2 = new HoodieRecord(key2, rowChange2);
|
|
||||||
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
|
|
||||||
HoodieKey key3 = new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath());
|
|
||||||
HoodieRecord record3 = new HoodieRecord(key3, rowChange3);
|
|
||||||
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
|
|
||||||
HoodieKey key4 = new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath());
|
|
||||||
HoodieRecord record4 = new HoodieRecord(key4, rowChange4);
|
|
||||||
JavaRDD<HoodieKey> keysRDD = jsc.parallelize(Arrays.asList(key1, key2, key3, key4));
|
|
||||||
|
|
||||||
// Also create the metadata and config
|
private String writeParquetFile(String partitionPath, String filename, List<HoodieRecord> records,
|
||||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
Schema schema,
|
||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
BloomFilter filter, boolean createCommitTime) throws IOException {
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
|
||||||
|
|
||||||
// Let's tag
|
if (filter == null) {
|
||||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
|
filter = new BloomFilter(10000, 0.0000001);
|
||||||
JavaPairRDD<HoodieKey, Optional<String>> taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table);
|
|
||||||
|
|
||||||
// Should not find any files
|
|
||||||
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
|
|
||||||
assertTrue(!record._2.isPresent());
|
|
||||||
}
|
|
||||||
|
|
||||||
// We create three parquet file, each having one record. (two different partitions)
|
|
||||||
String filename1 = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, null, true);
|
|
||||||
String filename2 = writeParquetFile("2016/01/31", Arrays.asList(record2), schema, null, true);
|
|
||||||
String filename3 = writeParquetFile("2015/01/31", Arrays.asList(record4), schema, null, true);
|
|
||||||
|
|
||||||
// We do the tag again
|
|
||||||
metadata = new HoodieTableMetaClient(fs, basePath);
|
|
||||||
table = HoodieTable.getHoodieTable(metadata, config);
|
|
||||||
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, table);
|
|
||||||
|
|
||||||
// Check results
|
|
||||||
for (Tuple2<HoodieKey, Optional<String>> record : taggedRecordRDD.collect()) {
|
|
||||||
if (record._1.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
|
||||||
assertTrue(record._2.isPresent());
|
|
||||||
Path path1 = new Path(record._2.get());
|
|
||||||
assertEquals(FSUtils.getFileId(filename1), FSUtils.getFileId(path1.getName()));
|
|
||||||
} else if (record._1.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
|
|
||||||
assertTrue(record._2.isPresent());
|
|
||||||
Path path2 = new Path(record._2.get());
|
|
||||||
assertEquals(FSUtils.getFileId(filename2), FSUtils.getFileId(path2.getName()));
|
|
||||||
} else if (record._1.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
|
||||||
assertTrue(!record._2.isPresent());
|
|
||||||
} else if (record._1.getRecordKey().equals("4eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
|
||||||
assertTrue(record._2.isPresent());
|
|
||||||
Path path3 = new Path(record._2.get());
|
|
||||||
assertEquals(FSUtils.getFileId(filename3), FSUtils.getFileId(path3.getName()));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
|
||||||
|
new AvroSchemaConverter().convert(schema), schema, filter);
|
||||||
@Test
|
String commitTime = FSUtils.getCommitTime(filename);
|
||||||
public void testBloomFilterFalseError() throws IOException, InterruptedException {
|
HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
||||||
// We have two hoodie records
|
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024,
|
||||||
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
new Configuration());
|
||||||
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
HoodieParquetWriter writer = new HoodieParquetWriter(
|
||||||
|
commitTime,
|
||||||
// We write record1 to a parquet file, using a bloom filter having both records
|
new Path(basePath + "/" + partitionPath + "/" + filename),
|
||||||
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
|
config,
|
||||||
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
schema);
|
||||||
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
|
int seqId = 1;
|
||||||
HoodieRecord record2 = new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
for (HoodieRecord record : records) {
|
||||||
|
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
|
||||||
BloomFilter filter = new BloomFilter(10000, 0.0000001);
|
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, commitTime, "" + seqId++);
|
||||||
filter.add(record2.getRecordKey());
|
HoodieAvroUtils
|
||||||
String filename = writeParquetFile("2016/01/31", Arrays.asList(record1), schema, filter, true);
|
.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(),
|
||||||
assertTrue(filter.mightContain(record1.getRecordKey()));
|
filename);
|
||||||
assertTrue(filter.mightContain(record2.getRecordKey()));
|
writer.writeAvro(record.getRecordKey(), avroRecord);
|
||||||
|
filter.add(record.getRecordKey());
|
||||||
// We do the tag
|
|
||||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
|
|
||||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
|
||||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
|
|
||||||
|
|
||||||
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config, jsc);
|
|
||||||
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, table);
|
|
||||||
|
|
||||||
// Check results
|
|
||||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
|
||||||
if (record.getKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
|
||||||
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename)));
|
|
||||||
} else if (record.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
|
|
||||||
assertFalse(record.isCurrentLocationKnown());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
writer.close();
|
||||||
|
|
||||||
private String writeParquetFile(String partitionPath, List<HoodieRecord> records, Schema schema,
|
if (createCommitTime) {
|
||||||
BloomFilter filter, boolean createCommitTime) throws IOException, InterruptedException {
|
// Also make sure the commit is valid
|
||||||
Thread.sleep(1000);
|
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME).mkdirs();
|
||||||
String commitTime = new SimpleDateFormat("yyyyMMddHHmmss").format(new Date());
|
new File(
|
||||||
String fileId = UUID.randomUUID().toString();
|
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + ".commit")
|
||||||
String filename = FSUtils.makeDataFileName(commitTime, 1, fileId);
|
.createNewFile();
|
||||||
|
|
||||||
|
|
||||||
return writeParquetFile(partitionPath, filename, records, schema, filter, createCommitTime);
|
|
||||||
}
|
}
|
||||||
|
return filename;
|
||||||
|
}
|
||||||
|
|
||||||
private String writeParquetFile(String partitionPath, String filename, List<HoodieRecord> records, Schema schema,
|
@After
|
||||||
BloomFilter filter, boolean createCommitTime) throws IOException {
|
public void clean() {
|
||||||
|
if (jsc != null) {
|
||||||
|
jsc.stop();
|
||||||
if (filter == null) {
|
|
||||||
filter = new BloomFilter(10000, 0.0000001);
|
|
||||||
}
|
|
||||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
|
|
||||||
String commitTime = FSUtils.getCommitTime(filename);
|
|
||||||
HoodieParquetConfig config = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
|
||||||
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024, new Configuration());
|
|
||||||
HoodieParquetWriter writer = new HoodieParquetWriter(
|
|
||||||
commitTime,
|
|
||||||
new Path(basePath + "/" + partitionPath + "/" + filename),
|
|
||||||
config,
|
|
||||||
schema);
|
|
||||||
int seqId = 1;
|
|
||||||
for (HoodieRecord record : records) {
|
|
||||||
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
|
|
||||||
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, commitTime, "" + seqId++);
|
|
||||||
HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), filename);
|
|
||||||
writer.writeAvro(record.getRecordKey(), avroRecord);
|
|
||||||
filter.add(record.getRecordKey());
|
|
||||||
}
|
|
||||||
writer.close();
|
|
||||||
|
|
||||||
if (createCommitTime) {
|
|
||||||
// Also make sure the commit is valid
|
|
||||||
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME).mkdirs();
|
|
||||||
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitTime + ".commit").createNewFile();
|
|
||||||
}
|
|
||||||
return filename;
|
|
||||||
}
|
}
|
||||||
|
if (basePath != null) {
|
||||||
@After
|
new File(basePath).delete();
|
||||||
public void clean() {
|
|
||||||
if (jsc != null) {
|
|
||||||
jsc.stop();
|
|
||||||
}
|
|
||||||
if (basePath != null) {
|
|
||||||
new File(basePath).delete();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,9 +16,11 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.io;
|
package com.uber.hoodie.io;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry;
|
import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry;
|
||||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||||
import com.uber.hoodie.common.model.HoodieArchivedLogFile;
|
|
||||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
@@ -29,6 +31,11 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
|||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
import org.apache.avro.generic.IndexedRecord;
|
import org.apache.avro.generic.IndexedRecord;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
@@ -37,197 +44,196 @@ import org.junit.Before;
|
|||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.rules.TemporaryFolder;
|
import org.junit.rules.TemporaryFolder;
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
|
||||||
import static org.junit.Assert.assertTrue;
|
|
||||||
|
|
||||||
public class TestHoodieCommitArchiveLog {
|
public class TestHoodieCommitArchiveLog {
|
||||||
private String basePath;
|
|
||||||
private FileSystem fs;
|
|
||||||
|
|
||||||
@Before
|
private String basePath;
|
||||||
public void init() throws Exception {
|
private FileSystem fs;
|
||||||
TemporaryFolder folder = new TemporaryFolder();
|
|
||||||
folder.create();
|
@Before
|
||||||
basePath = folder.getRoot().getAbsolutePath();
|
public void init() throws Exception {
|
||||||
HoodieTestUtils.init(basePath);
|
TemporaryFolder folder = new TemporaryFolder();
|
||||||
fs = FSUtils.getFs();
|
folder.create();
|
||||||
|
basePath = folder.getRoot().getAbsolutePath();
|
||||||
|
HoodieTestUtils.init(basePath);
|
||||||
|
fs = FSUtils.getFs();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testArchiveEmptyDataset() throws IOException {
|
||||||
|
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
|
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||||
|
.forTable("test-trip-table").build();
|
||||||
|
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
||||||
|
boolean result = archiveLog.archiveIfRequired();
|
||||||
|
assertTrue(result);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testArchiveDatasetWithArchival() throws IOException {
|
||||||
|
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
|
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||||
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4).build())
|
||||||
|
.forTable("test-trip-table").build();
|
||||||
|
HoodieTestUtils.init(basePath);
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "102");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "103");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "104");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
||||||
|
|
||||||
|
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||||
|
HoodieTimeline timeline =
|
||||||
|
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
||||||
|
|
||||||
|
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
||||||
|
|
||||||
|
HoodieTestUtils.createCleanFiles(basePath, "100");
|
||||||
|
HoodieTestUtils.createCleanFiles(basePath, "101");
|
||||||
|
HoodieTestUtils.createCleanFiles(basePath, "102");
|
||||||
|
HoodieTestUtils.createCleanFiles(basePath, "103");
|
||||||
|
HoodieTestUtils.createCleanFiles(basePath, "104");
|
||||||
|
HoodieTestUtils.createCleanFiles(basePath, "105");
|
||||||
|
|
||||||
|
//reload the timeline and get all the commmits before archive
|
||||||
|
timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline()
|
||||||
|
.filterCompletedInstants();
|
||||||
|
List<HoodieInstant> originalCommits = timeline.getInstants().collect(Collectors.toList());
|
||||||
|
|
||||||
|
assertEquals("Loaded 6 commits and the count should match", 12, timeline.countInstants());
|
||||||
|
|
||||||
|
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
||||||
|
|
||||||
|
assertTrue(archiveLog.archiveIfRequired());
|
||||||
|
|
||||||
|
//reload the timeline and remove the remaining commits
|
||||||
|
timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline()
|
||||||
|
.filterCompletedInstants();
|
||||||
|
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
|
||||||
|
|
||||||
|
//read the file
|
||||||
|
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(),
|
||||||
|
new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")),
|
||||||
|
HoodieArchivedMetaEntry.getClassSchema(), false);
|
||||||
|
|
||||||
|
int archivedRecordsCount = 0;
|
||||||
|
List<IndexedRecord> readRecords = new ArrayList<>();
|
||||||
|
//read the avro blocks and validate the number of records written in each avro block
|
||||||
|
while (reader.hasNext()) {
|
||||||
|
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
|
||||||
|
List<IndexedRecord> records = blk.getRecords();
|
||||||
|
readRecords.addAll(records);
|
||||||
|
assertEquals("Archived and read records for each block are same", 8, records.size());
|
||||||
|
archivedRecordsCount += records.size();
|
||||||
}
|
}
|
||||||
|
assertEquals("Total archived records and total read records are the same count", 8,
|
||||||
|
archivedRecordsCount);
|
||||||
|
|
||||||
@Test
|
//make sure the archived commits are the same as the (originalcommits - commitsleft)
|
||||||
public void testArchiveEmptyDataset() throws IOException {
|
List<String> readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> {
|
||||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
return r.get("commitTime").toString();
|
||||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
}).collect(Collectors.toList());
|
||||||
.forTable("test-trip-table").build();
|
Collections.sort(readCommits);
|
||||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
|
||||||
boolean result = archiveLog.archiveIfRequired();
|
|
||||||
assertTrue(result);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
assertEquals(
|
||||||
public void testArchiveDatasetWithArchival() throws IOException {
|
"Read commits map should match the originalCommits - commitsLoadedFromArchival",
|
||||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()),
|
||||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
readCommits);
|
||||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4).build())
|
}
|
||||||
.forTable("test-trip-table").build();
|
|
||||||
HoodieTestUtils.init(basePath);
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "102");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "103");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "104");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
|
||||||
|
|
||||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
@Test
|
||||||
HoodieTimeline timeline =
|
public void testArchiveDatasetWithNoArchival() throws IOException {
|
||||||
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
|
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||||
|
.forTable("test-trip-table").withCompactionConfig(
|
||||||
|
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
|
||||||
|
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");
|
||||||
|
|
||||||
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
HoodieTimeline timeline =
|
||||||
|
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
||||||
|
|
||||||
HoodieTestUtils.createCleanFiles(basePath, "100");
|
assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants());
|
||||||
HoodieTestUtils.createCleanFiles(basePath, "101");
|
boolean result = archiveLog.archiveIfRequired();
|
||||||
HoodieTestUtils.createCleanFiles(basePath, "102");
|
assertTrue(result);
|
||||||
HoodieTestUtils.createCleanFiles(basePath, "103");
|
timeline =
|
||||||
HoodieTestUtils.createCleanFiles(basePath, "104");
|
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline()
|
||||||
HoodieTestUtils.createCleanFiles(basePath, "105");
|
.filterCompletedInstants();
|
||||||
|
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4,
|
||||||
|
timeline.countInstants());
|
||||||
|
}
|
||||||
|
|
||||||
//reload the timeline and get all the commmits before archive
|
@Test
|
||||||
timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
|
public void testArchiveCommitSafety() throws IOException {
|
||||||
List<HoodieInstant> originalCommits = timeline.getInstants().collect(Collectors.toList());
|
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
|
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||||
|
.forTable("test-trip-table").withCompactionConfig(
|
||||||
|
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
|
||||||
|
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");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "104");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
||||||
|
|
||||||
assertEquals("Loaded 6 commits and the count should match", 12, timeline.countInstants());
|
HoodieTimeline timeline =
|
||||||
|
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
||||||
|
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
||||||
|
boolean result = archiveLog.archiveIfRequired();
|
||||||
|
assertTrue(result);
|
||||||
|
timeline =
|
||||||
|
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline()
|
||||||
|
.filterCompletedInstants();
|
||||||
|
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"));
|
||||||
|
}
|
||||||
|
|
||||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
@Test
|
||||||
|
public void testArchiveCommitSavepointNoHole() throws IOException {
|
||||||
|
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
|
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||||
|
.forTable("test-trip-table").withCompactionConfig(
|
||||||
|
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
|
||||||
|
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||||
|
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
||||||
|
HoodieTestDataGenerator.createSavepointFile(basePath, "101");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "102");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "103");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "104");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
||||||
|
|
||||||
assertTrue(archiveLog.archiveIfRequired());
|
HoodieTimeline timeline =
|
||||||
|
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
||||||
//reload the timeline and remove the remaining commits
|
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
||||||
timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
|
boolean result = archiveLog.archiveIfRequired();
|
||||||
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
|
assertTrue(result);
|
||||||
|
timeline =
|
||||||
//read the file
|
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline()
|
||||||
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(),
|
.filterCompletedInstants();
|
||||||
new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")), HoodieArchivedMetaEntry.getClassSchema(), false);
|
assertEquals(
|
||||||
|
"Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)",
|
||||||
int archivedRecordsCount = 0;
|
5, timeline.countInstants());
|
||||||
List<IndexedRecord> readRecords = new ArrayList<>();
|
assertTrue("Archived commits should always be safe",
|
||||||
//read the avro blocks and validate the number of records written in each avro block
|
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "101")));
|
||||||
while(reader.hasNext()) {
|
assertTrue("Archived commits should always be safe",
|
||||||
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
|
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102")));
|
||||||
List<IndexedRecord> records = blk.getRecords();
|
assertTrue("Archived commits should always be safe",
|
||||||
readRecords.addAll(records);
|
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")));
|
||||||
assertEquals("Archived and read records for each block are same", 8, records.size());
|
}
|
||||||
archivedRecordsCount += records.size();
|
|
||||||
}
|
|
||||||
assertEquals("Total archived records and total read records are the same count", 8, archivedRecordsCount);
|
|
||||||
|
|
||||||
//make sure the archived commits are the same as the (originalcommits - commitsleft)
|
|
||||||
List<String> readCommits = readRecords.stream().map(r -> (GenericRecord)r).map(r -> {
|
|
||||||
return r.get("commitTime").toString();
|
|
||||||
}).collect(Collectors.toList());
|
|
||||||
Collections.sort(readCommits);
|
|
||||||
|
|
||||||
assertEquals(
|
|
||||||
"Read commits map should match the originalCommits - commitsLoadedFromArchival",
|
|
||||||
originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()),
|
|
||||||
readCommits);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testArchiveDatasetWithNoArchival() throws IOException {
|
|
||||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
|
||||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
|
||||||
.forTable("test-trip-table").withCompactionConfig(
|
|
||||||
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
|
|
||||||
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");
|
|
||||||
|
|
||||||
HoodieTimeline timeline =
|
|
||||||
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
|
||||||
|
|
||||||
assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants());
|
|
||||||
boolean result = archiveLog.archiveIfRequired();
|
|
||||||
assertTrue(result);
|
|
||||||
timeline =
|
|
||||||
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
|
||||||
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4,
|
|
||||||
timeline.countInstants());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testArchiveCommitSafety() throws IOException {
|
|
||||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
|
||||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
|
||||||
.forTable("test-trip-table").withCompactionConfig(
|
|
||||||
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
|
|
||||||
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");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "104");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
|
||||||
|
|
||||||
HoodieTimeline timeline =
|
|
||||||
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
|
||||||
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
|
||||||
boolean result = archiveLog.archiveIfRequired();
|
|
||||||
assertTrue(result);
|
|
||||||
timeline =
|
|
||||||
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
|
||||||
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"));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testArchiveCommitSavepointNoHole() throws IOException {
|
|
||||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
|
||||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
|
||||||
.forTable("test-trip-table").withCompactionConfig(
|
|
||||||
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
|
|
||||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
|
||||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
|
||||||
HoodieTestDataGenerator.createSavepointFile(basePath, "101");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "102");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "103");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "104");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
|
||||||
|
|
||||||
HoodieTimeline timeline =
|
|
||||||
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
|
||||||
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
|
||||||
boolean result = archiveLog.archiveIfRequired();
|
|
||||||
assertTrue(result);
|
|
||||||
timeline =
|
|
||||||
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
|
||||||
assertEquals(
|
|
||||||
"Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)",
|
|
||||||
5, timeline.countInstants());
|
|
||||||
assertTrue("Archived commits should always be safe",
|
|
||||||
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "101")));
|
|
||||||
assertTrue("Archived commits should always be safe",
|
|
||||||
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102")));
|
|
||||||
assertTrue("Archived commits should always be safe",
|
|
||||||
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")));
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,7 +16,9 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.io;
|
package com.uber.hoodie.io;
|
||||||
|
|
||||||
import com.uber.hoodie.HoodieReadClient;
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
import com.uber.hoodie.HoodieWriteClient;
|
import com.uber.hoodie.HoodieWriteClient;
|
||||||
import com.uber.hoodie.WriteStatus;
|
import com.uber.hoodie.WriteStatus;
|
||||||
import com.uber.hoodie.common.HoodieClientTestUtils;
|
import com.uber.hoodie.common.HoodieClientTestUtils;
|
||||||
@@ -34,13 +36,16 @@ import com.uber.hoodie.config.HoodieCompactionConfig;
|
|||||||
import com.uber.hoodie.config.HoodieIndexConfig;
|
import com.uber.hoodie.config.HoodieIndexConfig;
|
||||||
import com.uber.hoodie.config.HoodieStorageConfig;
|
import com.uber.hoodie.config.HoodieStorageConfig;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
|
|
||||||
import com.uber.hoodie.index.HoodieIndex;
|
import com.uber.hoodie.index.HoodieIndex;
|
||||||
|
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
|
||||||
import com.uber.hoodie.io.compact.HoodieCompactor;
|
import com.uber.hoodie.io.compact.HoodieCompactor;
|
||||||
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.spark.SparkConf;
|
|
||||||
import org.apache.spark.api.java.JavaRDD;
|
import org.apache.spark.api.java.JavaRDD;
|
||||||
import org.apache.spark.api.java.JavaSparkContext;
|
import org.apache.spark.api.java.JavaSparkContext;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
@@ -48,161 +53,154 @@ import org.junit.Before;
|
|||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.rules.TemporaryFolder;
|
import org.junit.rules.TemporaryFolder;
|
||||||
|
|
||||||
import java.io.File;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
|
||||||
import static org.junit.Assert.assertTrue;
|
|
||||||
|
|
||||||
public class TestHoodieCompactor {
|
public class TestHoodieCompactor {
|
||||||
private transient JavaSparkContext jsc = null;
|
|
||||||
private String basePath = null;
|
|
||||||
private HoodieCompactor compactor;
|
|
||||||
private transient HoodieTestDataGenerator dataGen = null;
|
|
||||||
|
|
||||||
@Before
|
private transient JavaSparkContext jsc = null;
|
||||||
public void init() throws IOException {
|
private String basePath = null;
|
||||||
// Initialize a local spark env
|
private HoodieCompactor compactor;
|
||||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieCompactor"));
|
private transient HoodieTestDataGenerator dataGen = null;
|
||||||
|
|
||||||
// Create a temp folder as the base path
|
@Before
|
||||||
TemporaryFolder folder = new TemporaryFolder();
|
public void init() throws IOException {
|
||||||
folder.create();
|
// Initialize a local spark env
|
||||||
basePath = folder.getRoot().getAbsolutePath();
|
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieCompactor"));
|
||||||
HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ);
|
|
||||||
|
|
||||||
dataGen = new HoodieTestDataGenerator();
|
// Create a temp folder as the base path
|
||||||
compactor = new HoodieRealtimeTableCompactor();
|
TemporaryFolder folder = new TemporaryFolder();
|
||||||
|
folder.create();
|
||||||
|
basePath = folder.getRoot().getAbsolutePath();
|
||||||
|
HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ);
|
||||||
|
|
||||||
|
dataGen = new HoodieTestDataGenerator();
|
||||||
|
compactor = new HoodieRealtimeTableCompactor();
|
||||||
|
}
|
||||||
|
|
||||||
|
@After
|
||||||
|
public void clean() {
|
||||||
|
if (basePath != null) {
|
||||||
|
new File(basePath).delete();
|
||||||
}
|
}
|
||||||
|
if (jsc != null) {
|
||||||
@After
|
jsc.stop();
|
||||||
public void clean() {
|
|
||||||
if (basePath != null) {
|
|
||||||
new File(basePath).delete();
|
|
||||||
}
|
|
||||||
if (jsc != null) {
|
|
||||||
jsc.stop();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private HoodieWriteConfig getConfig() {
|
private HoodieWriteConfig getConfig() {
|
||||||
return getConfigBuilder().build();
|
return getConfigBuilder().build();
|
||||||
}
|
}
|
||||||
|
|
||||||
private HoodieWriteConfig.Builder getConfigBuilder() {
|
private HoodieWriteConfig.Builder getConfigBuilder() {
|
||||||
return HoodieWriteConfig.newBuilder().withPath(basePath)
|
return HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||||
.withCompactionConfig(
|
.withCompactionConfig(
|
||||||
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024)
|
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024)
|
||||||
.withInlineCompaction(false).build())
|
.withInlineCompaction(false).build())
|
||||||
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
|
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
|
||||||
.forTable("test-trip-table").withIndexConfig(
|
.forTable("test-trip-table").withIndexConfig(
|
||||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build());
|
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(expected = IllegalArgumentException.class)
|
@Test(expected = IllegalArgumentException.class)
|
||||||
public void testCompactionOnCopyOnWriteFail() throws Exception {
|
public void testCompactionOnCopyOnWriteFail() throws Exception {
|
||||||
HoodieTestUtils.initTableType(basePath, HoodieTableType.COPY_ON_WRITE);
|
HoodieTestUtils.initTableType(basePath, HoodieTableType.COPY_ON_WRITE);
|
||||||
|
|
||||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
|
||||||
|
|
||||||
|
compactor.compact(jsc, getConfig(), table);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCompactionEmpty() throws Exception {
|
||||||
|
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
||||||
|
HoodieWriteConfig config = getConfig();
|
||||||
|
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
|
||||||
|
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
||||||
|
|
||||||
|
String newCommitTime = writeClient.startCommit();
|
||||||
|
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||||
|
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||||
|
writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||||
|
|
||||||
|
HoodieCompactionMetadata result =
|
||||||
compactor.compact(jsc, getConfig(), table);
|
compactor.compact(jsc, getConfig(), table);
|
||||||
|
String basePath = table.getMetaClient().getBasePath();
|
||||||
|
assertTrue("If there is nothing to compact, result will be empty",
|
||||||
|
result.getFileIdAndFullPaths(basePath).isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testLogFileCountsAfterCompaction() throws Exception {
|
||||||
|
FileSystem fs = FSUtils.getFs();
|
||||||
|
// insert 100 records
|
||||||
|
HoodieWriteConfig config = getConfig();
|
||||||
|
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
||||||
|
String newCommitTime = "100";
|
||||||
|
writeClient.startCommitWithTime(newCommitTime);
|
||||||
|
|
||||||
|
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||||
|
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||||
|
List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||||
|
|
||||||
|
// Update all the 100 records
|
||||||
|
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||||
|
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
|
||||||
|
|
||||||
|
newCommitTime = "101";
|
||||||
|
writeClient.startCommitWithTime(newCommitTime);
|
||||||
|
|
||||||
|
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
||||||
|
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
||||||
|
HoodieIndex index = new HoodieBloomIndex<>(config, jsc);
|
||||||
|
updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect();
|
||||||
|
|
||||||
|
// Write them to corresponding avro logfiles
|
||||||
|
HoodieTestUtils
|
||||||
|
.writeRecordsToLogFiles(metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema,
|
||||||
|
updatedRecords);
|
||||||
|
|
||||||
|
// Verify that all data file has one log file
|
||||||
|
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||||
|
table = HoodieTable.getHoodieTable(metaClient, config);
|
||||||
|
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||||
|
List<FileSlice> groupedLogFiles =
|
||||||
|
table.getRTFileSystemView().getLatestFileSlices(partitionPath)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
for (FileSlice fileSlice : groupedLogFiles) {
|
||||||
|
assertEquals("There should be 1 log file written for every data file", 1,
|
||||||
|
fileSlice.getLogFiles().count());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
// Do a compaction
|
||||||
public void testCompactionEmpty() throws Exception {
|
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
|
table = HoodieTable.getHoodieTable(metaClient, config);
|
||||||
HoodieWriteConfig config = getConfig();
|
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
|
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
|
||||||
|
|
||||||
String newCommitTime = writeClient.startCommit();
|
HoodieCompactionMetadata result =
|
||||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
compactor.compact(jsc, getConfig(), table);
|
||||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
|
||||||
writeClient.insert(recordsRDD, newCommitTime).collect();
|
|
||||||
|
|
||||||
HoodieCompactionMetadata result =
|
// Verify that recently written compacted data file has no log file
|
||||||
compactor.compact(jsc, getConfig(), table);
|
metaClient = new HoodieTableMetaClient(fs, basePath);
|
||||||
String basePath = table.getMetaClient().getBasePath();
|
table = HoodieTable.getHoodieTable(metaClient, config);
|
||||||
assertTrue("If there is nothing to compact, result will be empty",
|
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
||||||
result.getFileIdAndFullPaths(basePath).isEmpty());
|
|
||||||
|
assertTrue("Compaction commit should be > than last insert",
|
||||||
|
HoodieTimeline.compareTimestamps(timeline.lastInstant().get().getTimestamp(), newCommitTime,
|
||||||
|
HoodieTimeline.GREATER));
|
||||||
|
|
||||||
|
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||||
|
List<FileSlice> groupedLogFiles = table.getRTFileSystemView()
|
||||||
|
.getLatestFileSlices(partitionPath)
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
for (FileSlice slice : groupedLogFiles) {
|
||||||
|
assertTrue(
|
||||||
|
"After compaction there should be no log files visiable on a Realtime view",
|
||||||
|
slice.getLogFiles().collect(Collectors.toList()).isEmpty());
|
||||||
|
}
|
||||||
|
assertTrue(result.getPartitionToCompactionWriteStats().containsKey(partitionPath));
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
// TODO - after modifying HoodieReadClient to support realtime tables - add more tests to make sure the data read is the updated data (compaction correctness)
|
||||||
public void testLogFileCountsAfterCompaction() throws Exception {
|
// TODO - add more test cases for compactions after a failed commit/compaction
|
||||||
FileSystem fs = FSUtils.getFs();
|
|
||||||
// insert 100 records
|
|
||||||
HoodieWriteConfig config = getConfig();
|
|
||||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
|
||||||
String newCommitTime = "100";
|
|
||||||
writeClient.startCommitWithTime(newCommitTime);
|
|
||||||
|
|
||||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
|
||||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
|
||||||
List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect();
|
|
||||||
|
|
||||||
// Update all the 100 records
|
|
||||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
|
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
|
|
||||||
|
|
||||||
newCommitTime = "101";
|
|
||||||
writeClient.startCommitWithTime(newCommitTime);
|
|
||||||
|
|
||||||
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
|
||||||
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
|
||||||
HoodieIndex index = new HoodieBloomIndex<>(config, jsc);
|
|
||||||
updatedRecords = index.tagLocation(updatedRecordsRDD, table).collect();
|
|
||||||
|
|
||||||
// Write them to corresponding avro logfiles
|
|
||||||
HoodieTestUtils
|
|
||||||
.writeRecordsToLogFiles(metaClient.getBasePath(), HoodieTestDataGenerator.avroSchema,
|
|
||||||
updatedRecords);
|
|
||||||
|
|
||||||
// Verify that all data file has one log file
|
|
||||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
|
||||||
table = HoodieTable.getHoodieTable(metaClient, config);
|
|
||||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
|
||||||
List<FileSlice> groupedLogFiles =
|
|
||||||
table.getRTFileSystemView().getLatestFileSlices(partitionPath)
|
|
||||||
.collect(Collectors.toList());
|
|
||||||
for (FileSlice fileSlice : groupedLogFiles) {
|
|
||||||
assertEquals("There should be 1 log file written for every data file", 1,
|
|
||||||
fileSlice.getLogFiles().count());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Do a compaction
|
|
||||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
|
||||||
table = HoodieTable.getHoodieTable(metaClient, config);
|
|
||||||
|
|
||||||
HoodieCompactionMetadata result =
|
|
||||||
compactor.compact(jsc, getConfig(), table);
|
|
||||||
|
|
||||||
// Verify that recently written compacted data file has no log file
|
|
||||||
metaClient = new HoodieTableMetaClient(fs, basePath);
|
|
||||||
table = HoodieTable.getHoodieTable(metaClient, config);
|
|
||||||
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
|
||||||
|
|
||||||
assertTrue("Compaction commit should be > than last insert",
|
|
||||||
HoodieTimeline.compareTimestamps(timeline.lastInstant().get().getTimestamp(), newCommitTime,
|
|
||||||
HoodieTimeline.GREATER));
|
|
||||||
|
|
||||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
|
||||||
List<FileSlice> groupedLogFiles = table.getRTFileSystemView()
|
|
||||||
.getLatestFileSlices(partitionPath)
|
|
||||||
.collect(Collectors.toList());
|
|
||||||
for (FileSlice slice: groupedLogFiles) {
|
|
||||||
assertTrue(
|
|
||||||
"After compaction there should be no log files visiable on a Realtime view",
|
|
||||||
slice.getLogFiles().collect(Collectors.toList()).isEmpty());
|
|
||||||
}
|
|
||||||
assertTrue(result.getPartitionToCompactionWriteStats().containsKey(partitionPath));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// TODO - after modifying HoodieReadClient to support realtime tables - add more tests to make sure the data read is the updated data (compaction correctness)
|
|
||||||
// TODO - add more test cases for compactions after a failed commit/compaction
|
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -17,12 +17,10 @@
|
|||||||
package com.uber.hoodie.io.strategy;
|
package com.uber.hoodie.io.strategy;
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertThat;
|
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
import com.beust.jcommander.internal.Lists;
|
import com.beust.jcommander.internal.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
|
|
||||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.io.compact.CompactionOperation;
|
import com.uber.hoodie.io.compact.CompactionOperation;
|
||||||
|
|||||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user