1
0

HUDI-123 Rename code packages/constants to org.apache.hudi (#830)

- Rename com.uber.hoodie to org.apache.hudi
- Flag to pass com.uber.hoodie Input formats for hoodie-sync
- Works with HUDI demo. 
- Also tested for backwards compatibility with datasets built by com.uber.hoodie packages
- Migration guide : https://cwiki.apache.org/confluence/display/HUDI/Migration+Guide+From+com.uber.hoodie+to+org.apache.hudi
This commit is contained in:
Balaji Varadarajan
2019-08-11 17:48:17 -07:00
committed by vinoth chandar
parent 722b6be04a
commit a4f9d7575f
546 changed files with 3858 additions and 3562 deletions

View File

@@ -0,0 +1,128 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter;
import java.util.List;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.HoodieWriteClient;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex.IndexType;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
/**
* Driver program that uses the Hoodie client with synthetic workload, and performs basic operations. <p>
*/
public class HoodieClientExample {
private static Logger logger = LogManager.getLogger(HoodieClientExample.class);
@Parameter(names = {"--help", "-h"}, help = true)
public Boolean help = false;
@Parameter(names = {"--table-path", "-p"}, description = "path for Hoodie sample table")
private String tablePath = "file:///tmp/hoodie/sample-table";
@Parameter(names = {"--table-name", "-n"}, description = "table name for Hoodie sample table")
private String tableName = "hoodie_rt";
@Parameter(names = {"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ")
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
public static void main(String[] args) throws Exception {
HoodieClientExample cli = new HoodieClientExample();
JCommander cmd = new JCommander(cli, args);
if (cli.help) {
cmd.usage();
System.exit(1);
}
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(tablePath, jsc.hadoopConfiguration());
if (!fs.exists(path)) {
HoodieTableMetaClient
.initTableType(jsc.hadoopConfiguration(), 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(IndexType.BLOOM).build())
.withCompactionConfig(
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 3).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);
/**
* Schedule a compaction and also perform compaction on a MOR dataset
*/
if (HoodieTableType.valueOf(tableType) == HoodieTableType.MERGE_ON_READ) {
Option<String> instant = client.scheduleCompaction(Option.empty());
JavaRDD<WriteStatus> writeStatues = client.compact(instant.get());
client.commitCompaction(instant.get(), writeStatues, Option.empty());
}
}
}

View File

@@ -0,0 +1,581 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi;
import static org.apache.hudi.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.util.AvroUtils;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaRDD;
import org.junit.Assert;
import org.junit.Test;
/**
* Test Cases for Async Compaction and Ingestion interaction
*/
public class TestAsyncCompaction extends TestHoodieClientBase {
private HoodieWriteConfig getConfig(Boolean autoCommit) {
return getConfigBuilder(autoCommit).build();
}
protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true).withCompactionConfig(
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false)
.withMaxNumDeltaCommitsBeforeCompaction(1).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build())
.forTable("test-trip-table")
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
.build());
}
@Override
public void tearDown() throws IOException {
super.tearDown();
}
@Test
public void testRollbackForInflightCompaction() throws Exception {
// Rollback inflight compaction
HoodieWriteConfig cfg = getConfig(false);
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
String firstInstantTime = "001";
String secondInstantTime = "004";
String compactionInstantTime = "005";
int numRecs = 2000;
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
records, cfg, true, new ArrayList<>());
// Schedule compaction but do not run them
scheduleCompaction(compactionInstantTime, client, cfg);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieInstant pendingCompactionInstant =
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
assertTrue("Pending Compaction instant has expected instant time",
pendingCompactionInstant.getTimestamp().equals(compactionInstantTime));
assertTrue("Pending Compaction instant has expected state",
pendingCompactionInstant.getState().equals(State.REQUESTED));
moveCompactionFromRequestedToInflight(compactionInstantTime, client, cfg);
// Reload and rollback inflight compaction
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
hoodieTable.rollback(jsc, compactionInstantTime, false);
client.rollbackInflightCompaction(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), hoodieTable);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
pendingCompactionInstant = metaClient.getCommitsAndCompactionTimeline().filterPendingCompactionTimeline()
.getInstants().findFirst().get();
assertEquals("compaction", pendingCompactionInstant.getAction());
assertEquals(State.REQUESTED, pendingCompactionInstant.getState());
assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp());
// We indirectly test for the race condition where a inflight instant was first deleted then created new. Every
// time this happens, the pending compaction instant file in Hoodie Meta path becomes an empty file (Note: Hoodie
// reads compaction plan from aux path which is untouched). TO test for regression, we simply get file status
// and look at the file size
FileStatus fstatus =
metaClient.getFs().getFileStatus(new Path(metaClient.getMetaPath(), pendingCompactionInstant.getFileName()));
assertTrue(fstatus.getLen() > 0);
}
private Path getInstantPath(HoodieTableMetaClient metaClient, String timestamp, String action, State state) {
HoodieInstant instant = new HoodieInstant(state, action, timestamp);
return new Path(metaClient.getMetaPath(), instant.getFileName());
}
@Test
public void testRollbackInflightIngestionWithPendingCompaction() throws Exception {
// Rollback inflight ingestion when there is pending compaction
HoodieWriteConfig cfg = getConfig(false);
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
String firstInstantTime = "001";
String secondInstantTime = "004";
String compactionInstantTime = "005";
String inflightInstantTime = "006";
String nextInflightInstantTime = "007";
int numRecs = 2000;
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
records, cfg, true, new ArrayList<>());
// Schedule compaction but do not run them
scheduleCompaction(compactionInstantTime, client, cfg);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
createNextDeltaCommit(inflightInstantTime, records, client, metaClient, cfg, true);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieInstant pendingCompactionInstant =
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
assertTrue("Pending Compaction instant has expected instant time",
pendingCompactionInstant.getTimestamp().equals(compactionInstantTime));
HoodieInstant inflightInstant =
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
assertTrue("inflight instant has expected instant time",
inflightInstant.getTimestamp().equals(inflightInstantTime));
//This should rollback
client.startCommitWithTime(nextInflightInstantTime);
//Validate
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
inflightInstant =
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
assertTrue("inflight instant has expected instant time",
inflightInstant.getTimestamp().equals(nextInflightInstantTime));
assertTrue("Expect only one inflight instant",
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().getInstants().count() == 1);
//Expect pending Compaction to be present
pendingCompactionInstant =
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
assertTrue("Pending Compaction instant has expected instant time",
pendingCompactionInstant.getTimestamp().equals(compactionInstantTime));
}
@Test
public void testInflightCompaction() throws Exception {
// There is inflight compaction. Subsequent compaction run must work correctly
HoodieWriteConfig cfg = getConfig(true);
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
String firstInstantTime = "001";
String secondInstantTime = "004";
String compactionInstantTime = "005";
String thirdInstantTime = "006";
String fourthInstantTime = "007";
int numRecs = 2000;
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
records, cfg, true, new ArrayList<>());
// Schedule and mark compaction instant as inflight
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
scheduleCompaction(compactionInstantTime, client, cfg);
moveCompactionFromRequestedToInflight(compactionInstantTime, client, cfg);
// Complete ingestions
runNextDeltaCommits(client, Arrays.asList(thirdInstantTime, fourthInstantTime),
records, cfg, false, Arrays.asList(compactionInstantTime));
// execute inflight compaction
executeCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, true);
}
@Test
public void testScheduleIngestionBeforePendingCompaction() throws Exception {
// Case: Failure case. Latest pending compaction instant time must be earlier than this instant time
HoodieWriteConfig cfg = getConfig(false);
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
String firstInstantTime = "001";
String secondInstantTime = "004";
String failedInstantTime = "005";
String compactionInstantTime = "006";
int numRecs = 2000;
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
records, cfg, true, new ArrayList<>());
// Schedule compaction but do not run them
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
scheduleCompaction(compactionInstantTime, client, cfg);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieInstant pendingCompactionInstant =
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
assertTrue("Pending Compaction instant has expected instant time",
pendingCompactionInstant.getTimestamp().equals(compactionInstantTime));
boolean gotException = false;
try {
runNextDeltaCommits(client, Arrays.asList(failedInstantTime),
records, cfg, false, Arrays.asList(compactionInstantTime));
} catch (IllegalArgumentException iex) {
// Latest pending compaction instant time must be earlier than this instant time. Should fail here
gotException = true;
}
assertTrue("Latest pending compaction instant time must be earlier than this instant time", gotException);
}
@Test
public void testScheduleCompactionAfterPendingIngestion() throws Exception {
// Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time
HoodieWriteConfig cfg = getConfig(false);
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
String firstInstantTime = "001";
String secondInstantTime = "004";
String inflightInstantTime = "005";
String compactionInstantTime = "006";
int numRecs = 2000;
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
records, cfg, true, new ArrayList<>());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
createNextDeltaCommit(inflightInstantTime, records, client, metaClient, cfg, true);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieInstant inflightInstant =
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
assertTrue("inflight instant has expected instant time",
inflightInstant.getTimestamp().equals(inflightInstantTime));
boolean gotException = false;
try {
// Schedule compaction but do not run them
scheduleCompaction(compactionInstantTime, client, cfg);
} catch (IllegalArgumentException iex) {
// Earliest ingestion inflight instant time must be later than compaction time. Should fail here
gotException = true;
}
assertTrue("Earliest ingestion inflight instant time must be later than compaction time", gotException);
}
@Test
public void testScheduleCompactionWithOlderOrSameTimestamp() throws Exception {
// Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time
HoodieWriteConfig cfg = getConfig(false);
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
String firstInstantTime = "001";
String secondInstantTime = "004";
String compactionInstantTime = "002";
int numRecs = 2000;
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
records, cfg, true, new ArrayList<>());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
boolean gotException = false;
try {
// Schedule compaction but do not run them
scheduleCompaction(compactionInstantTime, client, cfg);
} catch (IllegalArgumentException iex) {
gotException = true;
}
assertTrue("Compaction Instant to be scheduled cannot have older timestamp", gotException);
// Schedule with timestamp same as that of committed instant
gotException = false;
String dupCompactionInstantTime = secondInstantTime;
try {
// Schedule compaction but do not run them
scheduleCompaction(dupCompactionInstantTime, client, cfg);
} catch (IllegalArgumentException iex) {
gotException = true;
}
assertTrue("Compaction Instant to be scheduled cannot have same timestamp as committed instant",
gotException);
compactionInstantTime = "006";
scheduleCompaction(compactionInstantTime, client, cfg);
gotException = false;
try {
// Schedule compaction with the same times as a pending compaction
scheduleCompaction(dupCompactionInstantTime, client, cfg);
} catch (IllegalArgumentException iex) {
gotException = true;
}
assertTrue("Compaction Instant to be scheduled cannot have same timestamp as a pending compaction",
gotException);
}
@Test
public void testCompactionAfterTwoDeltaCommits() throws Exception {
// No Delta Commits after compaction request
HoodieWriteConfig cfg = getConfig(true);
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
String firstInstantTime = "001";
String secondInstantTime = "004";
String compactionInstantTime = "005";
int numRecs = 2000;
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
records, cfg, true, new ArrayList<>());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
scheduleAndExecuteCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, false);
}
@Test
public void testInterleavedCompaction() throws Exception {
//Case: Two delta commits before and after compaction schedule
HoodieWriteConfig cfg = getConfig(true);
HoodieWriteClient client = getHoodieWriteClient(cfg, true);
String firstInstantTime = "001";
String secondInstantTime = "004";
String compactionInstantTime = "005";
String thirdInstantTime = "006";
String fourthInstantTime = "007";
int numRecs = 2000;
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
records = runNextDeltaCommits(client, Arrays.asList(firstInstantTime, secondInstantTime),
records, cfg, true, new ArrayList<>());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
scheduleCompaction(compactionInstantTime, client, cfg);
runNextDeltaCommits(client, Arrays.asList(thirdInstantTime, fourthInstantTime),
records, cfg, false, Arrays.asList(compactionInstantTime));
executeCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, true);
}
/**
* HELPER METHODS FOR TESTING
**/
private void validateDeltaCommit(String latestDeltaCommit,
final Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToCompactionOperation,
HoodieWriteConfig cfg) throws IOException {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable table = getHoodieTable(metaClient, cfg);
List<FileSlice> fileSliceList = getCurrentLatestFileSlices(table, cfg);
fileSliceList.forEach(fileSlice -> {
Pair<String, HoodieCompactionOperation> opPair = fgIdToCompactionOperation.get(fileSlice.getFileGroupId());
if (opPair != null) {
assertTrue("Expect baseInstant to match compaction Instant",
fileSlice.getBaseInstantTime().equals(opPair.getKey()));
assertTrue("Expect atleast one log file to be present where the latest delta commit was written",
fileSlice.getLogFiles().count() > 0);
assertFalse("Expect no data-file to be present", fileSlice.getDataFile().isPresent());
} else {
assertTrue("Expect baseInstant to be less than or equal to latestDeltaCommit",
fileSlice.getBaseInstantTime().compareTo(latestDeltaCommit) <= 0);
}
});
}
private List<HoodieRecord> runNextDeltaCommits(HoodieWriteClient client, List<String> deltaInstants,
List<HoodieRecord> records, HoodieWriteConfig cfg, boolean insertFirst,
List<String> expPendingCompactionInstants) throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
List<Pair<HoodieInstant, HoodieCompactionPlan>> pendingCompactions =
CompactionUtils.getAllPendingCompactionPlans(metaClient);
List<String> gotPendingCompactionInstants =
pendingCompactions.stream().map(pc -> pc.getKey().getTimestamp()).sorted().collect(Collectors.toList());
assertEquals(expPendingCompactionInstants, gotPendingCompactionInstants);
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToCompactionOperation =
CompactionUtils.getAllPendingCompactionOperations(metaClient);
if (insertFirst) {
// Use first instant for inserting records
String firstInstant = deltaInstants.get(0);
deltaInstants = deltaInstants.subList(1, deltaInstants.size());
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
client.startCommitWithTime(firstInstant);
JavaRDD<WriteStatus> statuses = client.upsert(writeRecords, firstInstant);
List<WriteStatus> statusList = statuses.collect();
if (!cfg.shouldAutoCommit()) {
client.commit(firstInstant, statuses);
}
assertNoWriteErrors(statusList);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
List<HoodieDataFile> dataFilesToRead = getCurrentLatestDataFiles(hoodieTable, cfg);
assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit",
dataFilesToRead.stream().findAny().isPresent());
validateDeltaCommit(firstInstant, fgIdToCompactionOperation, cfg);
}
int numRecords = records.size();
for (String instantTime : deltaInstants) {
records = dataGen.generateUpdates(instantTime, numRecords);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
createNextDeltaCommit(instantTime, records, client, metaClient, cfg, false);
validateDeltaCommit(instantTime, fgIdToCompactionOperation, cfg);
}
return records;
}
private void moveCompactionFromRequestedToInflight(String compactionInstantTime, HoodieWriteClient client,
HoodieWriteConfig cfg) throws IOException {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieInstant compactionInstant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
HoodieCompactionPlan workload = AvroUtils.deserializeCompactionPlan(
metaClient.getActiveTimeline().getInstantAuxiliaryDetails(compactionInstant).get());
metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(compactionInstant);
HoodieInstant instant = metaClient.getActiveTimeline().reload().filterPendingCompactionTimeline().getInstants()
.filter(in -> in.getTimestamp().equals(compactionInstantTime)).findAny().get();
assertTrue("Instant must be marked inflight", instant.isInflight());
}
private void scheduleCompaction(String compactionInstantTime, HoodieWriteClient client, HoodieWriteConfig cfg)
throws IOException {
client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieInstant instant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().get();
assertEquals("Last compaction instant must be the one set",
instant.getTimestamp(), compactionInstantTime);
}
private void scheduleAndExecuteCompaction(String compactionInstantTime,
HoodieWriteClient client, HoodieTable table, HoodieWriteConfig cfg, int expectedNumRecs,
boolean hasDeltaCommitAfterPendingCompaction) throws IOException {
scheduleCompaction(compactionInstantTime, client, cfg);
executeCompaction(compactionInstantTime, client, table, cfg, expectedNumRecs, hasDeltaCommitAfterPendingCompaction);
}
private void executeCompaction(String compactionInstantTime,
HoodieWriteClient client, HoodieTable table, HoodieWriteConfig cfg, int expectedNumRecs,
boolean hasDeltaCommitAfterPendingCompaction) throws IOException {
client.compact(compactionInstantTime);
List<FileSlice> fileSliceList = getCurrentLatestFileSlices(table, cfg);
assertTrue("Ensure latest file-slices are not empty", fileSliceList.stream().findAny().isPresent());
assertFalse("Verify all file-slices have base-instant same as compaction instant",
fileSliceList.stream().filter(fs -> !fs.getBaseInstantTime().equals(compactionInstantTime))
.findAny().isPresent());
assertFalse("Verify all file-slices have data-files",
fileSliceList.stream().filter(fs -> !fs.getDataFile().isPresent()).findAny().isPresent());
if (hasDeltaCommitAfterPendingCompaction) {
assertFalse("Verify all file-slices have atleast one log-file",
fileSliceList.stream().filter(fs -> fs.getLogFiles().count() == 0).findAny().isPresent());
} else {
assertFalse("Verify all file-slices have no log-files",
fileSliceList.stream().filter(fs -> fs.getLogFiles().count() > 0).findAny().isPresent());
}
// verify that there is a commit
table = getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), cfg);
HoodieTimeline timeline = table.getMetaClient().getCommitTimeline().filterCompletedInstants();
String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp();
assertEquals("Expect compaction instant time to be the latest commit time",
latestCompactionCommitTime, compactionInstantTime);
Assert.assertEquals("Must contain expected records", expectedNumRecs,
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, "000").count());
}
private List<WriteStatus> createNextDeltaCommit(String instantTime, List<HoodieRecord> records,
HoodieWriteClient client, HoodieTableMetaClient metaClient, HoodieWriteConfig cfg, boolean skipCommit) {
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
client.startCommitWithTime(instantTime);
JavaRDD<WriteStatus> statuses = client.upsert(writeRecords, instantTime);
List<WriteStatus> statusList = statuses.collect();
assertNoWriteErrors(statusList);
if (!cfg.shouldAutoCommit() && !skipCommit) {
client.commit(instantTime, statuses);
}
Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().reload().getDeltaCommitTimeline()
.filterCompletedInstants().lastInstant();
if (skipCommit && !cfg.shouldAutoCommit()) {
assertTrue("Delta commit should not be latest instant",
deltaCommit.get().getTimestamp().compareTo(instantTime) < 0);
} else {
assertTrue(deltaCommit.isPresent());
assertEquals("Delta commit should be latest instant", instantTime, deltaCommit.get().getTimestamp());
}
return statusList;
}
private List<HoodieDataFile> getCurrentLatestDataFiles(HoodieTable table, HoodieWriteConfig cfg) throws IOException {
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(table.getMetaClient().getFs(), cfg.getBasePath());
HoodieTableFileSystemView
view = new HoodieTableFileSystemView(table.getMetaClient(), table.getCompletedCommitsTimeline(), allFiles);
List<HoodieDataFile> dataFilesToRead = view.getLatestDataFiles().collect(Collectors.toList());
return dataFilesToRead;
}
private List<FileSlice> getCurrentLatestFileSlices(HoodieTable table, HoodieWriteConfig cfg) throws IOException {
HoodieTableFileSystemView view = new HoodieTableFileSystemView(table.getMetaClient(),
table.getMetaClient().getActiveTimeline().reload().getCommitsAndCompactionTimeline());
List<FileSlice> fileSliceList =
Arrays.asList(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS).stream().flatMap(partition ->
view.getLatestFileSlices(partition)).collect(Collectors.toList());
return fileSliceList;
}
protected HoodieTableType getTableType() {
return HoodieTableType.MERGE_ON_READ;
}
}

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,328 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieRollbackException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaRDD;
import org.junit.Test;
/**
* Test Cases for rollback of snapshots and commits
*/
public class TestClientRollback extends TestHoodieClientBase {
@Override
public void tearDown() throws IOException {
super.tearDown();
}
/**
* Test case for rollback-savepoint interaction
*/
@Test
public void testSavepointAndRollback() throws Exception {
HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(
HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1)
.build()).build();
HoodieWriteClient client = getHoodieWriteClient(cfg);
HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
/**
* Write 1 (only inserts)
*/
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
/**
* Write 2 (updates)
*/
newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, records);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
client.savepoint("hoodie-unit-test", "test");
/**
* Write 3 (updates)
*/
newCommitTime = "003";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, records);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
List<String> partitionPaths = FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(),
getConfig().shouldAssumeDatePartitioning());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
final ReadOptimizedView view1 = table.getROFileSystemView();
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003"));
}).collect(Collectors.toList());
assertEquals("The data files for commit 003 should be present", 3, dataFiles.size());
dataFiles = partitionPaths.stream().flatMap(s -> {
return view1.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
}).collect(Collectors.toList());
assertEquals("The data files for commit 002 should be present", 3, dataFiles.size());
/**
* Write 4 (updates)
*/
newCommitTime = "004";
client.startCommitWithTime(newCommitTime);
records = dataGen.generateUpdates(newCommitTime, records);
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
final ReadOptimizedView view2 = table.getROFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> {
return view2.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004"));
}).collect(Collectors.toList());
assertEquals("The data files for commit 004 should be present", 3, dataFiles.size());
// rolling back to a non existent savepoint must not succeed
try {
client.rollbackToSavepoint("001");
fail("Rolling back to non-existent savepoint should not be allowed");
} catch (HoodieRollbackException e) {
// this is good
}
// rollback to savepoint 002
HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstants().findFirst().get();
client.rollbackToSavepoint(savepoint.getTimestamp());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
final ReadOptimizedView view3 = table.getROFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> {
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
}).collect(Collectors.toList());
assertEquals("The data files for commit 002 be available", 3, dataFiles.size());
dataFiles = partitionPaths.stream().flatMap(s -> {
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("003"));
}).collect(Collectors.toList());
assertEquals("The data files for commit 003 should be rolled back", 0, dataFiles.size());
dataFiles = partitionPaths.stream().flatMap(s -> {
return view3.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("004"));
}).collect(Collectors.toList());
assertEquals("The data files for commit 004 should be rolled back", 0, dataFiles.size());
}
/**
* Test Cases for effects of rollbacking completed/inflight commits
*/
@Test
public void testRollbackCommit() throws Exception {
// Let's create some commit files and parquet files
String commitTime1 = "20160501010101";
String commitTime2 = "20160502020601";
String commitTime3 = "20160506030611";
new File(basePath + "/.hoodie").mkdirs();
HoodieTestDataGenerator
.writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, basePath);
// Only first two have commit files
HoodieTestUtils.createCommitFiles(basePath, commitTime1, commitTime2);
// Third one has a .inflight intermediate commit file
HoodieTestUtils.createInflightCommitFiles(basePath, commitTime3);
// Make commit1
String file11 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime1, "id11");
String file12 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime1, "id12");
String file13 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime1, "id13");
// Make commit2
String file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21");
String file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22");
String file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23");
// Make commit3
String file31 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime3, "id31");
String file32 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime3, "id32");
String file33 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime3, "id33");
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
HoodieWriteClient client = getHoodieWriteClient(config, false);
// Rollback commit 1 (this should fail, since commit2 is still around)
try {
client.rollback(commitTime1);
assertTrue("Should have thrown an exception ", false);
} catch (HoodieRollbackException hrbe) {
// should get here
}
// Rollback commit3
client.rollback(commitTime3);
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
// simulate partial failure, where .inflight was not deleted, but data files were.
HoodieTestUtils.createInflightCommitFiles(basePath, commitTime3);
client.rollback(commitTime3);
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
// Rollback commit2
client.rollback(commitTime2);
assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2));
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
// simulate partial failure, where only .commit => .inflight renaming succeeded, leaving a
// .inflight commit and a bunch of data files around.
HoodieTestUtils.createInflightCommitFiles(basePath, commitTime2);
file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21");
file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22");
file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23");
client.rollback(commitTime2);
assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime2));
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
// Let's rollback commit1, Check results
client.rollback(commitTime1);
assertFalse(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime1));
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
}
/**
* Test auto-rollback of commits which are in flight
*/
@Test
public void testAutoRollbackInflightCommit() throws Exception {
// Let's create some commit files and parquet files
String commitTime1 = "20160501010101";
String commitTime2 = "20160502020601";
String commitTime3 = "20160506030611";
new File(basePath + "/.hoodie").mkdirs();
HoodieTestDataGenerator
.writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"}, basePath);
// One good commit
HoodieTestUtils.createCommitFiles(basePath, commitTime1);
// Two inflight commits
HoodieTestUtils.createInflightCommitFiles(basePath, commitTime2, commitTime3);
// Make commit1
String file11 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime1, "id11");
String file12 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime1, "id12");
String file13 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime1, "id13");
// Make commit2
String file21 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime2, "id21");
String file22 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime2, "id22");
String file23 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime2, "id23");
// Make commit3
String file31 = HoodieTestUtils.createDataFile(basePath, "2016/05/01", commitTime3, "id31");
String file32 = HoodieTestUtils.createDataFile(basePath, "2016/05/02", commitTime3, "id32");
String file33 = HoodieTestUtils.createDataFile(basePath, "2016/05/06", commitTime3, "id33");
// Turn auto rollback off
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
getHoodieWriteClient(config, false);
// Check results, nothing changed
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
// Turn auto rollback on
getHoodieWriteClient(config, true).startCommit();
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime3, file31)
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime3, file32)
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime3, file33));
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime2, file21)
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime2, file22)
|| HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime2, file23));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, "2016/05/01", commitTime1, file11)
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/02", commitTime1, file12)
&& HoodieTestUtils.doesDataFileExist(basePath, "2016/05/06", commitTime1, file13));
}
}

View File

@@ -0,0 +1,364 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi;
import static org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.hudi.CompactionAdminClient.ValidationOpResult;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.util.CompactionTestUtils;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class TestCompactionAdminClient extends TestHoodieClientBase {
private HoodieTableMetaClient metaClient;
private CompactionAdminClient client;
@Before
public void init() throws IOException {
super.init();
metaClient = HoodieTestUtils.initTableType(HoodieTestUtils.getDefaultHadoopConf(), basePath, MERGE_ON_READ);
client = new CompactionAdminClient(jsc, basePath);
}
@Override
public void tearDown() throws IOException {
super.tearDown();
}
@Test
public void testUnscheduleCompactionPlan() throws Exception {
int numEntriesPerInstant = 10;
CompactionTestUtils
.setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant, numEntriesPerInstant,
numEntriesPerInstant, numEntriesPerInstant);
// THere are delta-commits after compaction instant
validateUnSchedulePlan(client,
"000", "001", numEntriesPerInstant, 2 * numEntriesPerInstant);
// THere are delta-commits after compaction instant
validateUnSchedulePlan(client,
"002", "003", numEntriesPerInstant, 2 * numEntriesPerInstant);
// THere are no delta-commits after compaction instant
validateUnSchedulePlan(client,
"004", "005", numEntriesPerInstant, 0);
// THere are no delta-commits after compaction instant
validateUnSchedulePlan(client,
"006", "007", numEntriesPerInstant, 0);
}
@Test
public void testUnscheduleCompactionFileId() throws Exception {
int numEntriesPerInstant = 10;
CompactionTestUtils
.setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant, numEntriesPerInstant,
numEntriesPerInstant, numEntriesPerInstant);
Map<String, CompactionOperation> instantsWithOp =
Arrays.asList("001", "003", "005", "007").stream().map(instant -> {
try {
return Pair.of(instant, CompactionUtils.getCompactionPlan(metaClient, instant));
} catch (IOException ioe) {
throw new HoodieException(ioe);
}
}).map(instantWithPlan -> instantWithPlan.getRight().getOperations().stream().map(op -> Pair.of(
instantWithPlan.getLeft(), CompactionOperation.convertFromAvroRecordInstance(op))).findFirst().get())
.collect(Collectors.toMap(Pair::getLeft, Pair::getRight));
// THere are delta-commits after compaction instant
validateUnScheduleFileId(client,
"000", "001", instantsWithOp.get("001"), 2);
// THere are delta-commits after compaction instant
validateUnScheduleFileId(client,
"002", "003", instantsWithOp.get("003"), 2);
// THere are no delta-commits after compaction instant
validateUnScheduleFileId(client,
"004", "005", instantsWithOp.get("005"), 0);
// THere are no delta-commits after compaction instant
validateUnScheduleFileId(client,
"006", "007", instantsWithOp.get("007"), 0);
}
@Test
public void testRepairCompactionPlan() throws Exception {
int numEntriesPerInstant = 10;
CompactionTestUtils
.setupAndValidateCompactionOperations(metaClient,false, numEntriesPerInstant, numEntriesPerInstant,
numEntriesPerInstant, numEntriesPerInstant);
// THere are delta-commits after compaction instant
validateRepair("000", "001", numEntriesPerInstant, 2 * numEntriesPerInstant);
// THere are delta-commits after compaction instant
validateRepair("002", "003", numEntriesPerInstant, 2 * numEntriesPerInstant);
// THere are no delta-commits after compaction instant
validateRepair("004", "005", numEntriesPerInstant, 0);
// THere are no delta-commits after compaction instant
validateRepair("006", "007", numEntriesPerInstant, 0);
}
private void validateRepair(String ingestionInstant, String compactionInstant, int numEntriesPerInstant,
int expNumRepairs) throws Exception {
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles =
validateUnSchedulePlan(client, ingestionInstant, compactionInstant, numEntriesPerInstant, expNumRepairs, true);
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
List<ValidationOpResult> result = client.validateCompactionPlan(metaClient, compactionInstant, 1);
if (expNumRepairs > 0) {
Assert.assertTrue("Expect some failures in validation", result.stream().filter(r -> !r.isSuccess()).count() > 0);
}
// Now repair
List<Pair<HoodieLogFile, HoodieLogFile>> undoFiles = result.stream().flatMap(r ->
client.getRenamingActionsToAlignWithCompactionOperation(metaClient,
compactionInstant, r.getOperation(), Option.empty()).stream())
.map(rn -> {
try {
client.renameLogFile(metaClient, rn.getKey(), rn.getValue());
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
return rn;
}).collect(Collectors.toList());
Map<String, String> renameFilesFromUndo =
undoFiles.stream().collect(Collectors.toMap(p -> p.getRight().getPath().toString(),
x -> x.getLeft().getPath().toString()));
Map<String, String> expRenameFiles =
renameFiles.stream().collect(Collectors.toMap(p -> p.getLeft().getPath().toString(),
x -> x.getRight().getPath().toString()));
if (expNumRepairs > 0) {
Assert.assertFalse("Rename Files must be non-empty", renameFiles.isEmpty());
} else {
Assert.assertTrue("Rename Files must be empty", renameFiles.isEmpty());
}
expRenameFiles.entrySet().stream().forEach(r -> {
logger.info("Key :" + r.getKey() + " renamed to " + r.getValue() + " rolled back to "
+ renameFilesFromUndo.get(r.getKey()));
});
Assert.assertEquals("Undo must completely rollback renames", expRenameFiles, renameFilesFromUndo);
// Now expect validation to succeed
result = client.validateCompactionPlan(metaClient, compactionInstant, 1);
Assert.assertTrue("Expect no failures in validation", result.stream().filter(r -> !r.isSuccess()).count() == 0);
Assert.assertEquals("Expected Num Repairs", expNumRepairs, undoFiles.size());
}
/**
* Enssure compaction plan is valid
* @param compactionInstant Compaction Instant
* @throws Exception
*/
private void ensureValidCompactionPlan(String compactionInstant) throws Exception {
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
// Ensure compaction-plan is good to begin with
List<ValidationOpResult> validationResults = client.validateCompactionPlan(metaClient,
compactionInstant, 1);
Assert.assertFalse("Some validations failed",
validationResults.stream().filter(v -> !v.isSuccess()).findAny().isPresent());
}
private void validateRenameFiles(List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles,
String ingestionInstant, String compactionInstant, HoodieTableFileSystemView fsView) {
// Ensure new names of log-files are on expected lines
Set<HoodieLogFile> uniqNewLogFiles = new HashSet<>();
Set<HoodieLogFile> uniqOldLogFiles = new HashSet<>();
renameFiles.stream().forEach(lfPair -> {
Assert.assertFalse("Old Log File Names do not collide", uniqOldLogFiles.contains(lfPair.getKey()));
Assert.assertFalse("New Log File Names do not collide", uniqNewLogFiles.contains(lfPair.getValue()));
uniqOldLogFiles.add(lfPair.getKey());
uniqNewLogFiles.add(lfPair.getValue());
});
renameFiles.stream().forEach(lfPair -> {
HoodieLogFile oldLogFile = lfPair.getLeft();
HoodieLogFile newLogFile = lfPair.getValue();
Assert.assertEquals("Base Commit time is expected", ingestionInstant, newLogFile.getBaseCommitTime());
Assert.assertEquals("Base Commit time is expected", compactionInstant, oldLogFile.getBaseCommitTime());
Assert.assertEquals("File Id is expected", oldLogFile.getFileId(), newLogFile.getFileId());
HoodieLogFile lastLogFileBeforeCompaction =
fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], ingestionInstant)
.filter(fs -> fs.getFileId().equals(oldLogFile.getFileId()))
.map(fs -> fs.getLogFiles().findFirst().get()).findFirst().get();
Assert.assertEquals("Log Version expected",
lastLogFileBeforeCompaction.getLogVersion() + oldLogFile.getLogVersion(),
newLogFile.getLogVersion());
Assert.assertTrue("Log version does not collide",
newLogFile.getLogVersion() > lastLogFileBeforeCompaction.getLogVersion());
});
}
/**
* Validate Unschedule operations
*/
private List<Pair<HoodieLogFile, HoodieLogFile>> validateUnSchedulePlan(CompactionAdminClient client,
String ingestionInstant, String compactionInstant, int numEntriesPerInstant, int expNumRenames)
throws Exception {
return validateUnSchedulePlan(client, ingestionInstant, compactionInstant, numEntriesPerInstant,
expNumRenames, false);
}
/**
* Validate Unschedule operations
*/
private List<Pair<HoodieLogFile, HoodieLogFile>> validateUnSchedulePlan(CompactionAdminClient client,
String ingestionInstant, String compactionInstant, int numEntriesPerInstant, int expNumRenames,
boolean skipUnSchedule) throws Exception {
ensureValidCompactionPlan(compactionInstant);
// Check suggested rename operations
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles =
client.getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, 1,
Option.empty(), false);
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
// Log files belonging to file-slices created because of compaction request must be renamed
Set<HoodieLogFile> gotLogFilesToBeRenamed = renameFiles.stream().map(p -> p.getLeft()).collect(Collectors.toSet());
final HoodieTableFileSystemView fsView =
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
Set<HoodieLogFile> expLogFilesToBeRenamed = fsView.getLatestFileSlices(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0])
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant))
.flatMap(fs -> fs.getLogFiles())
.collect(Collectors.toSet());
Assert.assertEquals("Log files belonging to file-slices created because of compaction request must be renamed",
expLogFilesToBeRenamed, gotLogFilesToBeRenamed);
if (skipUnSchedule) {
// Do the renaming only but do not touch the compaction plan - Needed for repair tests
renameFiles.stream().forEach(lfPair -> {
try {
client.renameLogFile(metaClient, lfPair.getLeft(), lfPair.getRight());
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
});
} else {
validateRenameFiles(renameFiles, ingestionInstant, compactionInstant, fsView);
}
Map<String, Long> fileIdToCountsBeforeRenaming =
fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant)
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
// Call the main unschedule API
client.unscheduleCompactionPlan(compactionInstant, false, 1, false);
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
final HoodieTableFileSystemView newFsView =
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
// Expect all file-slice whose base-commit is same as compaction commit to contain no new Log files
newFsView.getLatestFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant, true)
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)).forEach(fs -> {
Assert.assertFalse("No Data file must be present", fs.getDataFile().isPresent());
Assert.assertTrue("No Log Files", fs.getLogFiles().count() == 0);
});
// Ensure same number of log-files before and after renaming per fileId
Map<String, Long> fileIdToCountsAfterRenaming =
newFsView.getAllFileGroups(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]).flatMap(fg -> fg.getAllFileSlices())
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
Assert.assertEquals("Each File Id has same number of log-files",
fileIdToCountsBeforeRenaming, fileIdToCountsAfterRenaming);
Assert.assertEquals("Not Empty", numEntriesPerInstant, fileIdToCountsAfterRenaming.size());
Assert.assertEquals("Expected number of renames", expNumRenames, renameFiles.size());
return renameFiles;
}
/**
* Validate Unschedule operations
*/
private void validateUnScheduleFileId(CompactionAdminClient client, String ingestionInstant,
String compactionInstant, CompactionOperation op, int expNumRenames) throws Exception {
ensureValidCompactionPlan(compactionInstant);
// Check suggested rename operations
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles =
client.getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, op,
Option.empty(), false);
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
// Log files belonging to file-slices created because of compaction request must be renamed
Set<HoodieLogFile> gotLogFilesToBeRenamed = renameFiles.stream().map(p -> p.getLeft()).collect(Collectors.toSet());
final HoodieTableFileSystemView fsView =
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
Set<HoodieLogFile> expLogFilesToBeRenamed = fsView.getLatestFileSlices(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0])
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant))
.filter(fs -> fs.getFileId().equals(op.getFileId()))
.flatMap(fs -> fs.getLogFiles())
.collect(Collectors.toSet());
Assert.assertEquals("Log files belonging to file-slices created because of compaction request must be renamed",
expLogFilesToBeRenamed, gotLogFilesToBeRenamed);
validateRenameFiles(renameFiles, ingestionInstant, compactionInstant, fsView);
Map<String, Long> fileIdToCountsBeforeRenaming =
fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant)
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
.filter(fs -> fs.getFileId().equals(op.getFileId()))
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
// Call the main unschedule API
client.unscheduleCompactionFileId(op.getFileGroupId(), false, false);
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
final HoodieTableFileSystemView newFsView =
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
// Expect all file-slice whose base-commit is same as compaction commit to contain no new Log files
newFsView.getLatestFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant, true)
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant))
.filter(fs -> fs.getFileId().equals(op.getFileId())).forEach(fs -> {
Assert.assertFalse("No Data file must be present", fs.getDataFile().isPresent());
Assert.assertTrue("No Log Files", fs.getLogFiles().count() == 0);
});
// Ensure same number of log-files before and after renaming per fileId
Map<String, Long> fileIdToCountsAfterRenaming =
newFsView.getAllFileGroups(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]).flatMap(fg -> fg.getAllFileSlices())
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
.filter(fs -> fs.getFileId().equals(op.getFileId()))
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
Assert.assertEquals("Each File Id has same number of log-files",
fileIdToCountsBeforeRenaming, fileIdToCountsAfterRenaming);
Assert.assertEquals("Not Empty", 1, fileIdToCountsAfterRenaming.size());
Assert.assertEquals("Expected number of renames", expNumRenames, renameFiles.size());
}
}

View File

@@ -0,0 +1,121 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi;
import java.io.IOException;
import java.util.Arrays;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.util.ConsistencyGuard;
import org.apache.hudi.common.util.ConsistencyGuardConfig;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.FailSafeConsistencyGuard;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
public class TestConsistencyGuard {
private String basePath;
protected transient FileSystem fs;
@Before
public void setup() throws IOException {
TemporaryFolder testFolder = new TemporaryFolder();
testFolder.create();
basePath = testFolder.getRoot().getAbsolutePath();
fs = FSUtils.getFs(basePath, new Configuration());
if (fs instanceof LocalFileSystem) {
LocalFileSystem lfs = (LocalFileSystem) fs;
// With LocalFileSystem, with checksum disabled, fs.open() returns an inputStream which is FSInputStream
// This causes ClassCastExceptions in LogRecordScanner (and potentially other places) calling fs.open
// So, for the tests, we enforce checksum verification to circumvent the problem
lfs.setVerifyChecksum(true);
}
}
@Test
public void testCheckPassingAppearAndDisAppear() throws Exception {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f2");
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f3");
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig(1, 1000, 1000));
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f2_1-0-1_000.parquet"));
passing.waitTillAllFilesAppear(basePath + "/partition/path",
Arrays.asList(basePath + "/partition/path/f1_1-0-1_000.parquet",
basePath + "/partition/path/f2_1-0-1_000.parquet"));
fs.delete(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"), false);
fs.delete(new Path(basePath + "/partition/path/f2_1-0-1_000.parquet"), false);
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f2_1-0-1_000.parquet"));
passing.waitTillAllFilesDisappear(basePath + "/partition/path",
Arrays.asList(basePath + "/partition/path/f1_1-0-1_000.parquet",
basePath + "/partition/path/f2_1-0-1_000.parquet"));
}
@Test(expected = TimeoutException.class)
public void testCheckFailingAppear() throws Exception {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
passing.waitTillAllFilesAppear(basePath + "/partition/path",
Arrays.asList(basePath + "/partition/path/f1_1-0-2_000.parquet",
basePath + "/partition/path/f2_1-0-2_000.parquet"));
}
@Test(expected = TimeoutException.class)
public void testCheckFailingAppears() throws Exception {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-2_000.parquet"));
}
@Test(expected = TimeoutException.class)
public void testCheckFailingDisappear() throws Exception {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
passing.waitTillAllFilesDisappear(basePath + "/partition/path",
Arrays.asList(basePath + "/partition/path/f1_1-0-1_000.parquet",
basePath + "/partition/path/f2_1-0-2_000.parquet"));
}
@Test(expected = TimeoutException.class)
public void testCheckFailingDisappears() throws Exception {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
}
private ConsistencyGuardConfig getConsistencyGuardConfig() {
return getConsistencyGuardConfig(3, 10, 10);
}
private ConsistencyGuardConfig getConsistencyGuardConfig(int maxChecks, int initalSleep, int maxSleep) {
return ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true)
.withInitialConsistencyCheckIntervalMs(initalSleep).withMaxConsistencyCheckIntervalMs(maxSleep)
.withMaxConsistencyChecks(maxChecks).build();
}
}

View File

@@ -0,0 +1,543 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.io.File;
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.Map;
import java.util.Set;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.HoodieCleanStat;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.TestRawTripPayload.MetadataMergeWriteStatus;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.SyncableFileSystemView;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
import org.apache.hudi.common.util.ConsistencyGuardConfig;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.index.HoodieIndex.IndexType;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SQLContext;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.rules.TemporaryFolder;
/**
* Base Class providing setup/cleanup and utility methods for testing Hoodie Client facing tests
*/
public class TestHoodieClientBase implements Serializable {
protected static Logger logger = LogManager.getLogger(TestHoodieClientBase.class);
protected transient JavaSparkContext jsc = null;
protected transient SQLContext sqlContext;
protected transient FileSystem fs;
protected String basePath = null;
protected TemporaryFolder folder = null;
protected transient HoodieTestDataGenerator dataGen = null;
private HoodieWriteClient writeClient;
private HoodieReadClient readClient;
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) {
return getHoodieWriteClient(cfg, false);
}
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit) {
return getHoodieWriteClient(cfg, rollbackInflightCommit, HoodieIndex.createIndex(cfg, jsc));
}
protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit,
HoodieIndex index) {
closeWriteClient();
writeClient = new HoodieWriteClient(jsc, cfg, rollbackInflightCommit, index);
return writeClient;
}
protected HoodieReadClient getHoodieReadClient(String basePath) {
closeReadClient();
readClient = new HoodieReadClient(jsc, basePath);
return readClient;
}
private void closeWriteClient() {
if (null != writeClient) {
writeClient.close();
writeClient = null;
}
}
private void closeReadClient() {
if (null != readClient) {
readClient.close();
readClient = null;
}
}
@Before
public void init() throws IOException {
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieClient"));
jsc.setLogLevel("ERROR");
//SQLContext stuff
sqlContext = new SQLContext(jsc);
folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
if (fs instanceof LocalFileSystem) {
LocalFileSystem lfs = (LocalFileSystem) fs;
// With LocalFileSystem, with checksum disabled, fs.open() returns an inputStream which is FSInputStream
// This causes ClassCastExceptions in LogRecordScanner (and potentially other places) calling fs.open
// So, for the tests, we enforce checksum verification to circumvent the problem
lfs.setVerifyChecksum(true);
}
HoodieTestUtils.initTableType(jsc.hadoopConfiguration(), basePath, getTableType());
dataGen = new HoodieTestDataGenerator();
}
@After
/**
* Properly release resources at end of each test
*/
public void tearDown() throws IOException {
closeWriteClient();
closeReadClient();
if (null != sqlContext) {
logger.info("Clearing sql context cache of spark-session used in previous test-case");
sqlContext.clearCache();
}
if (null != jsc) {
logger.info("Closing spark context used in previous test-case");
jsc.close();
}
// Create a temp folder as the base path
if (null != folder) {
logger.info("Explicitly removing workspace used in previously run test-case");
folder.delete();
}
if (null != fs) {
logger.warn("Closing file-system instance used in previous test-run");
fs.close();
}
}
/**
* Get Default HoodieWriteConfig for tests
*
* @return Default Hoodie Write Config for tests
*/
protected HoodieWriteConfig getConfig() {
return getConfigBuilder().build();
}
/**
* Get Config builder with default configs set
*
* @return Config Builder
*/
HoodieWriteConfig.Builder getConfigBuilder() {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2)
.withBulkInsertParallelism(2).withFinalizeWriteParallelism(2)
.withWriteStatusClass(MetadataMergeWriteStatus.class)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
.forTable("test-trip-table")
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build())
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
.build());
}
protected HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
((SyncableFileSystemView) (table.getRTFileSystemView())).reset();
return table;
}
/**
* Assert no failures in writing hoodie files
*
* @param statuses List of Write Status
*/
static void assertNoWriteErrors(List<WriteStatus> statuses) {
// Verify there are no errors
for (WriteStatus status : statuses) {
assertFalse("Errors found in write of " + status.getFileId(), status.hasErrors());
}
}
/**
* Ensure presence of partition meta-data at known depth
*
* @param partitionPaths Partition paths to check
* @param fs File System
* @throws IOException in case of error
*/
void assertPartitionMetadata(String[] partitionPaths, FileSystem fs) throws IOException {
for (String partitionPath : partitionPaths) {
assertTrue(HoodiePartitionMetadata.hasPartitionMetadata(fs, new Path(basePath, partitionPath)));
HoodiePartitionMetadata pmeta = new HoodiePartitionMetadata(fs, new Path(basePath, partitionPath));
pmeta.readFromFS();
Assert.assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_DEPTH, pmeta.getPartitionDepth());
}
}
/**
* Ensure records have location field set
*
* @param taggedRecords Tagged Records
* @param commitTime Commit Timestamp
*/
void checkTaggedRecords(List<HoodieRecord> taggedRecords, String commitTime) {
for (HoodieRecord rec : taggedRecords) {
assertTrue("Record " + rec + " found with no location.", rec.isCurrentLocationKnown());
assertEquals("All records should have commit time " + commitTime + ", since updates were made",
rec.getCurrentLocation().getInstantTime(), commitTime);
}
}
/**
* Assert that there is no duplicate key at the partition level
*
* @param records List of Hoodie records
*/
void assertNodupesWithinPartition(List<HoodieRecord> records) {
Map<String, Set<String>> partitionToKeys = new HashMap<>();
for (HoodieRecord r : records) {
String key = r.getRecordKey();
String partitionPath = r.getPartitionPath();
if (!partitionToKeys.containsKey(partitionPath)) {
partitionToKeys.put(partitionPath, new HashSet<>());
}
assertTrue("key " + key + " is duplicate within partition " + partitionPath,
!partitionToKeys.get(partitionPath).contains(key));
partitionToKeys.get(partitionPath).add(key);
}
}
/**
* Helper to generate records generation function for testing Prepped version of API. Prepped APIs expect the records
* to be already de-duped and have location set. This wrapper takes care of record-location setting. Uniqueness is
* guaranteed by record-generation function itself.
*
* @param writeConfig Hoodie Write Config
* @param recordGenFunction Records Generation function
* @return Wrapped function
*/
private Function2<List<HoodieRecord>, String, Integer> wrapRecordsGenFunctionForPreppedCalls(
final HoodieWriteConfig writeConfig,
final Function2<List<HoodieRecord>, String, Integer> recordGenFunction) {
return (commit, numRecords) -> {
final HoodieIndex index = HoodieIndex.createIndex(writeConfig, jsc);
List<HoodieRecord> records = recordGenFunction.apply(commit, numRecords);
final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, writeConfig, jsc);
JavaRDD<HoodieRecord> taggedRecords =
index.tagLocation(jsc.parallelize(records, 1), jsc, table);
return taggedRecords.collect();
};
}
/**
* Generate wrapper for record generation function for testing Prepped APIs
*
* @param isPreppedAPI Flag to indicate if this is for testing prepped-version of APIs
* @param writeConfig Hoodie Write Config
* @param wrapped Actual Records Generation function
* @return Wrapped Function
*/
Function2<List<HoodieRecord>, String, Integer> generateWrapRecordsFn(
boolean isPreppedAPI,
HoodieWriteConfig writeConfig,
Function2<List<HoodieRecord>, String, Integer> wrapped) {
if (isPreppedAPI) {
return wrapRecordsGenFunctionForPreppedCalls(writeConfig, wrapped);
} else {
return wrapped;
}
}
/**
* Helper to insert first batch of records and do regular assertions on the state after successful completion
*
* @param writeConfig Hoodie Write Config
* @param client Hoodie Write Client
* @param newCommitTime New Commit Timestamp to be used
* @param initCommitTime Begin Timestamp (usually "000")
* @param numRecordsInThisCommit Number of records to be added in the new commit
* @param writeFn Write Function to be used for insertion
* @param isPreppedAPI Boolean flag to indicate writeFn expects prepped records
* @param assertForCommit Enable Assertion of Writes
* @param expRecordsInThisCommit Expected number of records in this commit
* @return RDD of write-status
* @throws Exception in case of error
*/
JavaRDD<WriteStatus> insertFirstBatch(
HoodieWriteConfig writeConfig,
HoodieWriteClient client,
String newCommitTime,
String initCommitTime,
int numRecordsInThisCommit,
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
boolean isPreppedAPI,
boolean assertForCommit,
int expRecordsInThisCommit) throws Exception {
final Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts);
return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime,
numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit,
expRecordsInThisCommit, expRecordsInThisCommit, 1);
}
/**
* Helper to upsert batch of records and do regular assertions on the state after successful completion
*
* @param writeConfig Hoodie Write Config
* @param client Hoodie Write Client
* @param newCommitTime New Commit Timestamp to be used
* @param prevCommitTime Commit Timestamp used in previous commit
* @param commitTimesBetweenPrevAndNew Sample of Timestamps between prevCommitTime and newCommitTime
* @param initCommitTime Begin Timestamp (usually "000")
* @param numRecordsInThisCommit Number of records to be added in the new commit
* @param writeFn Write Function to be used for upsert
* @param isPreppedAPI Boolean flag to indicate writeFn expects prepped records
* @param assertForCommit Enable Assertion of Writes
* @param expRecordsInThisCommit Expected number of records in this commit
* @param expTotalRecords Expected number of records when scanned
* @param expTotalCommits Expected number of commits (including this commit)
* @return RDD of write-status
* @throws Exception in case of error
*/
JavaRDD<WriteStatus> updateBatch(
HoodieWriteConfig writeConfig,
HoodieWriteClient client,
String newCommitTime,
String prevCommitTime,
Option<List<String>> commitTimesBetweenPrevAndNew,
String initCommitTime,
int numRecordsInThisCommit,
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
boolean isPreppedAPI,
boolean assertForCommit,
int expRecordsInThisCommit,
int expTotalRecords,
int expTotalCommits)
throws Exception {
final Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateUniqueUpdates);
return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime,
numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit,
expRecordsInThisCommit, expTotalRecords, expTotalCommits);
}
/**
* Helper to insert/upsert batch of records and do regular assertions on the state after successful completion
*
* @param client Hoodie Write Client
* @param newCommitTime New Commit Timestamp to be used
* @param prevCommitTime Commit Timestamp used in previous commit
* @param commitTimesBetweenPrevAndNew Sample of Timestamps between prevCommitTime and newCommitTime
* @param initCommitTime Begin Timestamp (usually "000")
* @param numRecordsInThisCommit Number of records to be added in the new commit
* @param recordGenFunction Records Generation Function
* @param writeFn Write Function to be used for upsert
* @param assertForCommit Enable Assertion of Writes
* @param expRecordsInThisCommit Expected number of records in this commit
* @param expTotalRecords Expected number of records when scanned
* @param expTotalCommits Expected number of commits (including this commit)
* @throws Exception in case of error
*/
JavaRDD<WriteStatus> writeBatch(
HoodieWriteClient client,
String newCommitTime,
String prevCommitTime,
Option<List<String>> commitTimesBetweenPrevAndNew,
String initCommitTime,
int numRecordsInThisCommit,
Function2<List<HoodieRecord>, String, Integer> recordGenFunction,
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
boolean assertForCommit,
int expRecordsInThisCommit,
int expTotalRecords,
int expTotalCommits)
throws Exception {
//Write 1 (only inserts)
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = recordGenFunction.apply(newCommitTime, numRecordsInThisCommit);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> result = writeFn.apply(client, writeRecords, newCommitTime);
List<WriteStatus> statuses = result.collect();
assertNoWriteErrors(statuses);
// check the partition metadata is written out
assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs);
// verify that there is a commit
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
if (assertForCommit) {
assertEquals("Expecting " + expTotalCommits + " commits.", expTotalCommits,
timeline.findInstantsAfter(initCommitTime, Integer.MAX_VALUE).countInstants());
Assert.assertEquals("Latest commit should be " + newCommitTime, newCommitTime,
timeline.lastInstant().get().getTimestamp());
assertEquals("Must contain " + expRecordsInThisCommit + " records", expRecordsInThisCommit,
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
// Check the entire dataset has all records still
String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length];
for (int i = 0; i < fullPartitionPaths.length; i++) {
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
}
assertEquals("Must contain " + expTotalRecords + " records", expTotalRecords,
HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, fullPartitionPaths).count());
// Check that the incremental consumption from prevCommitTime
assertEquals("Incremental consumption from " + prevCommitTime
+ " should give all records in latest commit",
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, prevCommitTime).count());
if (commitTimesBetweenPrevAndNew.isPresent()) {
commitTimesBetweenPrevAndNew.get().forEach(ct -> {
assertEquals("Incremental consumption from " + ct + " should give all records in latest commit",
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
HoodieClientTestUtils.readSince(basePath, sqlContext, timeline, ct).count());
});
}
}
return result;
}
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
/**
* Get Cleaner state corresponding to a partition path
*
* @param hoodieCleanStatsTwo List of Clean Stats
* @param partitionPath Partition path for filtering
* @return Cleaner state corresponding to partition path
*/
HoodieCleanStat getCleanStat(List<HoodieCleanStat> hoodieCleanStatsTwo, String partitionPath) {
return hoodieCleanStatsTwo.stream().filter(e -> e.getPartitionPath().equals(partitionPath)).findFirst().get();
}
/**
* Utility to simulate commit touching files in a partition
*
* @param files List of file-Ids to be touched
* @param partitionPath Partition
* @param commitTime Commit Timestamp
* @throws IOException in case of error
*/
void updateAllFilesInPartition(List<String> files, String partitionPath, String commitTime)
throws IOException {
for (String fileId : files) {
HoodieTestUtils.createDataFile(basePath, partitionPath, commitTime, fileId);
}
}
/**
* Helper methods to create new data files in a partition
*
* @param partitionPath Partition
* @param commitTime Commit Timestamp
* @param numFiles Number of files to be added
* @return Created files
* @throws IOException in case of error
*/
List<String> createFilesInPartition(String partitionPath, String commitTime, int numFiles)
throws IOException {
List<String> files = new ArrayList<>();
for (int i = 0; i < numFiles; i++) {
files.add(HoodieTestUtils.createNewDataFile(basePath, partitionPath, commitTime));
}
return files;
}
// Functional Interfaces for passing lambda and Hoodie Write API contexts
@FunctionalInterface
interface Function2<R, T1, T2> {
R apply(T1 v1, T2 v2) throws IOException;
}
@FunctionalInterface
interface Function3<R, T1, T2, T3> {
R apply(T1 v1, T2 v2, T3 v3) throws IOException;
}
protected HoodieTableType getTableType() {
return HoodieTableType.COPY_ON_WRITE;
}
}

View File

@@ -0,0 +1,735 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi;
import static org.apache.hudi.common.util.ParquetUtils.readRowKeysFromParquet;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.FileInputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRollingStat;
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.ConsistencyGuardConfig;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ParquetUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaRDD;
import org.junit.Assert;
import org.junit.Test;
@SuppressWarnings("unchecked")
public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
@Override
public void tearDown() throws IOException {
super.tearDown();
}
/**
* Test Auto Commit behavior for HoodieWriteClient insert API
*/
@Test
public void testAutoCommitOnInsert() throws Exception {
testAutoCommit(HoodieWriteClient::insert, false);
}
/**
* Test Auto Commit behavior for HoodieWriteClient insertPrepped API
*/
@Test
public void testAutoCommitOnInsertPrepped() throws Exception {
testAutoCommit(HoodieWriteClient::insertPreppedRecords, true);
}
/**
* Test Auto Commit behavior for HoodieWriteClient upsert API
*/
@Test
public void testAutoCommitOnUpsert() throws Exception {
testAutoCommit(HoodieWriteClient::upsert, false);
}
/**
* Test Auto Commit behavior for HoodieWriteClient upsert Prepped API
*/
@Test
public void testAutoCommitOnUpsertPrepped() throws Exception {
testAutoCommit(HoodieWriteClient::upsertPreppedRecords, true);
}
/**
* Test Auto Commit behavior for HoodieWriteClient bulk-insert API
*/
@Test
public void testAutoCommitOnBulkInsert() throws Exception {
testAutoCommit(HoodieWriteClient::bulkInsert, false);
}
/**
* Test Auto Commit behavior for HoodieWriteClient bulk-insert prepped API
*/
@Test
public void testAutoCommitOnBulkInsertPrepped() throws Exception {
testAutoCommit((writeClient, recordRDD, commitTime)
-> writeClient.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty()), true);
}
/**
* Test auto-commit by applying write function
*
* @param writeFn One of HoodieWriteClient Write API
* @throws Exception in case of failure
*/
private void testAutoCommit(
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
boolean isPrepped) throws Exception {
// Set autoCommit false
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
HoodieWriteClient client = getHoodieWriteClient(cfg);
String prevCommitTime = "000";
String newCommitTime = "001";
int numRecords = 200;
JavaRDD<WriteStatus> result =
insertFirstBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, writeFn, isPrepped, false, numRecords);
assertFalse("If Autocommit is false, then commit should not be made automatically",
HoodieTestUtils.doesCommitExist(basePath, newCommitTime));
assertTrue("Commit should succeed", client.commit(newCommitTime, result));
assertTrue("After explicit commit, commit file should be created",
HoodieTestUtils.doesCommitExist(basePath, newCommitTime));
}
/**
* Test De-duplication behavior for HoodieWriteClient insert API
*/
@Test
public void testDeduplicationOnInsert() throws Exception {
testDeduplication(HoodieWriteClient::insert);
}
/**
* Test De-duplication behavior for HoodieWriteClient bulk-insert API
*/
@Test
public void testDeduplicationOnBulkInsert() throws Exception {
testDeduplication(HoodieWriteClient::bulkInsert);
}
/**
* Test De-duplication behavior for HoodieWriteClient upsert API
*/
@Test
public void testDeduplicationOnUpsert() throws Exception {
testDeduplication(HoodieWriteClient::upsert);
}
/**
* Test Deduplication Logic for write function
*
* @param writeFn One of HoddieWriteClient non-prepped write APIs
* @throws Exception in case of failure
*/
private void testDeduplication(
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn) throws Exception {
String newCommitTime = "001";
String recordKey = UUID.randomUUID().toString();
HoodieKey keyOne = new HoodieKey(recordKey, "2018-01-01");
HoodieRecord recordOne = new HoodieRecord(keyOne,
HoodieTestDataGenerator.generateRandomValue(keyOne, newCommitTime));
HoodieKey keyTwo = new HoodieKey(recordKey, "2018-02-01");
HoodieRecord recordTwo = new HoodieRecord(keyTwo,
HoodieTestDataGenerator.generateRandomValue(keyTwo, newCommitTime));
// Same key and partition as keyTwo
HoodieRecord recordThree = new HoodieRecord(keyTwo,
HoodieTestDataGenerator.generateRandomValue(keyTwo, newCommitTime));
JavaRDD<HoodieRecord> records = jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1);
// dedup should be done based on recordKey only
HoodieWriteClient clientWithDummyGlobalIndex = getWriteClientWithDummyIndex(true);
List<HoodieRecord> dedupedRecs = clientWithDummyGlobalIndex.deduplicateRecords(records, 1).collect();
assertEquals(1, dedupedRecs.size());
assertNodupesWithinPartition(dedupedRecs);
// dedup should be done based on both recordKey and partitionPath
HoodieWriteClient clientWithDummyNonGlobalIndex = getWriteClientWithDummyIndex(false);
dedupedRecs =
clientWithDummyNonGlobalIndex.deduplicateRecords(records, 1).collect();
assertEquals(2, dedupedRecs.size());
assertNodupesWithinPartition(dedupedRecs);
// Perform write-action and check
HoodieWriteClient client = getHoodieWriteClient(
getConfigBuilder().combineInput(true, true).build(), false);
client.startCommitWithTime(newCommitTime);
List<WriteStatus> statuses = writeFn.apply(client, records, newCommitTime).collect();
assertNoWriteErrors(statuses);
assertEquals(2, statuses.size());
assertNodupesWithinPartition(
statuses.stream().map(WriteStatus::getWrittenRecords)
.flatMap(Collection::stream).collect(Collectors.toList()));
}
/**
* Build a test Hoodie WriteClient with dummy index to configure isGlobal flag
*
* @param isGlobal Flag to control HoodieIndex.isGlobal
* @return Hoodie Write Client
* @throws Exception in case of error
*/
private HoodieWriteClient getWriteClientWithDummyIndex(final boolean isGlobal) throws Exception {
HoodieIndex index = mock(HoodieIndex.class);
when(index.isGlobal()).thenReturn(isGlobal);
return getHoodieWriteClient(getConfigBuilder().build(), false, index);
}
/**
* Test Upsert API
*/
@Test
public void testUpserts() throws Exception {
testUpsertsInternal(getConfig(),
HoodieWriteClient::upsert, false);
}
/**
* Test UpsertPrepped API
*/
@Test
public void testUpsertsPrepped() throws Exception {
testUpsertsInternal(getConfig(),
HoodieWriteClient::upsertPreppedRecords, true);
}
/**
* Test one of HoodieWriteClient upsert(Prepped) APIs
*
* @param hoodieWriteConfig Write Config
* @param writeFn One of Hoodie Write Function API
* @throws Exception in case of error
*/
private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig,
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
boolean isPrepped) throws Exception {
HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
//Write 1 (only inserts)
String newCommitTime = "001";
String initCommitTime = "000";
int numRecords = 200;
insertFirstBatch(hoodieWriteConfig,
client, newCommitTime, initCommitTime, numRecords, HoodieWriteClient::insert, isPrepped, true, numRecords);
// Write 2 (updates)
String prevCommitTime = newCommitTime;
newCommitTime = "004";
numRecords = 100;
String commitTimeBetweenPrevAndNew = "002";
updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime,
Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)),
initCommitTime, numRecords, writeFn, isPrepped, true, numRecords, 200, 2);
}
/**
* Tesst deletion of records
*/
@Test
public void testDeletes() throws Exception {
HoodieWriteClient client = getHoodieWriteClient(getConfig(), false);
/**
* Write 1 (inserts and deletes)
* Write actual 200 insert records and ignore 100 delete records
*/
String initCommitTime = "000";
String newCommitTime = "001";
final List<HoodieRecord> recordsInFirstBatch = new ArrayList<>();
Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
(String commitTime, Integer numRecordsInThisCommit) -> {
List<HoodieRecord> fewRecordsForInsert = dataGen.generateInserts(commitTime, 200);
List<HoodieRecord> fewRecordsForDelete = dataGen.generateDeletes(commitTime, 100);
recordsInFirstBatch.addAll(fewRecordsForInsert);
recordsInFirstBatch.addAll(fewRecordsForDelete);
return recordsInFirstBatch;
};
writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime,
//unused as genFn uses hard-coded number of inserts/updates/deletes
-1,
recordGenFunction, HoodieWriteClient::upsert, true,
200, 200, 1);
/**
* Write 2 (deletes+writes)
*/
String prevCommitTime = newCommitTime;
newCommitTime = "004";
final List<HoodieRecord> recordsInSecondBatch = new ArrayList<>();
recordGenFunction =
(String commitTime, Integer numRecordsInThisCommit) -> {
List<HoodieRecord> fewRecordsForDelete = recordsInFirstBatch.subList(0, 50);
List<HoodieRecord> fewRecordsForUpdate = recordsInFirstBatch.subList(50, 100);
recordsInSecondBatch.addAll(dataGen.generateDeletesFromExistingRecords(fewRecordsForDelete));
recordsInSecondBatch.addAll(fewRecordsForUpdate);
return recordsInSecondBatch;
};
writeBatch(client, newCommitTime, prevCommitTime, Option.empty(), initCommitTime,
100, recordGenFunction, HoodieWriteClient::upsert, true,
50, 150, 2);
}
/**
* Test scenario of new file-group getting added during upsert()
*/
@Test
public void testSmallInsertHandlingForUpserts() throws Exception {
final String testPartitionPath = "2016/09/26";
final int insertSplitLimit = 100;
// setup the small file handling params
HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max
dataGen = new HoodieTestDataGenerator(new String[]{testPartitionPath});
HoodieWriteClient client = getHoodieWriteClient(config, false);
// Inserts => will write file1
String commitTime1 = "001";
client.startCommitWithTime(commitTime1);
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, insertSplitLimit); // this writes ~500kb
Set<String> keys1 = HoodieClientTestUtils.getRecordKeys(inserts1);
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 1);
List<WriteStatus> statuses = client.upsert(insertRecordsRDD1, commitTime1).collect();
assertNoWriteErrors(statuses);
assertEquals("Just 1 file needs to be added.", 1, statuses.size());
String file1 = statuses.get(0).getFileId();
Assert.assertEquals("file should contain 100 records", readRowKeysFromParquet(jsc.hadoopConfiguration(),
new Path(basePath, statuses.get(0).getStat().getPath())).size(), 100);
// Update + Inserts such that they just expand file1
String commitTime2 = "002";
client.startCommitWithTime(commitTime2);
List<HoodieRecord> inserts2 = dataGen.generateInserts(commitTime2, 40);
Set<String> keys2 = HoodieClientTestUtils.getRecordKeys(inserts2);
List<HoodieRecord> insertsAndUpdates2 = new ArrayList<>();
insertsAndUpdates2.addAll(inserts2);
insertsAndUpdates2.addAll(dataGen.generateUpdates(commitTime2, inserts1));
JavaRDD<HoodieRecord> insertAndUpdatesRDD2 = jsc.parallelize(insertsAndUpdates2, 1);
statuses = client.upsert(insertAndUpdatesRDD2, commitTime2).collect();
assertNoWriteErrors(statuses);
assertEquals("Just 1 file needs to be updated.", 1, statuses.size());
assertEquals("Existing file should be expanded", file1, statuses.get(0).getFileId());
assertEquals("Existing file should be expanded", commitTime1, statuses.get(0).getStat().getPrevCommit());
Path newFile = new Path(basePath, statuses.get(0).getStat().getPath());
assertEquals("file should contain 140 records",
readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140);
List<GenericRecord> records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), newFile);
for (GenericRecord record : records) {
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
assertEquals("only expect commit2", commitTime2, record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString());
assertTrue("key expected to be part of commit2", keys2.contains(recordKey) || keys1.contains(recordKey));
}
// update + inserts such that file1 is updated and expanded, a new file2 is created.
String commitTime3 = "003";
client.startCommitWithTime(commitTime3);
List<HoodieRecord> insertsAndUpdates3 = dataGen.generateInserts(commitTime3, 200);
Set<String> keys3 = HoodieClientTestUtils.getRecordKeys(insertsAndUpdates3);
List<HoodieRecord> updates3 = dataGen.generateUpdates(commitTime3, inserts2);
insertsAndUpdates3.addAll(updates3);
JavaRDD<HoodieRecord> insertAndUpdatesRDD3 = jsc.parallelize(insertsAndUpdates3, 1);
statuses = client.upsert(insertAndUpdatesRDD3, commitTime3).collect();
assertNoWriteErrors(statuses);
assertEquals("2 files needs to be committed.", 2, statuses.size());
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = getHoodieTable(metadata, config);
ReadOptimizedView fileSystemView = table.getROFileSystemView();
List<HoodieDataFile> files = fileSystemView.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3)
.collect(Collectors.toList());
int numTotalInsertsInCommit3 = 0;
int numTotalUpdatesInCommit3 = 0;
for (HoodieDataFile file : files) {
if (file.getFileName().contains(file1)) {
assertEquals("Existing file should be expanded", commitTime3, file.getCommitTime());
records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath()));
for (GenericRecord record : records) {
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String recordCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
if (recordCommitTime.equals(commitTime3)) {
if (keys2.contains(recordKey)) {
keys2.remove(recordKey);
numTotalUpdatesInCommit3++;
} else {
numTotalInsertsInCommit3++;
}
}
}
assertEquals("All keys added in commit 2 must be updated in commit3 correctly", 0, keys2.size());
} else {
assertEquals("New file must be written for commit 3", commitTime3, file.getCommitTime());
records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath()));
for (GenericRecord record : records) {
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
assertEquals("only expect commit3", commitTime3,
record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString());
assertTrue("key expected to be part of commit3", keys3.contains(recordKey));
}
numTotalInsertsInCommit3 += records.size();
}
}
assertEquals("Total updates in commit3 must add up", inserts2.size(), numTotalUpdatesInCommit3);
assertEquals("Total inserts in commit3 must add up", keys3.size(), numTotalInsertsInCommit3);
}
/**
* Test scenario of new file-group getting added during insert()
*/
@Test
public void testSmallInsertHandlingForInserts() throws Exception {
final String testPartitionPath = "2016/09/26";
final int insertSplitLimit = 100;
// setup the small file handling params
HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max
dataGen = new HoodieTestDataGenerator(new String[]{testPartitionPath});
HoodieWriteClient client = getHoodieWriteClient(config, false);
// Inserts => will write file1
String commitTime1 = "001";
client.startCommitWithTime(commitTime1);
List<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, insertSplitLimit); // this writes ~500kb
Set<String> keys1 = HoodieClientTestUtils.getRecordKeys(inserts1);
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 1);
List<WriteStatus> statuses = client.insert(insertRecordsRDD1, commitTime1).collect();
assertNoWriteErrors(statuses);
assertPartitionMetadata(new String[]{testPartitionPath}, fs);
assertEquals("Just 1 file needs to be added.", 1, statuses.size());
String file1 = statuses.get(0).getFileId();
assertEquals("file should contain 100 records", readRowKeysFromParquet(jsc.hadoopConfiguration(),
new Path(basePath, statuses.get(0).getStat().getPath())).size(), 100);
// Second, set of Inserts should just expand file1
String commitTime2 = "002";
client.startCommitWithTime(commitTime2);
List<HoodieRecord> inserts2 = dataGen.generateInserts(commitTime2, 40);
Set<String> keys2 = HoodieClientTestUtils.getRecordKeys(inserts2);
JavaRDD<HoodieRecord> insertRecordsRDD2 = jsc.parallelize(inserts2, 1);
statuses = client.insert(insertRecordsRDD2, commitTime2).collect();
assertNoWriteErrors(statuses);
assertEquals("Just 1 file needs to be updated.", 1, statuses.size());
assertEquals("Existing file should be expanded", file1, statuses.get(0).getFileId());
assertEquals("Existing file should be expanded", commitTime1, statuses.get(0).getStat().getPrevCommit());
Path newFile = new Path(basePath, statuses.get(0).getStat().getPath());
assertEquals("file should contain 140 records",
readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140);
List<GenericRecord> records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), newFile);
for (GenericRecord record : records) {
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String recCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
assertTrue("Record expected to be part of commit 1 or commit2",
commitTime1.equals(recCommitTime) || commitTime2.equals(recCommitTime));
assertTrue("key expected to be part of commit 1 or commit2",
keys2.contains(recordKey) || keys1.contains(recordKey));
}
// Lots of inserts such that file1 is updated and expanded, a new file2 is created.
String commitTime3 = "003";
client.startCommitWithTime(commitTime3);
List<HoodieRecord> insert3 = dataGen.generateInserts(commitTime3, 200);
JavaRDD<HoodieRecord> insertRecordsRDD3 = jsc.parallelize(insert3, 1);
statuses = client.insert(insertRecordsRDD3, commitTime3).collect();
assertNoWriteErrors(statuses);
assertEquals("2 files needs to be committed.", 2, statuses.size());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = getHoodieTable(metaClient, config);
List<HoodieDataFile> files = table.getROFileSystemView()
.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3)
.collect(Collectors.toList());
assertEquals("Total of 2 valid data files", 2, files.size());
int totalInserts = 0;
for (HoodieDataFile file : files) {
assertEquals("All files must be at commit 3", commitTime3, file.getCommitTime());
records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath()));
totalInserts += records.size();
}
assertEquals("Total number of records must add up", totalInserts,
inserts1.size() + inserts2.size() + insert3.size());
}
/**
* Test to ensure commit metadata points to valid files
*/
@Test
public void testCommitWritesRelativePaths() throws Exception {
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
HoodieWriteClient client = getHoodieWriteClient(cfg);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
String commitTime = "000";
client.startCommitWithTime(commitTime);
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, commitTime);
assertTrue("Commit should succeed", client.commit(commitTime, result));
assertTrue("After explicit commit, commit file should be created",
HoodieTestUtils.doesCommitExist(basePath, commitTime));
// Get parquet file paths from commit metadata
String actionType = metaClient.getCommitActionType();
HoodieInstant commitInstant = new HoodieInstant(false, actionType, commitTime);
HoodieTimeline commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants();
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.getInstantDetails(commitInstant).get(), HoodieCommitMetadata.class);
String basePath = table.getMetaClient().getBasePath();
Collection<String> commitPathNames = commitMetadata.getFileIdAndFullPaths(basePath).values();
// Read from commit file
String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
FileInputStream inputStream = new FileInputStream(filename);
String everything = IOUtils.toString(inputStream);
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString(),
HoodieCommitMetadata.class);
HashMap<String, String> paths = metadata.getFileIdAndFullPaths(basePath);
inputStream.close();
// Compare values in both to make sure they are equal.
for (String pathName : paths.values()) {
assertTrue(commitPathNames.contains(pathName));
}
}
/**
* Test to ensure commit metadata points to valid files
*/
@Test
public void testRollingStatsInMetadata() throws Exception {
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
HoodieWriteClient client = getHoodieWriteClient(cfg);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
String commitTime = "000";
client.startCommitWithTime(commitTime);
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, commitTime);
assertTrue("Commit should succeed", client.commit(commitTime, result));
assertTrue("After explicit commit, commit file should be created",
HoodieTestUtils.doesCommitExist(basePath, commitTime));
// Read from commit file
String filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
FileInputStream inputStream = new FileInputStream(filename);
String everything = IOUtils.toString(inputStream);
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromJsonString(everything.toString(),
HoodieCommitMetadata.class);
HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata.fromJsonString(metadata.getExtraMetadata()
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY), HoodieRollingStatMetadata.class);
int inserts = 0;
for (Map.Entry<String, Map<String, HoodieRollingStat>> pstat : rollingStatMetadata.getPartitionToRollingStats()
.entrySet()) {
for (Map.Entry<String, HoodieRollingStat> stat : pstat.getValue().entrySet()) {
inserts += stat.getValue().getInserts();
}
}
Assert.assertEquals(inserts, 200);
// Update + Inserts such that they just expand file1
commitTime = "001";
client.startCommitWithTime(commitTime);
records = dataGen.generateUpdates(commitTime, records);
writeRecords = jsc.parallelize(records, 1);
result = client.upsert(writeRecords, commitTime);
assertTrue("Commit should succeed", client.commit(commitTime, result));
assertTrue("After explicit commit, commit file should be created",
HoodieTestUtils.doesCommitExist(basePath, commitTime));
// Read from commit file
filename = HoodieTestUtils.getCommitFilePath(basePath, commitTime);
inputStream = new FileInputStream(filename);
everything = IOUtils.toString(inputStream);
metadata = HoodieCommitMetadata.fromJsonString(everything.toString(), HoodieCommitMetadata.class);
rollingStatMetadata = HoodieCommitMetadata.fromJsonString(metadata.getExtraMetadata()
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY), HoodieRollingStatMetadata.class);
inserts = 0;
int upserts = 0;
for (Map.Entry<String, Map<String, HoodieRollingStat>> pstat : rollingStatMetadata.getPartitionToRollingStats()
.entrySet()) {
for (Map.Entry<String, HoodieRollingStat> stat : pstat.getValue().entrySet()) {
inserts += stat.getValue().getInserts();
upserts += stat.getValue().getUpserts();
}
}
Assert.assertEquals(inserts, 200);
Assert.assertEquals(upserts, 200);
}
/**
* Tests behavior of committing only when consistency is verified
*/
@Test
public void testConsistencyCheckDuringFinalize() throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
String commitTime = "000";
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
HoodieWriteClient client = getHoodieWriteClient(cfg);
Pair<Path, JavaRDD<WriteStatus>> result = testConsistencyCheck(metaClient, commitTime);
// Delete orphan marker and commit should succeed
metaClient.getFs().delete(result.getKey(), false);
assertTrue("Commit should succeed", client.commit(commitTime, result.getRight()));
assertTrue("After explicit commit, commit file should be created",
HoodieTestUtils.doesCommitExist(basePath, commitTime));
// Marker directory must be removed
assertFalse(metaClient.getFs().exists(new Path(metaClient.getMarkerFolderPath(commitTime))));
}
@Test
public void testRollbackAfterConsistencyCheckFailure() throws Exception {
String commitTime = "000";
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
HoodieWriteClient client = getHoodieWriteClient(cfg);
testConsistencyCheck(metaClient, commitTime);
// Rollback of this commit should succeed
client.rollback(commitTime);
assertFalse("After explicit rollback, commit file should not be present",
HoodieTestUtils.doesCommitExist(basePath, commitTime));
// Marker directory must be removed after rollback
assertFalse(metaClient.getFs().exists(new Path(metaClient.getMarkerFolderPath(commitTime))));
}
private Pair<Path, JavaRDD<WriteStatus>> testConsistencyCheck(HoodieTableMetaClient metaClient, String commitTime)
throws Exception {
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder()
.withConsistencyCheckEnabled(true)
.withMaxConsistencyCheckIntervalMs(1)
.withInitialConsistencyCheckIntervalMs(1)
.build())
.build();
HoodieWriteClient client = getHoodieWriteClient(cfg);
client.startCommitWithTime(commitTime);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(dataGen.generateInserts(commitTime, 200), 1);
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, commitTime);
result.collect();
// Create a dummy marker file to simulate the case that a marker file was created without data file.
// This should fail the commit
String partitionPath = Arrays.stream(fs.globStatus(new Path(String.format("%s/*/*/*/*",
metaClient.getMarkerFolderPath(commitTime))),
path -> path.toString().endsWith(HoodieTableMetaClient.MARKER_EXTN))).limit(1)
.map(status -> status.getPath().getParent().toString()).collect(Collectors.toList()).get(0);
Path markerFilePath = new Path(String.format("%s/%s", partitionPath,
FSUtils.makeMarkerFile(commitTime, "1-0-1", UUID.randomUUID().toString())));
metaClient.getFs().create(markerFilePath);
logger.info("Created a dummy marker path=" + markerFilePath);
try {
client.commit(commitTime, result);
fail("Commit should fail due to consistency check");
} catch (HoodieCommitException cme) {
assertTrue(cme.getCause() instanceof HoodieIOException);
}
return Pair.of(markerFilePath, result);
}
/**
* Build Hoodie Write Config for small data file sizes
*/
private HoodieWriteConfig getSmallInsertWriteConfig(int insertSplitSize) {
HoodieWriteConfig.Builder builder = getConfigBuilder();
return builder.withCompactionConfig(
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 15)
.insertSplitSize(insertSplitSize).build()) // tolerate upto 15 records
.withStorageConfig(
HoodieStorageConfig.newBuilder().limitFileSize(HoodieTestDataGenerator.SIZE_PER_RECORD * 20).build())
.build();
}
}

View File

@@ -0,0 +1,207 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.spark.api.java.JavaRDD;
import org.junit.Assert;
import org.junit.Test;
@SuppressWarnings("unchecked")
/**
* Test-cases for covering HoodieReadClient APIs
*/
public class TestHoodieReadClient extends TestHoodieClientBase {
@Override
public void tearDown() throws IOException {
super.tearDown();
}
/**
* Test ReadFilter API after writing new records using HoodieWriteClient.insert
*/
@Test
public void testReadFilterExistAfterInsert() throws Exception {
testReadFilterExist(getConfig(), HoodieWriteClient::insert);
}
/**
* Test ReadFilter API after writing new records using HoodieWriteClient.insertPrepped
*/
@Test
public void testReadFilterExistAfterInsertPrepped() throws Exception {
testReadFilterExist(getConfig(), HoodieWriteClient::insertPreppedRecords);
}
/**
* Test ReadFilter API after writing new records using HoodieWriteClient.bulkInsert
*/
@Test
public void testReadFilterExistAfterBulkInsert() throws Exception {
testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(), HoodieWriteClient::bulkInsert);
}
/**
* Test ReadFilter API after writing new records using HoodieWriteClient.bulkInsertPrepped
*/
@Test
public void testReadFilterExistAfterBulkInsertPrepped() throws Exception {
testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(),
(writeClient, recordRDD, commitTime) -> {
return writeClient.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty());
});
}
/**
* Helper to write new records using one of HoodieWriteClient's write API and use ReadClient to test filterExists()
* API works correctly
*
* @param config Hoodie Write Config
* @param writeFn Write Function for writing records
* @throws Exception in case of error
*/
private void testReadFilterExist(HoodieWriteConfig config,
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn) throws Exception {
HoodieWriteClient writeClient = getHoodieWriteClient(config);
String newCommitTime = writeClient.startCommit();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
HoodieReadClient readClient = getHoodieReadClient(config.getBasePath());
JavaRDD<HoodieRecord> filteredRDD = readClient.filterExists(recordsRDD);
// Should not find any files
assertTrue(filteredRDD.collect().size() == 100);
JavaRDD<HoodieRecord> smallRecordsRDD = jsc.parallelize(records.subList(0, 75), 1);
// We create three parquet file, each having one record. (3 different partitions)
List<WriteStatus> statuses = writeFn.apply(writeClient, smallRecordsRDD, newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
readClient = getHoodieReadClient(config.getBasePath());
filteredRDD = readClient.filterExists(recordsRDD);
List<HoodieRecord> result = filteredRDD.collect();
// Check results
Assert.assertEquals(25, result.size());
}
/**
* Test tagLocation API after insert()
*/
@Test
public void testTagLocationAfterInsert() throws Exception {
testTagLocation(getConfig(), HoodieWriteClient::insert,
HoodieWriteClient::upsert, false);
}
/**
* Test tagLocation API after insertPrepped()
*/
@Test
public void testTagLocationAfterInsertPrepped() throws Exception {
testTagLocation(getConfig(), HoodieWriteClient::insertPreppedRecords,
HoodieWriteClient::upsertPreppedRecords, true);
}
/**
* Test tagLocation API after bulk-insert()
*/
@Test
public void testTagLocationAfterBulkInsert() throws Exception {
testTagLocation(getConfigBuilder().withBulkInsertParallelism(1).build(), HoodieWriteClient::bulkInsert,
HoodieWriteClient::upsert, false);
}
/**
* Test tagLocation API after bulkInsertPrepped()
*/
@Test
public void testTagLocationAfterBulkInsertPrepped() throws Exception {
testTagLocation(getConfigBuilder().withBulkInsertParallelism(1).build(),
(writeClient, recordRDD, commitTime)
-> writeClient.bulkInsertPreppedRecords(recordRDD, commitTime, Option.empty()),
HoodieWriteClient::upsertPreppedRecords, true);
}
/**
* Helper method to test tagLocation after using different HoodieWriteClient write APIS
*
* @param hoodieWriteConfig Write Config
* @param insertFn Hoodie Write Client first Insert API
* @param updateFn Hoodie Write Client upsert API
* @param isPrepped isPrepped flag.
* @throws Exception in case of error
*/
private void testTagLocation(
HoodieWriteConfig hoodieWriteConfig,
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> updateFn,
boolean isPrepped)
throws Exception {
HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig);
//Write 1 (only inserts)
String newCommitTime = "001";
String initCommitTime = "000";
int numRecords = 200;
JavaRDD<WriteStatus> result =
insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, numRecords, insertFn, isPrepped,
true, numRecords);
// Construct HoodieRecord from the WriteStatus but set HoodieKey, Data and HoodieRecordLocation accordingly
// since they have been modified in the DAG
JavaRDD<HoodieRecord> recordRDD =
jsc.parallelize(
result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream)
.map(record -> new HoodieRecord(record.getKey(), null))
.collect(Collectors.toList()));
// Should have 100 records in table (check using Index), all in locations marked at commit
HoodieReadClient readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath());
List<HoodieRecord> taggedRecords = readClient.tagLocation(recordRDD).collect();
checkTaggedRecords(taggedRecords, newCommitTime);
// Write 2 (updates)
String prevCommitTime = newCommitTime;
newCommitTime = "004";
numRecords = 100;
String commitTimeBetweenPrevAndNew = "002";
result = updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime,
Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)),
initCommitTime, numRecords, updateFn, isPrepped,
true, numRecords, 200, 2);
recordRDD =
jsc.parallelize(
result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream)
.map(record -> new HoodieRecord(record.getKey(), null))
.collect(Collectors.toList()));
// Index should be able to locate all updates in correct locations.
readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath());
taggedRecords = readClient.tagLocation(recordRDD).collect();
checkTaggedRecords(taggedRecords, newCommitTime);
}
}

View File

@@ -0,0 +1,180 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi;
import static org.junit.Assert.assertEquals;
import java.io.Serializable;
import java.util.List;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.minicluster.HdfsTestService;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
public class TestMultiFS implements Serializable {
private static String dfsBasePath;
private static HdfsTestService hdfsTestService;
private static MiniDFSCluster dfsCluster;
private static DistributedFileSystem dfs;
private static Logger logger = LogManager.getLogger(TestMultiFS.class);
private static JavaSparkContext jsc;
private static SQLContext sqlContext;
private String tablePath = "file:///tmp/hoodie/sample-table";
protected String tableName = "hoodie_rt";
private HoodieWriteClient hdfsWriteClient;
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
@BeforeClass
public static void initClass() throws Exception {
hdfsTestService = new HdfsTestService();
dfsCluster = hdfsTestService.start(true);
// Create a temp folder as the base path
dfs = dfsCluster.getFileSystem();
dfsBasePath = dfs.getWorkingDirectory().toString();
dfs.mkdirs(new Path(dfsBasePath));
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");
jsc = new JavaSparkContext(sparkConf);
sqlContext = new SQLContext(jsc);
}
private HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig config) throws Exception {
if (null != hdfsWriteClient) {
hdfsWriteClient.close();
}
hdfsWriteClient = new HoodieWriteClient(jsc, config);
return hdfsWriteClient;
}
@After
public void teardown() {
if (null != hdfsWriteClient) {
hdfsWriteClient.close();
hdfsWriteClient = null;
}
}
@AfterClass
public static void cleanupClass() throws Exception {
if (jsc != null) {
jsc.stop();
}
if (hdfsTestService != null) {
hdfsTestService.stop();
dfsCluster.shutdown();
}
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the
// same JVM
FileSystem.closeAll();
}
protected HoodieWriteConfig getHoodieWriteConfig(String basePath) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withEmbeddedTimelineServerEnabled(true)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable(tableName).withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
}
@Test
public void readLocalWriteHDFS() throws Exception {
// Generator of some records to be loaded in.
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
// Initialize table and filesystem
HoodieTableMetaClient
.initTableType(jsc.hadoopConfiguration(), dfsBasePath, HoodieTableType.valueOf(tableType), tableName,
HoodieAvroPayload.class.getName());
//Create write client to write some records in
HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath);
HoodieWriteClient hdfsWriteClient = getHoodieWriteClient(cfg);
// Write generated data to hdfs (only inserts)
String readCommitTime = hdfsWriteClient.startCommit();
logger.info("Starting commit " + readCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(readCommitTime, 100);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
hdfsWriteClient.upsert(writeRecords, readCommitTime);
// Read from hdfs
FileSystem fs = FSUtils.getFs(dfsBasePath, HoodieTestUtils.getDefaultHadoopConf());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), dfsBasePath);
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
Dataset<Row> readRecords = HoodieClientTestUtils.readCommit(dfsBasePath, sqlContext, timeline, readCommitTime);
assertEquals("Should contain 100 records", readRecords.count(), records.size());
// Write to local
HoodieTableMetaClient
.initTableType(jsc.hadoopConfiguration(), tablePath, HoodieTableType.valueOf(tableType), tableName,
HoodieAvroPayload.class.getName());
HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath);
HoodieWriteClient localWriteClient = getHoodieWriteClient(localConfig);
String writeCommitTime = localWriteClient.startCommit();
logger.info("Starting write commit " + writeCommitTime);
List<HoodieRecord> localRecords = dataGen.generateInserts(writeCommitTime, 100);
JavaRDD<HoodieRecord> localWriteRecords = jsc.parallelize(localRecords, 1);
logger.info("Writing to path: " + tablePath);
localWriteClient.upsert(localWriteRecords, writeCommitTime);
logger.info("Reading from path: " + tablePath);
fs = FSUtils.getFs(tablePath, HoodieTestUtils.getDefaultHadoopConf());
metaClient = new HoodieTableMetaClient(fs.getConf(), tablePath);
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
Dataset<Row> localReadRecords = HoodieClientTestUtils.readCommit(tablePath, sqlContext, timeline, writeCommitTime);
assertEquals("Should contain 100 records", localReadRecords.count(), localRecords.size());
hdfsWriteClient.close();
localWriteClient.close();
}
}

View File

@@ -0,0 +1,55 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import org.apache.hudi.common.model.HoodieRecord;
import org.junit.Test;
import org.mockito.Mockito;
public class TestWriteStatus {
@Test
public void testFailureFraction() throws IOException {
WriteStatus status = new WriteStatus(true, 0.1);
Throwable t = new Exception("some error in writing");
for (int i = 0; i < 1000; i++) {
status.markFailure(Mockito.mock(HoodieRecord.class), t, null);
}
assertTrue(status.getFailedRecords().size() > 0);
assertTrue(status.getFailedRecords().size() < 150); //150 instead of 100, to prevent flaky test
assertTrue(status.hasErrors());
}
@Test
public void testSuccessRecordTracking() {
WriteStatus status = new WriteStatus(false, 1.0);
Throwable t = new Exception("some error in writing");
for (int i = 0; i < 1000; i++) {
status.markSuccess(Mockito.mock(HoodieRecord.class), null);
status.markFailure(Mockito.mock(HoodieRecord.class), t, null);
}
assertEquals(1000, status.getFailedRecords().size());
assertTrue(status.hasErrors());
assertTrue(status.getWrittenRecords().isEmpty());
assertEquals(2000, status.getTotalRecords());
}
}

View File

@@ -0,0 +1,249 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common;
import java.io.File;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.HoodieReadClient;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.avro.HoodieAvroWriteSupport;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.HoodieAvroUtils;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.io.storage.HoodieParquetConfig;
import org.apache.hudi.io.storage.HoodieParquetWriter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
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.
*/
public class HoodieClientTestUtils {
private static final transient Logger log = LogManager.getLogger(HoodieClientTestUtils.class);
public static List<WriteStatus> collectStatuses(Iterator<List<WriteStatus>> statusListItr) {
List<WriteStatus> statuses = new ArrayList<>();
while (statusListItr.hasNext()) {
statuses.addAll(statusListItr.next());
}
return statuses;
}
public static Set<String> getRecordKeys(List<HoodieRecord> hoodieRecords) {
Set<String> keys = new HashSet<>();
for (HoodieRecord rec : hoodieRecords) {
keys.add(rec.getRecordKey());
}
return keys;
}
private static void fakeMetaFile(String basePath, String commitTime, String suffix) throws IOException {
String parentPath = basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME;
new File(parentPath).mkdirs();
new File(parentPath + "/" + commitTime + suffix).createNewFile();
}
public static void fakeCommitFile(String basePath, String commitTime) throws IOException {
fakeMetaFile(basePath, commitTime, HoodieTimeline.COMMIT_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, long length)
throws Exception {
String parentPath = String.format("%s/%s", basePath, partitionPath);
new File(parentPath).mkdirs();
String path = String.format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, "1-0-1", fileId));
new File(path).createNewFile();
new RandomAccessFile(path, "rw").setLength(length);
}
public static SparkConf getSparkConfForTest(String appName) {
SparkConf sparkConf = new SparkConf().setAppName(appName)
.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.setMaster("local[8]");
return HoodieReadClient.addHoodieSupport(sparkConf);
}
public static HashMap<String, String> getLatestFileIDsToFullPath(String basePath, HoodieTimeline commitTimeline,
List<HoodieInstant> commitsToReturn) throws IOException {
HashMap<String, String> fileIdToFullPath = new HashMap<>();
for (HoodieInstant commit : commitsToReturn) {
HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get(),
HoodieCommitMetadata.class);
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath));
}
return fileIdToFullPath;
}
public static Dataset<Row> readCommit(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
String commitTime) {
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
if (!commitTimeline.containsInstant(commitInstant)) {
new HoodieException("No commit exists at " + commitTime);
}
try {
HashMap<String, String> paths = getLatestFileIDsToFullPath(basePath, commitTimeline,
Arrays.asList(commitInstant));
log.info("Path :" + paths.values());
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) {
throw new HoodieException("Error reading commit " + commitTime, e);
}
}
/**
* Obtain all new data written into the Hoodie dataset since the given timestamp.
*/
public static Dataset<Row> readSince(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
String lastCommitTime) {
List<HoodieInstant> commitsToReturn = commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE)
.getInstants().collect(Collectors.toList());
try {
// Go over the commit metadata, and obtain the new files that need to be read.
HashMap<String, String> fileIdToFullPath = getLatestFileIDsToFullPath(basePath, commitTimeline, commitsToReturn);
return sqlContext.read().parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()]))
.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime));
} catch (IOException e) {
throw new HoodieException("Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e);
}
}
/**
* Reads the paths under the a hoodie dataset out as a DataFrame
*/
public static Dataset<Row> read(JavaSparkContext jsc, String basePath, SQLContext
sqlContext,
FileSystem
fs, String...
paths) {
List<String> filteredPaths = new ArrayList<>();
try {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true);
for (String path : paths) {
ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(
metaClient, metaClient.getCommitsTimeline().filterCompletedInstants(), fs.globStatus(new Path(path)));
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);
}
}
public static String writeParquetFile(String basePath,
String partitionPath,
String filename,
List<HoodieRecord> records,
Schema schema,
BloomFilter filter,
boolean createCommitTime) throws IOException {
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,
HoodieTestUtils.getDefaultHadoopConf(),
Double.valueOf(HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO));
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) {
HoodieTestUtils.createMetadataFolder(basePath);
HoodieTestUtils.createCommitFiles(basePath, commitTime);
}
return filename;
}
public static String writeParquetFile(String basePath,
String partitionPath,
List<HoodieRecord> records,
Schema schema,
BloomFilter filter,
boolean createCommitTime) throws IOException, InterruptedException {
Thread.sleep(1000);
String commitTime = HoodieTestUtils.makeNewCommitTime();
String fileId = UUID.randomUUID().toString();
String filename = FSUtils.makeDataFileName(commitTime, "1-0-1", fileId);
HoodieTestUtils.createCommitFiles(basePath, commitTime);
return HoodieClientTestUtils
.writeParquetFile(basePath, partitionPath, filename, records, schema, filter, createCommitTime);
}
}

View File

@@ -0,0 +1,100 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common;
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.generic.GenericRecord;
import org.apache.avro.generic.GenericRecordBuilder;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
import org.apache.hadoop.io.ArrayWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.util.HoodieAvroUtils;
import org.apache.hudi.hadoop.realtime.HoodieRealtimeInputFormat;
/**
* Utility methods to aid in testing MergeOnRead (workaround for HoodieReadClient for MOR)
*/
public class HoodieMergeOnReadTestUtils {
public static List<GenericRecord> getRecordsUsingInputFormat(List<String> inputPaths, String basePath)
throws IOException {
JobConf jobConf = new JobConf();
Schema schema = HoodieAvroUtils.addMetadataFields(Schema.parse(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA));
HoodieRealtimeInputFormat inputFormat = new HoodieRealtimeInputFormat();
setPropsForInputFormat(inputFormat, jobConf, schema, basePath);
return inputPaths.stream().map(path -> {
setInputPath(jobConf, path);
List<GenericRecord> records = new ArrayList<>();
try {
List<InputSplit> splits = Arrays.asList(inputFormat.getSplits(jobConf, 1));
RecordReader recordReader = inputFormat.getRecordReader(splits.get(0), jobConf, null);
Void key = (Void) recordReader.createKey();
ArrayWritable writable = (ArrayWritable) recordReader.createValue();
while (recordReader.next(key, writable)) {
GenericRecordBuilder newRecord = new GenericRecordBuilder(schema);
// writable returns an array with [field1, field2, _hoodie_commit_time,
// _hoodie_commit_seqno]
Writable[] values = writable.get();
schema.getFields().forEach(field -> {
newRecord.set(field, values[2]);
});
records.add(newRecord.build());
}
} catch (IOException ie) {
ie.printStackTrace();
}
return records;
}).reduce((a, b) -> {
a.addAll(b);
return a;
}).get();
}
private static void setPropsForInputFormat(HoodieRealtimeInputFormat inputFormat, JobConf jobConf, Schema schema,
String basePath) {
List<Schema.Field> fields = schema.getFields();
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
Configuration conf = HoodieTestUtils.getDefaultHadoopConf();
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
jobConf.set("partition_columns", "datestr");
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
conf.set("partition_columns", "datestr");
inputFormat.setConf(conf);
jobConf.addResource(conf);
}
private static void setInputPath(JobConf jobConf, String inputPath) {
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
jobConf.set("map.input.dir", inputPath);
}
}

View File

@@ -0,0 +1,386 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common;
import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.AvroUtils;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.HoodieAvroUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
/**
* Class to be used in tests to keep generating test inserts and updates against a corpus.
* <p>
* Test data uses a toy Uber trips, data model.
*/
public class HoodieTestDataGenerator {
// 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_FIRST_PARTITION_PATH = "2016/03/15";
public static final String DEFAULT_SECOND_PARTITION_PATH = "2015/03/16";
public static final String DEFAULT_THIRD_PARTITION_PATH = "2015/03/17";
public static final String[] DEFAULT_PARTITION_PATHS = {
DEFAULT_FIRST_PARTITION_PATH,
DEFAULT_SECOND_PARTITION_PATH,
DEFAULT_THIRD_PARTITION_PATH
};
public static final int DEFAULT_PARTITION_DEPTH = 3;
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\"}]}";
public static Schema avroSchema = new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA);
public static Schema avroSchemaWithMetadataFields = HoodieAvroUtils.addMetadataFields(avroSchema);
private static Random rand = new Random(46474747);
private final Map<Integer, KeyPartition> existingKeys;
private final String[] partitionPaths;
private int numExistingKeys;
public HoodieTestDataGenerator(String[] partitionPaths) {
this(partitionPaths, new HashMap<>());
}
public HoodieTestDataGenerator() {
this(DEFAULT_PARTITION_PATHS);
}
public HoodieTestDataGenerator(String[] partitionPaths, Map<Integer, KeyPartition> keyPartitionMap) {
this.partitionPaths = Arrays.copyOf(partitionPaths, partitionPaths.length);
this.existingKeys = keyPartitionMap;
}
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);
}
}
/**
* 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);
return new TestRawTripPayload(rec.toString(), key.getRecordKey(), key.getPartitionPath(), TRIP_EXAMPLE_SCHEMA);
}
/**
* Generates a new avro record of the above schema format, retaining the key if optionally provided.
*/
public static HoodieAvroPayload generateAvroPayload(HoodieKey key, String commitTime) throws IOException {
GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0);
return new HoodieAvroPayload(Option.of(rec));
}
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 {
createCommitFile(basePath, commitTime, HoodieTestUtils.getDefaultHadoopConf());
}
public static void createCommitFile(String basePath, String commitTime, Configuration configuration)
throws IOException {
Path commitFile = new Path(
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime));
FileSystem fs = FSUtils.getFs(basePath, configuration);
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 createCompactionRequestedFile(String basePath, String commitTime, Configuration configuration)
throws IOException {
Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
+ HoodieTimeline.makeRequestedCompactionFileName(commitTime));
FileSystem fs = FSUtils.getFs(basePath, configuration);
FSDataOutputStream os = fs.create(commitFile, true);
os.close();
}
public static void createCompactionAuxiliaryMetadata(String basePath, HoodieInstant instant,
Configuration configuration) throws IOException {
Path commitFile = new Path(
basePath + "/" + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + "/" + instant.getFileName());
FileSystem fs = FSUtils.getFs(basePath, configuration);
FSDataOutputStream os = fs.create(commitFile, true);
HoodieCompactionPlan workload = new HoodieCompactionPlan();
try {
// Write empty commit metadata
os.writeBytes(new String(AvroUtils.serializeCompactionPlan(workload).get(), StandardCharsets.UTF_8));
} finally {
os.close();
}
}
public static void createSavepointFile(String basePath, String commitTime, Configuration configuration)
throws IOException {
Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME
+ "/" + HoodieTimeline.makeSavePointFileName(commitTime));
FileSystem fs = FSUtils.getFs(basePath, configuration);
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();
}
}
/**
* Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
*/
public List<HoodieRecord> generateInserts(String commitTime, Integer n) throws IOException {
return generateInsertsStream(commitTime, n).collect(Collectors.toList());
}
/**
* Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
*/
public Stream<HoodieRecord> generateInsertsStream(String commitTime, Integer n) {
int currSize = getNumExistingKeys();
return IntStream.range(0, n).boxed().map(i -> {
String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
KeyPartition kp = new KeyPartition();
kp.key = key;
kp.partitionPath = partitionPath;
existingKeys.put(currSize + i, kp);
numExistingKeys++;
try {
return new HoodieRecord(key, generateRandomValue(key, commitTime));
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
});
}
public List<HoodieRecord> generateSameKeyInserts(String commitTime, List<HoodieRecord> origin) throws IOException {
List<HoodieRecord> copy = new ArrayList<>();
for (HoodieRecord r: origin) {
HoodieKey key = r.getKey();
HoodieRecord record = new HoodieRecord(key, generateRandomValue(key, commitTime));
copy.add(record);
}
return copy;
}
public List<HoodieRecord> generateInsertsWithHoodieAvroPayload(String commitTime, int limit) throws
IOException {
List<HoodieRecord> inserts = new ArrayList<>();
int currSize = getNumExistingKeys();
for (int i = 0; i < limit; i++) {
String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
HoodieRecord record = new HoodieRecord(key, generateAvroPayload(key, commitTime));
inserts.add(record);
KeyPartition kp = new KeyPartition();
kp.key = key;
kp.partitionPath = partitionPath;
existingKeys.put(currSize + i, kp);
numExistingKeys++;
}
return inserts;
}
public List<HoodieRecord> generateUpdatesWithHoodieAvroPayload(String commitTime, List<HoodieRecord> baseRecords)
throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
for (HoodieRecord baseRecord : baseRecords) {
HoodieRecord record = new HoodieRecord(baseRecord.getKey(), generateAvroPayload(baseRecord.getKey(), commitTime));
updates.add(record);
}
return updates;
}
public List<HoodieRecord> generateDeletes(String commitTime, Integer n) throws IOException {
List<HoodieRecord> inserts = generateInserts(commitTime, n);
return generateDeletesFromExistingRecords(inserts);
}
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);
}
return deletes;
}
public HoodieRecord generateDeleteRecord(HoodieRecord existingRecord) throws IOException {
HoodieKey key = existingRecord.getKey();
return generateDeleteRecord(key);
}
public HoodieRecord generateDeleteRecord(HoodieKey key) throws IOException {
TestRawTripPayload payload = new TestRawTripPayload(Option.empty(), key.getRecordKey(), key.getPartitionPath(),
null, true);
return new HoodieRecord(key, payload);
}
public HoodieRecord generateUpdateRecord(HoodieKey key, String commitTime) throws IOException {
return new HoodieRecord(key, generateRandomValue(key, commitTime));
}
public List<HoodieRecord> generateUpdates(String commitTime, List<HoodieRecord> baseRecords) throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
for (HoodieRecord baseRecord : baseRecords) {
HoodieRecord record = generateUpdateRecord(baseRecord.getKey(), commitTime);
updates.add(record);
}
return updates;
}
/**
* Generates new updates, randomly distributed across the keys above. There can be duplicates within the returned list
*
* @param commitTime Commit Timestamp
* @param n Number of updates (including dups)
* @return list of hoodie record updates
*/
public List<HoodieRecord> generateUpdates(String commitTime, Integer n) throws IOException {
List<HoodieRecord> updates = new ArrayList<>();
for (int i = 0; i < n; i++) {
KeyPartition kp = existingKeys.get(rand.nextInt(numExistingKeys - 1));
HoodieRecord record = generateUpdateRecord(kp.key, commitTime);
updates.add(record);
}
return updates;
}
/**
* Generates deduped updates of keys previously inserted, randomly distributed across the keys above.
*
* @param commitTime Commit Timestamp
* @param n Number of unique records
* @return list of hoodie record updates
*/
public List<HoodieRecord> generateUniqueUpdates(String commitTime, Integer n) {
return generateUniqueUpdatesStream(commitTime, n).collect(Collectors.toList());
}
/**
* Generates deduped updates of keys previously inserted, randomly distributed across the keys above.
*
* @param commitTime Commit Timestamp
* @param n Number of unique records
* @return stream of hoodie record updates
*/
public Stream<HoodieRecord> generateUniqueUpdatesStream(String commitTime, Integer n) {
final Set<KeyPartition> used = new HashSet<>();
if (n > numExistingKeys) {
throw new IllegalArgumentException("Requested unique updates is greater than number of available keys");
}
return IntStream.range(0, n).boxed().map(i -> {
int index = numExistingKeys == 1 ? 0 : rand.nextInt(numExistingKeys - 1);
KeyPartition kp = existingKeys.get(index);
// Find the available keyPartition starting from randomly chosen one.
while (used.contains(kp)) {
index = (index + 1) % numExistingKeys;
kp = existingKeys.get(index);
}
used.add(kp);
try {
return new HoodieRecord(kp.key, generateRandomValue(kp.key, commitTime));
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
});
}
public String[] getPartitionPaths() {
return partitionPaths;
}
public int getNumExistingKeys() {
return numExistingKeys;
}
public static class KeyPartition implements Serializable {
HoodieKey key;
String partitionPath;
}
public void close() {
existingKeys.clear();
}
}

View File

@@ -0,0 +1,196 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common;
import com.fasterxml.jackson.databind.ObjectMapper;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.StringWriter;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.zip.Deflater;
import java.util.zip.DeflaterOutputStream;
import java.util.zip.InflaterInputStream;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.io.IOUtils;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.avro.MercifulJsonConverter;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
/**
* Example row change event based on some example data used by testcases. The data avro schema is
* src/test/resources/schema1.
*/
public class TestRawTripPayload implements HoodieRecordPayload<TestRawTripPayload> {
private static final transient ObjectMapper mapper = new ObjectMapper();
private String partitionPath;
private String rowKey;
private byte[] jsonDataCompressed;
private int dataSize;
private boolean isDeleted;
public TestRawTripPayload(Option<String> jsonData, String rowKey, String partitionPath, 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, String schemaStr) throws IOException {
this(Option.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 Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException {
return this.getInsertValue(schema);
}
@Override
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
if (isDeleted) {
return Option.empty();
} else {
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
return Option.of(jsonConverter.convert(getJsonData()));
}
}
@Override
public Option<Map<String, String>> getMetadata() {
// Let's assume we want to count the number of input row change events
// that are processed. Let the time-bucket for this row change event be 1506582000.
Map<String, String> metadataMap = new HashMap<>();
metadataMap.put("InputRecordCount_1506582000", "2");
return Option.of(metadataMap);
}
public String getRowKey() {
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();
}
private String unCompressData(byte[] data) throws IOException {
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<>();
public MetadataMergeWriteStatus(Boolean trackSuccessRecords, Double failureFraction) {
super(trackSuccessRecords, failureFraction);
}
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 static String addStrsAsInt(String a, String b) {
return String.valueOf(Integer.parseInt(a) + Integer.parseInt(b));
}
@Override
public void markSuccess(HoodieRecord record, Option<Map<String, String>> recordMetadata) {
super.markSuccess(record, recordMetadata);
if (recordMetadata.isPresent()) {
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
}
}
@Override
public void markFailure(HoodieRecord record, Throwable t, Option<Map<String, String>> recordMetadata) {
super.markFailure(record, t, recordMetadata);
if (recordMetadata.isPresent()) {
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
}
}
private Map<String, String> getMergedMetadataMap() {
return mergedMetadataMap;
}
}
}

View File

@@ -0,0 +1,62 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.config;
import static org.junit.Assert.assertEquals;
import com.google.common.collect.Maps;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.Date;
import java.util.Map;
import java.util.Properties;
import org.apache.hudi.config.HoodieWriteConfig.Builder;
import org.junit.Test;
public class HoodieWriteConfigTest {
@Test
public void testPropertyLoading() throws IOException {
Builder builder = HoodieWriteConfig.newBuilder().withPath("/tmp");
Map<String, String> params = Maps.newHashMap();
params.put(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP, "1");
params.put(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP, "5");
params.put(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP, "2");
ByteArrayOutputStream outStream = saveParamsIntoOutputStream(params);
ByteArrayInputStream inputStream = new ByteArrayInputStream(outStream.toByteArray());
try {
builder = builder.fromInputStream(inputStream);
} finally {
outStream.close();
inputStream.close();
}
HoodieWriteConfig config = builder.build();
assertEquals(config.getMaxCommitsToKeep(), 5);
assertEquals(config.getMinCommitsToKeep(), 2);
}
private ByteArrayOutputStream saveParamsIntoOutputStream(Map<String, String> params) throws IOException {
Properties properties = new Properties();
properties.putAll(params);
ByteArrayOutputStream outStream = new ByteArrayOutputStream();
properties.store(outStream, "Saved on " + new Date(System.currentTimeMillis()));
return outStream;
}
}

View File

@@ -0,0 +1,89 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.func;
import static org.apache.hudi.func.CopyOnWriteLazyInsertIterable.getTransformFunction;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.util.List;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.func.CopyOnWriteLazyInsertIterable.HoodieInsertValueGenResult;
import org.junit.After;
import org.junit.Assert;
import org.junit.Test;
import scala.Tuple2;
public class TestBoundedInMemoryExecutor {
private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator();
private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
private SparkBoundedInMemoryExecutor<HoodieRecord,
Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> executor = null;
@After
public void afterTest() {
if (this.executor != null) {
this.executor.shutdownNow();
this.executor = null;
}
}
@Test
public void testExecutor() throws Exception {
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, 100);
HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class);
when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024);
BoundedInMemoryQueueConsumer<HoodieInsertValueGenResult<HoodieRecord>, Integer> consumer =
new BoundedInMemoryQueueConsumer<HoodieInsertValueGenResult<HoodieRecord>, Integer>() {
private int count = 0;
@Override
protected void consumeOneRecord(HoodieInsertValueGenResult<HoodieRecord> record) {
count++;
}
@Override
protected void finish() {
}
@Override
protected Integer getResult() {
return count;
}
};
executor = new SparkBoundedInMemoryExecutor(hoodieWriteConfig,
hoodieRecords.iterator(), consumer, getTransformFunction(HoodieTestDataGenerator.avroSchema));
int result = executor.execute();
// It should buffer and write 100 records
Assert.assertEquals(result, 100);
// There should be no remaining records in the buffer
Assert.assertFalse(executor.isRemaining());
}
}

View File

@@ -0,0 +1,340 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.func;
import static org.apache.hudi.func.CopyOnWriteLazyInsertIterable.getTransformFunction;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.Semaphore;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.io.FileUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.util.DefaultSizeEstimator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SizeEstimator;
import org.apache.hudi.common.util.queue.BoundedInMemoryQueue;
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer;
import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.func.CopyOnWriteLazyInsertIterable.HoodieInsertValueGenResult;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import scala.Tuple2;
public class TestBoundedInMemoryQueue {
private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator();
private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
private ExecutorService executorService = null;
@Before
public void beforeTest() {
this.executorService = Executors.newFixedThreadPool(2);
}
@After
public void afterTest() {
if (this.executorService != null) {
this.executorService.shutdownNow();
this.executorService = null;
}
}
// Test to ensure that we are reading all records from queue iterator in the same order
// without any exceptions.
@SuppressWarnings("unchecked")
@Test(timeout = 60000)
public void testRecordReading() throws Exception {
final int numRecords = 128;
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema));
// Produce
Future<Boolean> resFuture =
executorService.submit(() -> {
new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue);
queue.close();
return true;
});
final Iterator<HoodieRecord> originalRecordIterator = hoodieRecords.iterator();
int recordsRead = 0;
while (queue.iterator().hasNext()) {
final HoodieRecord originalRecord = originalRecordIterator.next();
final Option<IndexedRecord> originalInsertValue = originalRecord.getData()
.getInsertValue(HoodieTestDataGenerator.avroSchema);
final HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next();
// Ensure that record ordering is guaranteed.
Assert.assertEquals(originalRecord, payload.record);
// cached insert value matches the expected insert value.
Assert.assertEquals(originalInsertValue,
payload.record.getData().getInsertValue(HoodieTestDataGenerator.avroSchema));
recordsRead++;
}
Assert.assertFalse(queue.iterator().hasNext() || originalRecordIterator.hasNext());
// all the records should be read successfully.
Assert.assertEquals(numRecords, recordsRead);
// should not throw any exceptions.
resFuture.get();
}
/**
* Test to ensure that we are reading all records from queue iterator when we have multiple producers
*/
@SuppressWarnings("unchecked")
@Test(timeout = 60000)
public void testCompositeProducerRecordReading() throws Exception {
final int numRecords = 1000;
final int numProducers = 40;
final List<List<HoodieRecord>> recs = new ArrayList<>();
final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
new BoundedInMemoryQueue(FileUtils.ONE_KB, getTransformFunction(HoodieTestDataGenerator.avroSchema));
// Record Key to <Producer Index, Rec Index within a producer>
Map<String, Tuple2<Integer, Integer>> keyToProducerAndIndexMap = new HashMap<>();
for (int i = 0; i < numProducers; i++) {
List<HoodieRecord> pRecs = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
int j = 0;
for (HoodieRecord r : pRecs) {
Assert.assertTrue(!keyToProducerAndIndexMap.containsKey(r.getRecordKey()));
keyToProducerAndIndexMap.put(r.getRecordKey(), new Tuple2<>(i, j));
j++;
}
recs.add(pRecs);
}
List<BoundedInMemoryQueueProducer<HoodieRecord>> producers = new ArrayList<>();
for (int i = 0; i < recs.size(); i++) {
final List<HoodieRecord> r = recs.get(i);
// Alternate between pull and push based iterators
if (i % 2 == 0) {
producers.add(new IteratorBasedQueueProducer<>(r.iterator()));
} else {
producers.add(new FunctionBasedQueueProducer<>((buf) -> {
Iterator<HoodieRecord> itr = r.iterator();
while (itr.hasNext()) {
try {
buf.insertRecord(itr.next());
} catch (Exception e) {
throw new HoodieException(e);
}
}
return true;
}));
}
}
final List<Future<Boolean>> futureList = producers.stream().map(producer -> {
return executorService.submit(() -> {
producer.produce(queue);
return true;
});
}).collect(Collectors.toList());
// Close queue
Future<Boolean> closeFuture = executorService.submit(() -> {
try {
for (Future f : futureList) {
f.get();
}
queue.close();
} catch (Exception e) {
throw new RuntimeException(e);
}
return true;
});
// Used to ensure that consumer sees the records generated by a single producer in FIFO order
Map<Integer, Integer> lastSeenMap = IntStream.range(0, numProducers).boxed()
.collect(Collectors.toMap(Function.identity(), x -> -1));
Map<Integer, Integer> countMap = IntStream.range(0, numProducers).boxed()
.collect(Collectors.toMap(Function.identity(), x -> 0));
// Read recs and ensure we have covered all producer recs.
while (queue.iterator().hasNext()) {
final HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next();
final HoodieRecord rec = payload.record;
Tuple2<Integer, Integer> producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey());
Integer lastSeenPos = lastSeenMap.get(producerPos._1());
countMap.put(producerPos._1(), countMap.get(producerPos._1()) + 1);
lastSeenMap.put(producerPos._1(), lastSeenPos + 1);
// Ensure we are seeing the next record generated
Assert.assertEquals(lastSeenPos + 1, producerPos._2().intValue());
}
for (int i = 0; i < numProducers; i++) {
// Ensure we have seen all the records for each producers
Assert.assertEquals(Integer.valueOf(numRecords), countMap.get(i));
}
//Ensure Close future is done
closeFuture.get();
}
// Test to ensure that record queueing is throttled when we hit memory limit.
@SuppressWarnings("unchecked")
@Test(timeout = 60000)
public void testMemoryLimitForBuffering() throws Exception {
final int numRecords = 128;
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
// maximum number of records to keep in memory.
final int recordLimit = 5;
final SizeEstimator<HoodieInsertValueGenResult<HoodieRecord>> sizeEstimator =
new DefaultSizeEstimator<>();
HoodieInsertValueGenResult<HoodieRecord> payload = getTransformFunction(HoodieTestDataGenerator.avroSchema)
.apply(hoodieRecords.get(0));
final long objSize = sizeEstimator.sizeEstimate(payload);
final long memoryLimitInBytes = recordLimit * objSize;
final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
new BoundedInMemoryQueue(memoryLimitInBytes,
getTransformFunction(HoodieTestDataGenerator.avroSchema));
// Produce
Future<Boolean> resFuture = executorService.submit(() -> {
new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue);
return true;
});
// waiting for permits to expire.
while (!isQueueFull(queue.rateLimiter)) {
Thread.sleep(10);
}
Assert.assertEquals(0, queue.rateLimiter.availablePermits());
Assert.assertEquals(recordLimit, queue.currentRateLimit);
Assert.assertEquals(recordLimit, queue.size());
Assert.assertEquals(recordLimit - 1, queue.samplingRecordCounter.get());
// try to read 2 records.
Assert.assertEquals(hoodieRecords.get(0), queue.iterator().next().record);
Assert.assertEquals(hoodieRecords.get(1), queue.iterator().next().record);
// waiting for permits to expire.
while (!isQueueFull(queue.rateLimiter)) {
Thread.sleep(10);
}
// No change is expected in rate limit or number of queued records. We only expect
// queueing thread to read
// 2 more records into the queue.
Assert.assertEquals(0, queue.rateLimiter.availablePermits());
Assert.assertEquals(recordLimit, queue.currentRateLimit);
Assert.assertEquals(recordLimit, queue.size());
Assert.assertEquals(recordLimit - 1 + 2, queue.samplingRecordCounter.get());
}
// Test to ensure that exception in either queueing thread or BufferedIterator-reader thread
// is propagated to
// another thread.
@SuppressWarnings("unchecked")
@Test(timeout = 60000)
public void testException() throws Exception {
final int numRecords = 256;
final List<HoodieRecord> hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords);
final SizeEstimator<Tuple2<HoodieRecord, Option<IndexedRecord>>> sizeEstimator =
new DefaultSizeEstimator<>();
// queue memory limit
HoodieInsertValueGenResult<HoodieRecord> payload = getTransformFunction(HoodieTestDataGenerator.avroSchema)
.apply(hoodieRecords.get(0));
final long objSize = sizeEstimator.sizeEstimate(new Tuple2<>(payload.record, payload.insertValue));
final long memoryLimitInBytes = 4 * objSize;
// first let us throw exception from queueIterator reader and test that queueing thread
// stops and throws
// correct exception back.
BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>> queue1 =
new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema));
// Produce
Future<Boolean> resFuture = executorService.submit(() -> {
new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue1);
return true;
});
// waiting for permits to expire.
while (!isQueueFull(queue1.rateLimiter)) {
Thread.sleep(10);
}
// notify queueing thread of an exception and ensure that it exits.
final Exception e = new Exception("Failing it :)");
queue1.markAsFailed(e);
try {
resFuture.get();
Assert.fail("exception is expected");
} catch (ExecutionException e1) {
Assert.assertEquals(HoodieException.class, e1.getCause().getClass());
Assert.assertEquals(e, e1.getCause().getCause());
}
// second let us raise an exception while doing record queueing. this exception should get
// propagated to
// queue iterator reader.
final RuntimeException expectedException = new RuntimeException("failing record reading");
final Iterator<HoodieRecord> mockHoodieRecordsIterator = mock(Iterator.class);
when(mockHoodieRecordsIterator.hasNext()).thenReturn(true);
when(mockHoodieRecordsIterator.next()).thenThrow(expectedException);
BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>> queue2 =
new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema));
// Produce
Future<Boolean> res = executorService.submit(() -> {
try {
new IteratorBasedQueueProducer<>(mockHoodieRecordsIterator).produce(queue2);
} catch (Exception ex) {
queue2.markAsFailed(ex);
throw ex;
}
return true;
});
try {
queue2.iterator().hasNext();
Assert.fail("exception is expected");
} catch (Exception e1) {
Assert.assertEquals(expectedException, e1.getCause());
}
// queueing thread should also have exited. make sure that it is not running.
try {
res.get();
Assert.fail("exception is expected");
} catch (ExecutionException e2) {
Assert.assertEquals(expectedException, e2.getCause());
}
}
private boolean isQueueFull(Semaphore rateLimiter) {
return (rateLimiter.availablePermits() == 0 && rateLimiter.hasQueuedThreads());
}
}

View File

@@ -0,0 +1,63 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.func;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.IOException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.parquet.hadoop.ParquetReader;
import org.junit.Assert;
import org.junit.Test;
public class TestParquetReaderIterator {
@Test
public void testParquetIteratorIdempotency() throws IOException {
ParquetReader reader = mock(ParquetReader.class);
// only 1 record in reader
when(reader.read()).thenReturn(1).thenReturn(null);
ParquetReaderIterator<Integer> iterator = new ParquetReaderIterator<>(reader);
int idempotencyCheckCounter = 0;
// call hasNext() 3 times
while (idempotencyCheckCounter < 3) {
Assert.assertTrue(iterator.hasNext());
idempotencyCheckCounter++;
}
}
@Test
public void testParquetIterator() throws IOException {
ParquetReader reader = mock(ParquetReader.class);
// only one record to read
when(reader.read()).thenReturn(1).thenReturn(null);
ParquetReaderIterator<Integer> iterator = new ParquetReaderIterator<>(reader);
// should return value even though hasNext() hasn't been called
Assert.assertTrue(iterator.next() == 1);
// no more entries to iterate on
Assert.assertFalse(iterator.hasNext());
try {
iterator.next();
} catch (HoodieIOException e) {
// should throw an exception since there is only 1 record
}
}
}

View File

@@ -0,0 +1,157 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.func;
import static org.junit.Assert.fail;
import java.io.File;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.TestRawTripPayload;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.ParquetUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.io.HoodieCreateHandle;
import org.apache.hudi.io.HoodieMergeHandle;
import org.apache.hudi.table.HoodieCopyOnWriteTable;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
public class TestUpdateMapFunction implements Serializable {
private String basePath = null;
private transient JavaSparkContext jsc = null;
@Before
public void init() throws Exception {
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
this.basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath);
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestUpdateMapFunction"));
}
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
@Test
public void testSchemaEvolutionOnUpdate() throws Exception {
// Create a bunch of records with a old version of schema
final HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
final HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
final List<WriteStatus> statuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
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> insertRecords = new ArrayList<>();
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
insertRecords
.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
insertRecords
.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
insertRecords
.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
HoodieCreateHandle createHandle = new HoodieCreateHandle(config, "100", table, rowChange1.getPartitionPath(),
"f1-0", insertRecords.iterator());
createHandle.write();
WriteStatus insertResult = createHandle.close();
return insertResult;
}).collect();
final Path commitFile = new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100"));
FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()).create(commitFile);
// Now try an update with an evolved schema
// Evolved schema does not have guarantee on preserving the original field ordering
final HoodieWriteConfig config2 = makeHoodieClientConfig("/exampleEvolvedSchema.txt");
final Schema schema = Schema.parse(config2.getSchema());
final WriteStatus insertResult = statuses.get(0);
String fileId = insertResult.getFileId();
final HoodieCopyOnWriteTable table2 = new HoodieCopyOnWriteTable(config2, jsc);
Assert.assertEquals(1, jsc.parallelize(Arrays.asList(1)).map(x -> {
// New content with values for the newly added field
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";
List<HoodieRecord> updateRecords = new ArrayList<>();
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
rowChange1);
record1.setCurrentLocation(new HoodieRecordLocation("100", fileId));
updateRecords.add(record1);
try {
HoodieMergeHandle mergeHandle = new HoodieMergeHandle(config2, "101", table2, updateRecords.iterator(), fileId);
Configuration conf = new Configuration();
AvroReadSupport.setAvroReadSchema(conf, mergeHandle.getWriterSchema());
List<GenericRecord> oldRecords = ParquetUtils.readAvroRecords(conf,
new Path(config2.getBasePath() + "/" + insertResult.getStat().getPath()));
for (GenericRecord rec : oldRecords) {
mergeHandle.write(rec);
}
mergeHandle.close();
} catch (ClassCastException e) {
fail("UpdateFunction could not read records written with exampleSchema.txt using the "
+ "exampleEvolvedSchema.txt");
}
return 1;
}).collect().size());
}
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();
}
}

View File

@@ -0,0 +1,146 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.index;
import java.io.File;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieHBaseIndexConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.hbase.DefaultHBaseQPSResourceAllocator;
import org.apache.hudi.index.hbase.HBaseIndex;
import org.apache.hudi.index.hbase.HBaseIndexQPSResourceAllocator;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
public class TestHBaseQPSResourceAllocator {
private static JavaSparkContext jsc = null;
private static String tableName = "test_table";
private String basePath = null;
private static HBaseTestingUtility utility;
private static Configuration hbaseConfig;
private static String QPS_TEST_SUFFIX_PATH = "qps_test_suffix";
@AfterClass
public static void clean() {
if (jsc != null) {
jsc.stop();
}
}
@BeforeClass
public static void init() throws Exception {
utility = new HBaseTestingUtility();
utility.startMiniCluster();
hbaseConfig = utility.getConnection().getConfiguration();
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestQPSResourceAllocator"));
}
@After
public void clear() {
if (basePath != null) {
new File(basePath).delete();
}
}
@Before
public void before() throws Exception {
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath() + QPS_TEST_SUFFIX_PATH;
// Initialize table
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
}
@Test
public void testsDefaultQPSResourceAllocator() {
HoodieWriteConfig config = getConfig(Option.empty());
HBaseIndex index = new HBaseIndex(config);
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config);
Assert.assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(),
DefaultHBaseQPSResourceAllocator.class.getName());
Assert.assertEquals(config.getHbaseIndexQPSFraction(),
hBaseIndexQPSResourceAllocator.acquireQPSResources(config.getHbaseIndexQPSFraction(), 100), 0.0f);
}
@Test
public void testsExplicitDefaultQPSResourceAllocator() {
HoodieWriteConfig config = getConfig(Option.of(HoodieHBaseIndexConfig.DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
HBaseIndex index = new HBaseIndex(config);
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config);
Assert.assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(),
DefaultHBaseQPSResourceAllocator.class.getName());
Assert.assertEquals(config.getHbaseIndexQPSFraction(),
hBaseIndexQPSResourceAllocator.acquireQPSResources(config.getHbaseIndexQPSFraction(), 100), 0.0f);
}
@Test
public void testsInvalidQPSResourceAllocator() {
HoodieWriteConfig config = getConfig(Option.of("InvalidResourceAllocatorClassName"));
HBaseIndex index = new HBaseIndex(config);
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config);
Assert.assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(),
DefaultHBaseQPSResourceAllocator.class.getName());
Assert.assertEquals(config.getHbaseIndexQPSFraction(),
hBaseIndexQPSResourceAllocator.acquireQPSResources(config.getHbaseIndexQPSFraction(), 100), 0.0f);
}
private HoodieWriteConfig getConfig(Option<String> resourceAllocatorClass) {
HoodieHBaseIndexConfig hoodieHBaseIndexConfig = getConfigWithResourceAllocator(resourceAllocatorClass);
return getConfigBuilder(hoodieHBaseIndexConfig).build();
}
private HoodieWriteConfig.Builder getConfigBuilder(HoodieHBaseIndexConfig hoodieHBaseIndexConfig) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(1, 1).withCompactionConfig(
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).withInlineCompaction(false)
.build()).withAutoCommit(false)
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
.forTable("test-trip-table").withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.HBASE)
.withHBaseIndexConfig(hoodieHBaseIndexConfig)
.build());
}
private HoodieHBaseIndexConfig getConfigWithResourceAllocator(Option<String> resourceAllocatorClass) {
HoodieHBaseIndexConfig.Builder builder =
new HoodieHBaseIndexConfig.Builder()
.hbaseZkPort(Integer.valueOf(hbaseConfig.get("hbase.zookeeper.property.clientPort")))
.hbaseZkQuorum(hbaseConfig.get("hbase.zookeeper.quorum")).hbaseTableName(tableName)
.hbaseIndexGetBatchSize(100);
if (resourceAllocatorClass.isPresent()) {
builder.withQPSResourceAllocatorType(resourceAllocatorClass.get());
}
return builder.build();
}
}

View File

@@ -0,0 +1,458 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.index;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Matchers.anyObject;
import static org.mockito.Mockito.atMost;
import static org.mockito.Mockito.times;
import java.io.File;
import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Connection;
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.hudi.HoodieWriteClient;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieHBaseIndexConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.hbase.DefaultHBaseQPSResourceAllocator;
import org.apache.hudi.index.hbase.HBaseIndex;
import org.apache.hudi.index.hbase.HBaseIndex.HbasePutBatchSizeCalculator;
import org.apache.hudi.index.hbase.HBaseIndexQPSResourceAllocator;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.junit.runners.MethodSorters;
import org.mockito.Mockito;
import scala.Tuple2;
/**
* Note :: HBaseTestingUtility is really flaky with issues where the HbaseMiniCluster fails to shutdown across tests,
* (see one problem here : https://issues.apache .org/jira/browse/HBASE-15835). Hence, the need to use
* MethodSorters.NAME_ASCENDING to make sure the tests run in order. Please alter the order of tests running carefully.
*/
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
public class TestHbaseIndex {
private static JavaSparkContext jsc = null;
private static HBaseTestingUtility utility;
private static Configuration hbaseConfig;
private static String tableName = "test_table";
private String basePath = null;
private transient FileSystem fs;
private HoodieWriteClient writeClient;
public TestHbaseIndex() throws Exception {
}
@AfterClass
public static void clean() throws Exception {
if (jsc != null) {
jsc.stop();
}
if (utility != null) {
utility.shutdownMiniCluster();
}
}
@BeforeClass
public static void init() throws Exception {
// Initialize HbaseMiniCluster
utility = new HBaseTestingUtility();
utility.startMiniCluster();
hbaseConfig = utility.getConnection().getConfiguration();
utility.createTable(TableName.valueOf(tableName), Bytes.toBytes("_s"));
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHbaseIndex"));
jsc.hadoopConfiguration().addResource(utility.getConfiguration());
}
@After
public void clear() throws Exception {
if (null != writeClient) {
writeClient.close();
writeClient = null;
}
if (basePath != null) {
new File(basePath).delete();
}
}
@Before
public void before() throws Exception {
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
// Initialize table
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
}
private HoodieWriteClient getWriteClient(HoodieWriteConfig config) throws Exception {
if (null != writeClient) {
writeClient.close();
}
writeClient = new HoodieWriteClient(jsc, config);
return writeClient;
}
@Test
public void testSimpleTagLocationAndUpdate() throws Exception {
String newCommitTime = "001";
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
// Load to memory
HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config);
HoodieWriteClient writeClient = getWriteClient(config);
writeClient.startCommit();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Test tagLocation without any entries in index
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
// Insert 200 records
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
assertNoWriteErrors(writeStatues.collect());
// Now tagLocation for these records, hbaseIndex should not tag them since it was a failed
// commit
javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
// Now commit this & update location of records inserted and validate no errors
writeClient.commit(newCommitTime, writeStatues);
// Now tagLocation for these records, hbaseIndex should tag them correctly
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assertTrue(javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200);
assertTrue(javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count() == 200);
assertTrue(javaRDD.filter(
record -> (record.getCurrentLocation() != null && record.getCurrentLocation().getInstantTime()
.equals(newCommitTime))).distinct().count() == 200);
}
@Test
public void testTagLocationAndDuplicateUpdate() throws Exception {
String newCommitTime = "001";
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 10);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
// Load to memory
HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config);
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
writeClient.startCommit();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
JavaRDD<HoodieRecord> javaRDD1 = index.tagLocation(writeRecords, jsc, hoodieTable);
// Duplicate upsert and ensure correctness is maintained
writeClient.upsert(writeRecords, newCommitTime);
assertNoWriteErrors(writeStatues.collect());
// Now commit this & update location of records inserted and validate no errors
writeClient.commit(newCommitTime, writeStatues);
// Now tagLocation for these records, hbaseIndex should tag them correctly
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assertTrue(javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 10);
assertTrue(javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count() == 10);
assertTrue(javaRDD.filter(
record -> (record.getCurrentLocation() != null && record.getCurrentLocation().getInstantTime()
.equals(newCommitTime))).distinct().count() == 10);
}
@Test
public void testSimpleTagLocationAndUpdateWithRollback() throws Exception {
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
// Load to memory
HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config);
HoodieWriteClient writeClient = getWriteClient(config);
String newCommitTime = writeClient.startCommit();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Insert 200 records
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
assertNoWriteErrors(writeStatues.collect());
// commit this upsert
writeClient.commit(newCommitTime, writeStatues);
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Now tagLocation for these records, hbaseIndex should tag them
JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 200);
// check tagged records are tagged with correct fileIds
List<String> fileIds = writeStatues.map(status -> status.getFileId()).collect();
assert (javaRDD.filter(record -> record.getCurrentLocation().getFileId() == null).collect().size() == 0);
List<String> taggedFileIds = javaRDD.map(record -> record.getCurrentLocation().getFileId()).distinct().collect();
// both lists should match
assertTrue(taggedFileIds.containsAll(fileIds) && fileIds.containsAll(taggedFileIds));
// Rollback the last commit
writeClient.rollback(newCommitTime);
hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled
// back commit
javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0);
}
@Test
public void testTotalGetsBatching() throws Exception {
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config);
// Mock hbaseConnection and related entities
Connection hbaseConnection = Mockito.mock(Connection.class);
HTable table = Mockito.mock(HTable.class);
Mockito.when(hbaseConnection.getTable(TableName.valueOf(tableName))).thenReturn(table);
Mockito.when(table.get((List<Get>) anyObject())).thenReturn(new Result[0]);
// only for test, set the hbaseConnection to mocked object
index.setHbaseConnection(hbaseConnection);
HoodieWriteClient writeClient = getWriteClient(config);
// start a commit and generate test data
String newCommitTime = writeClient.startCommit();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Insert 250 records
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
assertNoWriteErrors(writeStatues.collect());
// Now tagLocation for these records, hbaseIndex should tag them
index.tagLocation(writeRecords, jsc, hoodieTable);
// 3 batches should be executed given batchSize = 100 and parallelism = 1
Mockito.verify(table, times(3)).get((List<Get>) anyObject());
}
@Test
public void testTotalPutsBatching() throws Exception {
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config);
HoodieWriteClient writeClient = getWriteClient(config);
// start a commit and generate test data
String newCommitTime = writeClient.startCommit();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
// Insert 200 records
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
// commit this upsert
writeClient.commit(newCommitTime, writeStatues);
// Mock hbaseConnection and related entities
Connection hbaseConnection = Mockito.mock(Connection.class);
HTable table = Mockito.mock(HTable.class);
Mockito.when(hbaseConnection.getTable(TableName.valueOf(tableName))).thenReturn(table);
Mockito.when(table.get((List<Get>) anyObject())).thenReturn(new Result[0]);
// only for test, set the hbaseConnection to mocked object
index.setHbaseConnection(hbaseConnection);
// Get all the files generated
int numberOfDataFileIds = (int) writeStatues.map(status -> status.getFileId()).distinct().count();
index.updateLocation(writeStatues, jsc, hoodieTable);
// 3 batches should be executed given batchSize = 100 and <=numberOfDataFileIds getting updated,
// so each fileId ideally gets updates
Mockito.verify(table, atMost(numberOfDataFileIds)).put((List<Put>) anyObject());
}
@Test
public void testPutBatchSizeCalculation() {
HbasePutBatchSizeCalculator batchSizeCalculator = new HbasePutBatchSizeCalculator();
// All asserts cases below are derived out of the first
// example below, with change in one parameter at a time.
int putBatchSize = batchSizeCalculator.getBatchSize(10, 16667, 1200, 200, 100, 0.1f);
// Expected batchSize is 8 because in that case, total request sent in one second is below
// 8 (batchSize) * 200 (parallelism) * 10 (maxReqsInOneSecond) * 10 (numRegionServers) * 0.1 (qpsFraction)) => 16000
// We assume requests get distributed to Region Servers uniformly, so each RS gets 1600 request
// 1600 happens to be 10% of 16667 (maxQPSPerRegionServer) as expected.
Assert.assertEquals(putBatchSize, 8);
// Number of Region Servers are halved, total requests sent in a second are also halved, so batchSize is also halved
int putBatchSize2 = batchSizeCalculator.getBatchSize(5, 16667, 1200, 200, 100, 0.1f);
Assert.assertEquals(putBatchSize2, 4);
// If the parallelism is halved, batchSize has to double
int putBatchSize3 = batchSizeCalculator.getBatchSize(10, 16667, 1200, 100, 100, 0.1f);
Assert.assertEquals(putBatchSize3, 16);
// If the parallelism is halved, batchSize has to double.
// This time parallelism is driven by numTasks rather than numExecutors
int putBatchSize4 = batchSizeCalculator.getBatchSize(10, 16667, 100, 200, 100, 0.1f);
Assert.assertEquals(putBatchSize4, 16);
// If sleepTimeMs is halved, batchSize has to halve
int putBatchSize5 = batchSizeCalculator.getBatchSize(10, 16667, 1200, 200, 100, 0.05f);
Assert.assertEquals(putBatchSize5, 4);
// If maxQPSPerRegionServer is doubled, batchSize also doubles
int putBatchSize6 = batchSizeCalculator.getBatchSize(10, 33334, 1200, 200, 100, 0.1f);
Assert.assertEquals(putBatchSize6, 16);
}
@Test
public void testsHBasePutAccessParallelism() {
HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config);
final JavaRDD<WriteStatus> writeStatusRDD = jsc.parallelize(
Arrays.asList(
getSampleWriteStatus(1, 2),
getSampleWriteStatus(0, 3),
getSampleWriteStatus(10, 0)),
10);
final Tuple2<Long, Integer> tuple = index.getHBasePutAccessParallelism(writeStatusRDD);
final int hbasePutAccessParallelism = Integer.parseInt(tuple._2.toString());
final int hbaseNumPuts = Integer.parseInt(tuple._1.toString());
Assert.assertEquals(10, writeStatusRDD.getNumPartitions());
Assert.assertEquals(2, hbasePutAccessParallelism);
Assert.assertEquals(11, hbaseNumPuts);
}
@Test
public void testsHBasePutAccessParallelismWithNoInserts() {
HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config);
final JavaRDD<WriteStatus> writeStatusRDD = jsc.parallelize(
Arrays.asList(
getSampleWriteStatus(0, 2),
getSampleWriteStatus(0, 1)),
10);
final Tuple2<Long, Integer> tuple = index.getHBasePutAccessParallelism(writeStatusRDD);
final int hbasePutAccessParallelism = Integer.parseInt(tuple._2.toString());
final int hbaseNumPuts = Integer.parseInt(tuple._1.toString());
Assert.assertEquals(10, writeStatusRDD.getNumPartitions());
Assert.assertEquals(0, hbasePutAccessParallelism);
Assert.assertEquals(0, hbaseNumPuts);
}
@Test
public void testsHBaseIndexDefaultQPSResourceAllocator() {
HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config);
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config);
Assert.assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(),
DefaultHBaseQPSResourceAllocator.class.getName());
Assert.assertEquals(config.getHbaseIndexQPSFraction(),
hBaseIndexQPSResourceAllocator.acquireQPSResources(config.getHbaseIndexQPSFraction(), 100), 0.0f);
}
private WriteStatus getSampleWriteStatus(final int numInserts, final int numUpdateWrites) {
final WriteStatus writeStatus = new WriteStatus(false, 0.1);
HoodieWriteStat hoodieWriteStat = new HoodieWriteStat();
hoodieWriteStat.setNumInserts(numInserts);
hoodieWriteStat.setNumUpdateWrites(numUpdateWrites);
writeStatus.setStat(hoodieWriteStat);
return writeStatus;
}
private void assertNoWriteErrors(List<WriteStatus> statuses) {
// Verify there are no errors
for (WriteStatus status : statuses) {
assertFalse("Errors found in write of " + status.getFileId(), status.hasErrors());
}
}
private HoodieWriteConfig getConfig() {
return getConfigBuilder().build();
}
private HoodieWriteConfig.Builder getConfigBuilder() {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(1, 1).withCompactionConfig(
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).withInlineCompaction(false)
.build()).withAutoCommit(false)
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
.forTable("test-trip-table").withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.HBASE)
.withHBaseIndexConfig(
new HoodieHBaseIndexConfig.Builder()
.hbaseZkPort(Integer.valueOf(hbaseConfig.get("hbase.zookeeper.property.clientPort")))
.hbaseZkQuorum(hbaseConfig.get("hbase.zookeeper.quorum")).hbaseTableName(tableName)
.hbaseIndexGetBatchSize(100).build())
.build());
}
}

View File

@@ -0,0 +1,79 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.index;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.IOException;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.config.HoodieHBaseIndexConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.bloom.HoodieBloomIndex;
import org.apache.hudi.index.hbase.HBaseIndex;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
public class TestHoodieIndex {
private transient JavaSparkContext jsc = null;
private String basePath = null;
@Before
public void init() throws IOException {
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieIndex"));
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
}
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
@Test
public void testCreateIndex() throws Exception {
HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
// Different types
HoodieWriteConfig config = clientConfigBuilder.withPath(basePath).withIndexConfig(
indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE)
.withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder().build()).build()).build();
assertTrue(HoodieIndex.createIndex(config, jsc) instanceof HBaseIndex);
config = clientConfigBuilder.withPath(basePath)
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
assertTrue(HoodieIndex.createIndex(config, jsc) instanceof InMemoryHashIndex);
config = clientConfigBuilder.withPath(basePath)
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
assertTrue(HoodieIndex.createIndex(config, jsc) instanceof HoodieBloomIndex);
}
}

View File

@@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.index.bloom;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.hudi.common.util.collection.Pair;
import org.junit.Test;
public class TestBucketizedBloomCheckPartitioner {
@Test
public void testAssignmentCorrectness() {
Map<String, Long> fileToComparisons = new HashMap<String, Long>() {
{
put("f1", 40L);
put("f2", 35L);
put("f3", 20L);
}
};
BucketizedBloomCheckPartitioner p = new BucketizedBloomCheckPartitioner(4, fileToComparisons, 10);
Map<String, List<Integer>> assignments = p.getFileGroupToPartitions();
assertEquals("f1 should have 4 buckets", 4, assignments.get("f1").size());
assertEquals("f2 should have 4 buckets", 4, assignments.get("f2").size());
assertEquals("f3 should have 2 buckets", 2, assignments.get("f3").size());
assertArrayEquals("f1 spread across 3 partitions", new Integer[]{0, 0, 1, 3},
assignments.get("f1").toArray());
assertArrayEquals("f2 spread across 3 partitions", new Integer[]{1, 2, 2, 0},
assignments.get("f2").toArray());
assertArrayEquals("f3 spread across 2 partitions", new Integer[]{3, 1},
assignments.get("f3").toArray());
}
@Test
public void testUniformPacking() {
// evenly distribute 10 buckets/file across 100 partitions
Map<String, Long> comparisons1 = new HashMap<String, Long>() {
{
IntStream.range(0, 10).forEach(f -> put("f" + f, 100L));
}
};
BucketizedBloomCheckPartitioner partitioner = new BucketizedBloomCheckPartitioner(100, comparisons1, 10);
Map<String, List<Integer>> assignments = partitioner.getFileGroupToPartitions();
assignments.entrySet().stream().forEach(e -> assertEquals(10, e.getValue().size()));
Map<Integer, Long> partitionToNumBuckets = assignments.entrySet().stream()
.flatMap(e -> e.getValue().stream().map(p -> Pair.of(p, e.getKey())))
.collect(Collectors.groupingBy(Pair::getLeft, Collectors.counting()));
partitionToNumBuckets.entrySet().stream().forEach(e -> assertEquals(1L, e.getValue().longValue()));
}
@Test
public void testNumPartitions() {
Map<String, Long> comparisons1 = new HashMap<String, Long>() {
{
IntStream.range(0, 10).forEach(f -> put("f" + f, 100L));
}
};
BucketizedBloomCheckPartitioner p = new BucketizedBloomCheckPartitioner(10000, comparisons1, 10);
assertEquals("num partitions must equal total buckets", 100, p.numPartitions());
}
}

View File

@@ -0,0 +1,505 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.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.collect.Lists;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
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.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.BloomFilter;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.TestRawTripPayload;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.HoodieAvroUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.io.HoodieKeyLookupHandle;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import scala.Tuple2;
@RunWith(Parameterized.class)
public class TestHoodieBloomIndex {
private JavaSparkContext jsc = null;
private String basePath = null;
private transient FileSystem fs;
private String schemaStr;
private Schema schema;
private boolean rangePruning;
private boolean treeFiltering;
private boolean bucketizedChecking;
@Parameterized.Parameters(name = "{index}: Test with rangePruning={0}, treeFiltering ={1}, bucketizedChecking is:{2}")
public static Collection<Object[]> data() {
Object[][] data = new Object[][]{{true, true, true}, {false, true, true}, {true, true, false},
{true, false, true}};
return Arrays.asList(data);
}
public TestHoodieBloomIndex(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) {
this.rangePruning = rangePruning;
this.treeFiltering = treeFiltering;
this.bucketizedChecking = bucketizedChecking;
}
@Before
public void init() throws IOException {
// 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();
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
HoodieTestUtils.init(jsc.hadoopConfiguration(), 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));
}
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
private HoodieWriteConfig makeConfig() {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withIndexConfig(HoodieIndexConfig.newBuilder()
.bloomIndexPruneByRanges(rangePruning)
.bloomIndexTreebasedFilter(treeFiltering)
.bloomIndexBucketizedChecking(bucketizedChecking)
.bloomIndexKeysPerBucket(2)
.build())
.build();
return config;
}
@Test
public void testLoadInvolvedFiles() throws IOException {
HoodieWriteConfig config = makeConfig();
HoodieBloomIndex index = new HoodieBloomIndex(config);
// Create some partitions, and put some files
// "2016/01/21": 0 file
// "2016/04/01": 1 file (2_0_20160401010101.parquet)
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet,
// 4_0_20150312101010.parquet)
new File(basePath + "/2016/01/21").mkdirs();
new File(basePath + "/2016/04/01").mkdirs();
new File(basePath + "/2015/03/12").mkdirs();
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);
HoodieClientTestUtils
.writeParquetFile(basePath, "2016/04/01", "2_0_20160401010101.parquet",
Lists.newArrayList(), schema, null, false);
HoodieClientTestUtils
.writeParquetFile(basePath, "2015/03/12", "1_0_20150312101010.parquet",
Lists.newArrayList(), schema, null, false);
HoodieClientTestUtils
.writeParquetFile(basePath, "2015/03/12", "3_0_20150312101010.parquet",
Arrays.asList(record1), schema, null, false);
HoodieClientTestUtils
.writeParquetFile(basePath, "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");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, jsc, table);
// Still 0, as no valid commit
assertEquals(filesList.size(), 0);
// Add some commits
new File(basePath + "/.hoodie").mkdirs();
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
table = HoodieTable.getHoodieTable(metadata, config, jsc);
filesList = index.loadInvolvedFiles(partitions, jsc, table);
assertEquals(filesList.size(), 4);
if (rangePruning) {
// these files will not have the key ranges
assertNull(filesList.get(0)._2().getMaxRecordKey());
assertNull(filesList.get(0)._2().getMinRecordKey());
assertFalse(filesList.get(1)._2().hasKeyRanges());
assertNotNull(filesList.get(2)._2().getMaxRecordKey());
assertNotNull(filesList.get(2)._2().getMinRecordKey());
assertTrue(filesList.get(3)._2().hasKeyRanges());
// no longer sorted, but should have same files.
List<Tuple2<String, BloomIndexFileInfo>> expected = Arrays.asList(
new Tuple2<>("2016/04/01", new BloomIndexFileInfo("2")),
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("1")),
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("3", "000", "000")),
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("4", "001", "003")));
assertEquals(expected, filesList);
}
}
@Test
public void testRangePruning() {
HoodieWriteConfig config = makeConfig();
HoodieBloomIndex index = new HoodieBloomIndex(config);
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")));
JavaPairRDD<String, String> partitionRecordKeyPairRDD = jsc.parallelize(Arrays.asList(
new Tuple2<>("2017/10/22", "003"), new Tuple2<>("2017/10/22", "002"), new Tuple2<>("2017/10/22", "005"),
new Tuple2<>("2017/10/22", "004"))).mapToPair(t -> t);
List<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.getRecordKey(), Collectors.mapping(t -> t._1, Collectors.toList())));
assertEquals(4, recordKeyToFileComps.size());
assertEquals(new HashSet<>(Arrays.asList("f1", "f3", "f4")), new HashSet<>(recordKeyToFileComps.get("002")));
assertEquals(new HashSet<>(Arrays.asList("f1", "f3", "f4")), new HashSet<>(recordKeyToFileComps.get("003")));
assertEquals(new HashSet<>(Arrays.asList("f1", "f4")), new HashSet<>(recordKeyToFileComps.get("004")));
assertEquals(new HashSet<>(Arrays.asList("f1", "f4")), new HashSet<>(recordKeyToFileComps.get("005")));
}
@Test
public void testCheckUUIDsAgainstOneFile() throws IOException, InterruptedException {
// 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 = HoodieClientTestUtils
.writeParquetFile(basePath, "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 = HoodieKeyLookupHandle.checkCandidatesAgainstFile(jsc.hadoopConfiguration(), 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(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = makeConfig();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
try {
bloomIndex.tagLocation(recordRDD, jsc, table);
} catch (IllegalArgumentException e) {
fail("EmptyRDD should not result in IllegalArgumentException: Positive number of slices " + "required");
}
}
@Test
public void testTagLocation() throws Exception {
// We have some records to be tagged (two different partitions)
String rowKey1 = UUID.randomUUID().toString();
String rowKey2 = UUID.randomUUID().toString();
String rowKey3 = UUID.randomUUID().toString();
String recordStr1 = "{\"_row_key\":\"" + rowKey1 + "\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"" + rowKey2 + "\","
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"" + rowKey3 + "\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
// place same row key under a different partition.
String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\","
+ "\"time\":\"2015-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);
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
// Also create the metadata and config
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = makeConfig();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table);
// Should not find any files
for (HoodieRecord record : taggedRecordRDD.collect()) {
assertFalse(record.isCurrentLocationKnown());
}
// We create three parquet file, each having one record. (two different partitions)
String filename1 =
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Arrays.asList(record1), schema, null, true);
String filename2 =
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Arrays.asList(record2), schema, null, true);
String filename3 =
HoodieClientTestUtils.writeParquetFile(basePath, "2015/01/31", Arrays.asList(record4), schema, null, true);
// We do the tag again
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metadata, config, jsc);
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table);
// Check results
for (HoodieRecord record : taggedRecordRDD.collect()) {
if (record.getRecordKey().equals(rowKey1)) {
if (record.getPartitionPath().equals("2015/01/31")) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename3)));
} else {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename1)));
}
} else if (record.getRecordKey().equals(rowKey2)) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename2)));
} else if (record.getRecordKey().equals(rowKey3)) {
assertTrue(!record.isCurrentLocationKnown());
}
}
}
@Test
public void testCheckExists() throws Exception {
// We have some records to be tagged (two different partitions)
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}";
// record key same as recordStr2
String recordStr4 = "{\"_row_key\":\"2eb5b87b-1feu-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());
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
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = makeConfig();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
// Let's tag
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> taggedRecordRDD = bloomIndex
.fetchRecordLocation(keysRDD, jsc, table);
// Should not find any files
for (Tuple2<HoodieKey, Option<Pair<String, String>>> record : taggedRecordRDD.collect()) {
assertTrue(!record._2.isPresent());
}
// We create three parquet file, each having one record. (two different partitions)
String filename1 =
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Arrays.asList(record1), schema, null, true);
String filename2 =
HoodieClientTestUtils.writeParquetFile(basePath, "2016/01/31", Arrays.asList(record2), schema, null, true);
String filename3 =
HoodieClientTestUtils.writeParquetFile(basePath, "2015/01/31", Arrays.asList(record4), schema, null, true);
// We do the tag again
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metadata, config, jsc);
taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, table);
// Check results
for (Tuple2<HoodieKey, Option<Pair<String, String>>> record : taggedRecordRDD.collect()) {
if (record._1.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
assertTrue(record._2.isPresent());
assertEquals(FSUtils.getFileId(filename1), record._2.get().getRight());
} else if (record._1.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
assertTrue(record._2.isPresent());
if (record._1.getPartitionPath().equals("2015/01/31")) {
assertEquals(FSUtils.getFileId(filename3), record._2.get().getRight());
} else {
assertEquals(FSUtils.getFileId(filename2), record._2.get().getRight());
}
} else if (record._1.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
assertTrue(!record._2.isPresent());
}
}
}
@Test
public void testBloomFilterFalseError() throws IOException, InterruptedException {
// 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}";
// We write record1 to a parquet file, using a bloom filter having both records
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);
BloomFilter filter = new BloomFilter(10000, 0.0000001);
filter.add(record2.getRecordKey());
String filename = HoodieClientTestUtils
.writeParquetFile(basePath, "2016/01/31",
Arrays.asList(record1), schema, filter, true);
assertTrue(filter.mightContain(record1.getRecordKey()));
assertTrue(filter.mightContain(record2.getRecordKey()));
// We do the tag
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = makeConfig();
HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, 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());
}
}
}
}

View File

@@ -0,0 +1,313 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.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 com.google.common.collect.Lists;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.TestRawTripPayload;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.HoodieAvroUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import scala.Tuple2;
public class TestHoodieGlobalBloomIndex {
private JavaSparkContext jsc = null;
private String basePath = null;
private transient FileSystem fs;
private String schemaStr;
private Schema schema;
public TestHoodieGlobalBloomIndex() throws Exception {
}
@Before
public void init() throws IOException {
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieGlobalBloomIndex"));
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
HoodieTestUtils.init(jsc.hadoopConfiguration(), 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));
}
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
@Test
public void testLoadInvolvedFiles() throws IOException {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config);
// Create some partitions, and put some files, along with the meta file
// "2016/01/21": 0 file
// "2016/04/01": 1 file (2_0_20160401010101.parquet)
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet,
// 4_0_20150312101010.parquet)
new File(basePath + "/2016/01/21").mkdirs();
new File(basePath + "/2016/01/21/" + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE).createNewFile();
new File(basePath + "/2016/04/01").mkdirs();
new File(basePath + "/2016/04/01/" + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE).createNewFile();
new File(basePath + "/2015/03/12").mkdirs();
new File(basePath + "/2015/03/12/" + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE).createNewFile();
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);
HoodieClientTestUtils
.writeParquetFile(basePath, "2016/04/01", "2_0_20160401010101.parquet",
Lists.newArrayList(), schema, null, false);
HoodieClientTestUtils
.writeParquetFile(basePath, "2015/03/12", "1_0_20150312101010.parquet",
Lists.newArrayList(), schema, null, false);
HoodieClientTestUtils
.writeParquetFile(basePath, "2015/03/12", "3_0_20150312101010.parquet",
Arrays.asList(record1), schema, null, false);
HoodieClientTestUtils
.writeParquetFile(basePath, "2015/03/12", "4_0_20150312101010.parquet",
Arrays.asList(record2, record3, record4), schema, null, false);
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01");
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
// partitions will NOT be respected by this loadInvolvedFiles(...) call
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, jsc, table);
// Still 0, as no valid commit
assertEquals(filesList.size(), 0);
// Add some commits
new File(basePath + "/.hoodie").mkdirs();
new File(basePath + "/.hoodie/20160401010101.commit").createNewFile();
new File(basePath + "/.hoodie/20150312101010.commit").createNewFile();
table = HoodieTable.getHoodieTable(metadata, config, jsc);
filesList = index.loadInvolvedFiles(partitions, jsc, table);
assertEquals(filesList.size(), 4);
Map<String, BloomIndexFileInfo> filesMap = toFileMap(filesList);
// key ranges checks
assertNull(filesMap.get("2016/04/01/2").getMaxRecordKey());
assertNull(filesMap.get("2016/04/01/2").getMinRecordKey());
assertFalse(filesMap.get("2015/03/12/1").hasKeyRanges());
assertNotNull(filesMap.get("2015/03/12/3").getMaxRecordKey());
assertNotNull(filesMap.get("2015/03/12/3").getMinRecordKey());
assertTrue(filesMap.get("2015/03/12/3").hasKeyRanges());
Map<String, BloomIndexFileInfo> expected = new HashMap<>();
expected.put("2016/04/01/2", new BloomIndexFileInfo("2"));
expected.put("2015/03/12/1", new BloomIndexFileInfo("1"));
expected.put("2015/03/12/3",
new BloomIndexFileInfo("3", "000", "000"));
expected.put("2015/03/12/4",
new BloomIndexFileInfo("4", "001", "003"));
assertEquals(expected, filesMap);
}
@Test
public void testExplodeRecordRDDWithFileComparisons() {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config);
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")));
partitionToFileIndexInfo.put("2017/10/23", Arrays.asList(
new BloomIndexFileInfo("f4", "002", "007"), new BloomIndexFileInfo("f5", "009", "010")));
// the partition partition of the key of the incoming records will be ignored
JavaPairRDD<String, String> partitionRecordKeyPairRDD = jsc.parallelize(Arrays.asList(
new Tuple2<>("2017/10/21", "003"), new Tuple2<>("2017/10/22", "002"), new Tuple2<>("2017/10/22", "005"),
new Tuple2<>("2017/10/23", "004"))).mapToPair(t -> t);
List<Tuple2<String, HoodieKey>> comparisonKeyList =
index.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect();
/* expecting:
f4, HoodieKey { recordKey=003 partitionPath=2017/10/23}
f1, HoodieKey { recordKey=003 partitionPath=2017/10/22}
f3, HoodieKey { recordKey=003 partitionPath=2017/10/22}
f4, HoodieKey { recordKey=002 partitionPath=2017/10/23}
f1, HoodieKey { recordKey=002 partitionPath=2017/10/22}
f3, HoodieKey { recordKey=002 partitionPath=2017/10/22}
f4, HoodieKey { recordKey=005 partitionPath=2017/10/23}
f1, HoodieKey { recordKey=005 partitionPath=2017/10/22}
f4, HoodieKey { recordKey=004 partitionPath=2017/10/23}
f1, HoodieKey { recordKey=004 partitionPath=2017/10/22}
*/
assertEquals(10, comparisonKeyList.size());
Map<String, List<String>> recordKeyToFileComps = comparisonKeyList.stream()
.collect(Collectors.groupingBy(t -> t._2.getRecordKey(), Collectors.mapping(Tuple2::_1, Collectors.toList())));
assertEquals(4, recordKeyToFileComps.size());
assertEquals(new HashSet<>(Arrays.asList("f4", "f1", "f3")), new HashSet<>(recordKeyToFileComps.get("002")));
assertEquals(new HashSet<>(Arrays.asList("f4", "f1", "f3")), new HashSet<>(recordKeyToFileComps.get("003")));
assertEquals(new HashSet<>(Arrays.asList("f4", "f1")), new HashSet<>(recordKeyToFileComps.get("004")));
assertEquals(new HashSet<>(Arrays.asList("f4", "f1")), new HashSet<>(recordKeyToFileComps.get("005")));
}
@Test
public void testTagLocation() throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config);
// Create some partitions, and put some files, along with the meta file
// "2016/01/21": 0 file
// "2016/04/01": 1 file (2_0_20160401010101.parquet)
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet,
// 4_0_20150312101010.parquet)
new File(basePath + "/2016/01/21").mkdirs();
new File(basePath + "/2016/01/21/" + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE).createNewFile();
new File(basePath + "/2016/04/01").mkdirs();
new File(basePath + "/2016/04/01/" + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE).createNewFile();
new File(basePath + "/2015/03/12").mkdirs();
new File(basePath + "/2015/03/12/" + HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE).createNewFile();
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);
// this record will be saved in table and will be tagged to the incoming record5
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);
// this has the same record key as record4 but different time so different partition, but globalbloomIndex should
// tag the original partition of the saved record4
TestRawTripPayload rowChange5 = new TestRawTripPayload(
"{\"_row_key\":\"003\",\"time\":\"2016-02-31T03:16:41.415Z\",\"number\":12}");
HoodieRecord record5 = new HoodieRecord(new HoodieKey(rowChange5.getRowKey(), rowChange5.getPartitionPath()),
rowChange4);
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record5));
String filename0 =
HoodieClientTestUtils.writeParquetFile(basePath, "2016/04/01", Arrays.asList(record1), schema, null, false);
String filename1 =
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Lists.newArrayList(), schema, null, false);
String filename2 =
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Arrays.asList(record2), schema, null, false);
String filename3 =
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Arrays.asList(record4), schema, null, false);
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
// Add some commits
new File(basePath + "/.hoodie").mkdirs();
// partitions will NOT be respected by this loadInvolvedFiles(...) call
JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, jsc, table);
for (HoodieRecord record : taggedRecordRDD.collect()) {
if (record.getRecordKey().equals("000")) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename0)));
} else if (record.getRecordKey().equals("001")) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename2)));
} else if (record.getRecordKey().equals("002")) {
assertTrue(!record.isCurrentLocationKnown());
} else if (record.getRecordKey().equals("004")) {
assertTrue(record.getCurrentLocation().getFileId().equals(FSUtils.getFileId(filename3)));
}
}
}
// convert list to map to avoid sorting order dependencies
private Map<String, BloomIndexFileInfo> toFileMap(List<Tuple2<String, BloomIndexFileInfo>> filesList) {
Map<String, BloomIndexFileInfo> filesMap = new HashMap<>();
for (Tuple2<String, BloomIndexFileInfo> t : filesList) {
filesMap.put(t._1() + "/" + t._2().getFileId(), t._2());
}
return filesMap;
}
}

View File

@@ -0,0 +1,176 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.index.bloom;
import static junit.framework.TestCase.assertEquals;
import static junit.framework.TestCase.assertTrue;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Random;
import java.util.UUID;
import org.junit.Test;
/**
* Tests {@link KeyRangeLookupTree}
*/
public class TestKeyRangeLookupTree {
private static final Random RANDOM = new Random();
private KeyRangeLookupTree keyRangeLookupTree;
private Map<String, HashSet<String>> expectedMatches;
public TestKeyRangeLookupTree() {
keyRangeLookupTree = new KeyRangeLookupTree();
expectedMatches = new HashMap<>();
}
/**
* Tests for single node in the tree for different inputs.
*/
@Test
public void testFileGroupLookUpOneEntry() {
KeyRangeNode toInsert = new KeyRangeNode(Long.toString(300), Long.toString(450), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
testRangeOfInputs(290, 305);
testRangeOfInputs(390, 400);
testRangeOfInputs(445, 455);
testRangeOfInputs(600, 605);
}
/**
* Tests for many entries in the tree with same start value and different end values
*/
@Test
public void testFileGroupLookUpManyEntriesWithSameStartValue() {
String startKey = Long.toString(120);
long endKey = 250;
KeyRangeNode toInsert = new KeyRangeNode(startKey, Long.toString(endKey), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
for (int i = 0; i < 10; i++) {
endKey += 1 + RANDOM.nextInt(100);
toInsert = new KeyRangeNode(startKey, Long.toString(endKey), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
}
testRangeOfInputs(110, endKey + 5);
}
/**
* Tests for many duplicte entries in the tree
*/
@Test
public void testFileGroupLookUpManyDulicateEntries() {
KeyRangeNode toInsert = new KeyRangeNode(Long.toString(1200), Long.toString(2000), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
for (int i = 0; i < 10; i++) {
toInsert = new KeyRangeNode(Long.toString(1200), Long.toString(2000), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
}
testRangeOfInputs(1050, 1100);
testRangeOfInputs(1500, 1600);
testRangeOfInputs(1990, 2100);
}
// Tests helpers
/**
* Tests for curated entries in look up tree.
*/
@Test
public void testFileGroupLookUp() {
// testing with hand curated inputs
KeyRangeNode toInsert = new KeyRangeNode(Long.toString(500), Long.toString(600), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
toInsert = new KeyRangeNode(Long.toString(750), Long.toString(950), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
toInsert = new KeyRangeNode(Long.toString(120), Long.toString(620), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
toInsert = new KeyRangeNode(Long.toString(550), Long.toString(775), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
toInsert = new KeyRangeNode(Long.toString(725), Long.toString(850), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
toInsert = new KeyRangeNode(Long.toString(750), Long.toString(825), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
toInsert = new KeyRangeNode(Long.toString(750), Long.toString(990), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
toInsert = new KeyRangeNode(Long.toString(800), Long.toString(820), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
toInsert = new KeyRangeNode(Long.toString(200), Long.toString(550), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
toInsert = new KeyRangeNode(Long.toString(520), Long.toString(600), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
toInsert = new KeyRangeNode(Long.toString(120), Long.toString(620), UUID.randomUUID().toString());
updateExpectedMatchesToTest(toInsert);
keyRangeLookupTree.insert(toInsert);
testRangeOfInputs(110, 999);
}
/**
* Method to test the look up tree for different range of input keys.
*
* @param start starting value of the look up key
* @param end end value of the look up tree
*/
private void testRangeOfInputs(long start, long end) {
for (long i = start; i <= end; i++) {
String iStr = Long.toString(i);
if (!expectedMatches.containsKey(iStr)) {
assertEquals(Collections.EMPTY_SET, keyRangeLookupTree.getMatchingIndexFiles(iStr));
} else {
assertTrue(expectedMatches.get(iStr).equals(keyRangeLookupTree.getMatchingIndexFiles(iStr)));
}
}
}
/**
* Updates the expected matches for a given {@link KeyRangeNode}
*
* @param toInsert the {@link KeyRangeNode} to be inserted
*/
private void updateExpectedMatchesToTest(KeyRangeNode toInsert) {
long startKey = Long.parseLong(toInsert.getMinRecordKey());
long endKey = Long.parseLong(toInsert.getMaxRecordKey());
for (long i = startKey; i <= endKey; i++) {
String iStr = Long.toString(i);
if (!expectedMatches.containsKey(iStr)) {
expectedMatches.put(iStr, new HashSet<>());
}
expectedMatches.get(iStr).add(toInsert.getFileNameList().get(0));
}
}
}

View File

@@ -0,0 +1,458 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.io;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import com.google.common.collect.Sets;
import java.io.File;
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.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.minicluster.HdfsTestService;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
public class TestHoodieCommitArchiveLog {
//NOTE : Be careful in using DFS (FileSystem.class) vs LocalFs(RawLocalFileSystem.class)
//The implementation and gurantees of many API's differ, for example check rename(src,dst)
// We need to use DFS here instead of LocalFs since the FsDataInputStream.getWrappedStream() returns a
// FsDataInputStream instead of a InputStream and thus throws java.lang.ClassCastException:
// org.apache.hadoop.fs.FSDataInputStream cannot be cast to org.apache.hadoop.fs.FSInputStream
private static MiniDFSCluster dfsCluster;
private static DistributedFileSystem dfs;
private static HdfsTestService hdfsTestService;
private String basePath;
private Configuration hadoopConf;
private JavaSparkContext jsc = null;
@AfterClass
public static void cleanUp() throws Exception {
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the
// same JVM
FileSystem.closeAll();
if (hdfsTestService != null) {
hdfsTestService.stop();
dfsCluster.shutdown();
}
}
@BeforeClass
public static void setUpDFS() throws IOException {
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the
// same JVM
FileSystem.closeAll();
if (hdfsTestService == null) {
hdfsTestService = new HdfsTestService();
dfsCluster = hdfsTestService.start(true);
// Create a temp folder as the base path
dfs = dfsCluster.getFileSystem();
}
}
@Before
public void init() throws Exception {
TemporaryFolder folder = new TemporaryFolder();
folder.create();
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieCommitArchiveLog"));
basePath = folder.getRoot().getAbsolutePath();
hadoopConf = dfs.getConf();
jsc.hadoopConfiguration().addResource(dfs.getConf());
dfs.mkdirs(new Path(basePath));
HoodieTestUtils.init(hadoopConf, basePath);
}
@After
public void clean() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
@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,
new HoodieTableMetaClient(dfs.getConf(), cfg.getBasePath(), true));
boolean result = archiveLog.archiveIfRequired(jsc);
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().retainCommits(1).archiveCommitsWith(2, 4).build())
.forTable("test-trip-table").build();
HoodieTestUtils.init(hadoopConf, basePath);
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "104"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "104", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "105"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "105"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath);
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
HoodieTestUtils.createCleanFiles(basePath, "100", dfs.getConf());
HoodieTestUtils.createInflightCleanFiles(basePath, dfs.getConf(), "101");
HoodieTestUtils.createCleanFiles(basePath, "101", dfs.getConf());
HoodieTestUtils.createCleanFiles(basePath, "102", dfs.getConf());
HoodieTestUtils.createCleanFiles(basePath, "103", dfs.getConf());
HoodieTestUtils.createCleanFiles(basePath, "104", dfs.getConf());
HoodieTestUtils.createCleanFiles(basePath, "105", dfs.getConf());
HoodieTestUtils.createInflightCleanFiles(basePath, dfs.getConf(), "106", "107");
//reload the timeline and get all the commmits before archive
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
List<HoodieInstant> originalCommits = timeline.getInstants().collect(Collectors.toList());
assertEquals("Loaded 6 commits and the count should match", 12, timeline.countInstants());
// verify in-flight instants before archive
verifyInflightInstants(metaClient, 3);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg,
new HoodieTableMetaClient(dfs.getConf(), basePath, true));
assertTrue(archiveLog.archiveIfRequired(jsc));
//reload the timeline and remove the remaining commits
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
// Check compaction instants
List<HoodieInstant> instants =
HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
new Path(metaClient.getMetaAuxiliaryPath()),
HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
assertEquals("Should delete all compaction instants < 104", 4, instants.size());
assertFalse("Requested Compaction must be absent for 100", instants.contains(
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100")));
assertFalse("Inflight Compaction must be absent for 100", instants.contains(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100")));
assertFalse("Requested Compaction must be absent for 101", instants.contains(
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101")));
assertFalse("Inflight Compaction must be absent for 101", instants.contains(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101")));
assertFalse("Requested Compaction must be absent for 102", instants.contains(
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102")));
assertFalse("Inflight Compaction must be absent for 102", instants.contains(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102")));
assertFalse("Requested Compaction must be absent for 103", instants.contains(
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103")));
assertFalse("Inflight Compaction must be absent for 103", instants.contains(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103")));
assertTrue("Requested Compaction must be present for 104", instants.contains(
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104")));
assertTrue("Inflight Compaction must be present for 104", instants.contains(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "104")));
assertTrue("Requested Compaction must be present for 105", instants.contains(
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "105")));
assertTrue("Inflight Compaction must be present for 105", instants.contains(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "105")));
//read the file
Reader reader = HoodieLogFormat.newReader(dfs,
new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1_1-0-1")),
HoodieArchivedMetaEntry.getClassSchema());
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);
//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);
// verify in-flight instants after archive
verifyInflightInstants(metaClient, 3);
reader.close();
}
@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().retainCommits(1).archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf());
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103"), dfs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf());
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
assertEquals("Loaded 4 commits and the count should match", 4, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired(jsc);
assertTrue(result);
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants());
List<HoodieInstant> instants =
HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
new Path(metaClient.getMetaAuxiliaryPath()),
HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
assertEquals("Should not delete any aux compaction files when maxCommitsToKeep is 5", 8, instants.size());
assertTrue("Requested Compaction must be present for 100", instants.contains(
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100")));
assertTrue("Inflight Compaction must be present for 100", instants.contains(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100")));
assertTrue("Requested Compaction must be present for 101", instants.contains(
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101")));
assertTrue("Inflight Compaction must be present for 101", instants.contains(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101")));
assertTrue("Requested Compaction must be present for 102", instants.contains(
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102")));
assertTrue("Inflight Compaction must be present for 102", instants.contains(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102")));
assertTrue("Requested Compaction must be present for 103", instants.contains(
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103")));
assertTrue("Inflight Compaction must be present for 103", instants.contains(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103")));
}
@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().retainCommits(1).archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "104", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf());
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired(jsc);
assertTrue(result);
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().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().retainCommits(1).archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf());
HoodieTestDataGenerator.createSavepointFile(basePath, "101", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "104", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf());
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired(jsc);
assertTrue(result);
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().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")));
}
@Test
public void testArchiveCommitCompactionNoHole() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.forTable("test-trip-table").withCompactionConfig(
HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build()).build();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(dfs.getConf(), basePath);
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf());
HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "101", dfs.getConf());
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf());
HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "104", dfs.getConf());
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104"), dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "106", dfs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "107", dfs.getConf());
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline();
assertEquals("Loaded 6 commits and the count should match", 8, timeline.countInstants());
boolean result = archiveLog.archiveIfRequired(jsc);
assertTrue(result);
timeline = metaClient.getActiveTimeline().reload().getCommitsAndCompactionTimeline();
assertFalse("Instants before oldest pending compaction can be removed",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "100")));
assertEquals(
"Since we have a pending compaction at 101, we should never archive any commit "
+ "after 101 (we only " + "archive 100)", 7, timeline.countInstants());
assertTrue("Requested Compaction must still be present",
timeline.containsInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "105")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "106")));
assertTrue("Instants greater than oldest pending compaction must be present",
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "107")));
}
private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expectedTotalInstants) {
HoodieTimeline timeline = metaClient.getActiveTimeline().reload()
.getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION)).filterInflights();
assertEquals("Loaded inflight clean actions and the count should match", expectedTotalInstants,
timeline.countInstants());
}
}

View File

@@ -0,0 +1,213 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.io;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.HoodieWriteClient;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieMemoryConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.index.bloom.HoodieBloomIndex;
import org.apache.hudi.io.compact.HoodieCompactor;
import org.apache.hudi.io.compact.HoodieRealtimeTableCompactor;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
public class TestHoodieCompactor {
private transient JavaSparkContext jsc = null;
private String basePath = null;
private HoodieCompactor compactor;
private transient HoodieTestDataGenerator dataGen = null;
private transient FileSystem fs;
private Configuration hadoopConf;
private HoodieWriteClient writeClient;
@Before
public void init() throws IOException {
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieCompactor"));
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
fs = FSUtils.getFs(basePath, hadoopConf);
HoodieTestUtils.initTableType(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
dataGen = new HoodieTestDataGenerator();
compactor = new HoodieRealtimeTableCompactor();
}
@After
public void clean() {
if (null != writeClient) {
writeClient.close();
writeClient = null;
}
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
private HoodieWriteClient getWriteClient(HoodieWriteConfig config) throws Exception {
if (null != writeClient) {
writeClient.close();
}
writeClient = new HoodieWriteClient(jsc, config);
return writeClient;
}
private HoodieWriteConfig getConfig() {
return getConfigBuilder()
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(1).build())
.build();
}
private HoodieWriteConfig.Builder getConfigBuilder() {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).withCompactionConfig(
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).withInlineCompaction(false)
.build()).withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
.withMemoryConfig(HoodieMemoryConfig.newBuilder().withMaxDFSStreamBufferSize(1 * 1024 * 1024).build())
.forTable("test-trip-table")
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build());
}
@Test(expected = HoodieNotSupportedException.class)
public void testCompactionOnCopyOnWriteFail() throws Exception {
HoodieTestUtils.initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
}
@Test
public void testCompactionEmpty() throws Exception {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieWriteConfig config = getConfig();
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
HoodieWriteClient writeClient = getWriteClient(config);
String newCommitTime = writeClient.startCommit();
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
writeClient.insert(recordsRDD, newCommitTime).collect();
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
JavaRDD<WriteStatus> result =
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
assertTrue("If there is nothing to compact, result will be empty", result.isEmpty());
}
@Test
public void testWriteStatusContentsAfterCompaction() throws Exception {
// insert 100 records
HoodieWriteConfig config = getConfig();
HoodieWriteClient writeClient = getWriteClient(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(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
newCommitTime = "101";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
HoodieIndex index = new HoodieBloomIndex<>(config);
updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, table).collect();
// Write them to corresponding avro logfiles
HoodieTestUtils
.writeRecordsToLogFiles(fs, metaClient.getBasePath(), HoodieTestDataGenerator.avroSchemaWithMetadataFields,
updatedRecords);
// Verify that all data file has one log file
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
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(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
JavaRDD<WriteStatus> result =
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
// Verify that all partition paths are present in the WriteStatus result
for (String partitionPath : dataGen.getPartitionPaths()) {
List<WriteStatus> writeStatuses = result.collect();
assertTrue(writeStatuses.stream()
.filter(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath))
.count() > 0);
}
}
// 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
}

View File

@@ -0,0 +1,371 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.io;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotEquals;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.HoodieWriteClient;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.table.HoodieTable;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SQLContext;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
@SuppressWarnings("unchecked")
public class TestHoodieMergeHandle {
protected transient JavaSparkContext jsc = null;
protected transient SQLContext sqlContext;
protected transient FileSystem fs;
protected String basePath = null;
protected transient HoodieTestDataGenerator dataGen = null;
private HoodieWriteClient writeClient;
@Before
public void init() throws IOException {
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeHandle"));
//SQLContext stuff
sqlContext = new SQLContext(jsc);
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
dataGen = new HoodieTestDataGenerator();
}
@After
public void clean() {
if (null != writeClient) {
writeClient.close();
writeClient = null;
}
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
private HoodieWriteClient getWriteClient(HoodieWriteConfig config) throws Exception {
if (null != writeClient) {
writeClient.close();
}
writeClient = new HoodieWriteClient(jsc, config);
return writeClient;
}
@Test
public void testUpsertsForMultipleRecordsInSameFile() throws Exception {
// Create records in a single partition
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
dataGen = new HoodieTestDataGenerator(new String[]{partitionPath});
// Build a write config with bulkinsertparallelism set
HoodieWriteConfig cfg = getConfigBuilder().build();
HoodieWriteClient client = getWriteClient(cfg);
FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
/**
* Write 1 (only inserts)
* This will do a bulk insert of 44 records of which there are 2 records repeated 21 times each.
* id1 (21 records), id2 (21 records), id3, id4
*/
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 4);
HoodieRecord record1 = records.get(0);
HoodieRecord record2 = records.get(1);
for (int i = 0; i < 20; i++) {
HoodieRecord dup = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime);
records.add(dup);
}
for (int i = 0; i < 20; i++) {
HoodieRecord dup = dataGen.generateUpdateRecord(record2.getKey(), newCommitTime);
records.add(dup);
}
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
List<WriteStatus> statuses = client.bulkInsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
// verify that there is a commit
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
Assert.assertEquals("Latest commit should be 001", newCommitTime, timeline.lastInstant().get().getTimestamp());
assertEquals("Must contain 44 records",
records.size(),
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
/**
* Write 2 (insert)
* This will do a bulk insert of 1 record with the same row_key as record1 in the previous insert - id1.
* At this point, we will have 2 files with the row_keys as shown here -
* File 1 - id1 (21 records), id2 (21 records), id3, id4
* File 2 - id1
*/
newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
// Do 1 more bulk insert with the same dup record1
List<HoodieRecord> newRecords = new ArrayList<>();
HoodieRecord sameAsRecord1 = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime);
newRecords.add(sameAsRecord1);
writeRecords = jsc.parallelize(newRecords, 1);
statuses = client.bulkInsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
// verify that there are 2 commits
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
assertEquals("Expecting two commits.", 2, timeline.findInstantsAfter("000", Integer.MAX_VALUE)
.countInstants());
Assert.assertEquals("Latest commit should be 002", newCommitTime, timeline.lastInstant().get().getTimestamp());
Dataset<Row> dataSet = getRecords();
assertEquals("Must contain 45 records", 45, dataSet.count());
/**
* Write 3 (insert)
* This will bulk insert 2 new completely new records.
* At this point, we will have 2 files with the row_keys as shown here -
* File 1 - id1 (21 records), id2 (21 records), id3, id4
* File 2 - id1
* File 3 - id5, id6
*/
newCommitTime = "003";
client.startCommitWithTime(newCommitTime);
newRecords = dataGen.generateInserts(newCommitTime, 2);
writeRecords = jsc.parallelize(newRecords, 1);
statuses = client.bulkInsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
// verify that there are now 3 commits
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
assertEquals("Expecting three commits.", 3, timeline.findInstantsAfter("000", Integer.MAX_VALUE)
.countInstants());
Assert.assertEquals("Latest commit should be 003", newCommitTime, timeline.lastInstant().get().getTimestamp());
dataSet = getRecords();
assertEquals("Must contain 47 records", 47, dataSet.count());
/**
* Write 4 (updates)
* This will generate 2 upsert records with id1 and id2. The rider and driver names in the update records
* will be rider-004 and driver-004.
* After the upsert is complete, all the records with id1 in File 1 and File 2 must be updated, all the records
* with id2 in File 2 must also be updated.
* Also, none of the other records in File 1, File 2 and File 3 must be updated.
*/
newCommitTime = "004";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> updateRecords = new ArrayList<>();
// This exists in 001 and 002 and should be updated in both
sameAsRecord1 = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime);
updateRecords.add(sameAsRecord1);
// This exists in 001 and should be updated
HoodieRecord sameAsRecord2 = dataGen.generateUpdateRecord(record2.getKey(), newCommitTime);
updateRecords.add(sameAsRecord2);
JavaRDD<HoodieRecord> updateRecordsRDD = jsc.parallelize(updateRecords, 1);
statuses = client.upsert(updateRecordsRDD, newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
// verify there are now 4 commits
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
assertEquals("Expecting four commits.", 4, timeline.findInstantsAfter("000", Integer.MAX_VALUE)
.countInstants());
Assert.assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(), newCommitTime);
// Check the entire dataset has 47 records still
dataSet = getRecords();
assertEquals("Must contain 47 records", 47, dataSet.count());
Row[] rows = (Row[]) dataSet.collect();
int record1Count = 0;
int record2Count = 0;
for (Row row : rows) {
if (row.getAs("_hoodie_record_key").equals(record1.getKey().getRecordKey())) {
record1Count++;
// assert each duplicate record is updated
assertEquals(row.getAs("rider"), "rider-004");
assertEquals(row.getAs("driver"), "driver-004");
} else if (row.getAs("_hoodie_record_key").equals(record2.getKey().getRecordKey())) {
record2Count++;
// assert each duplicate record is updated
assertEquals(row.getAs("rider"), "rider-004");
assertEquals(row.getAs("driver"), "driver-004");
} else {
assertNotEquals(row.getAs("rider"), "rider-004");
assertNotEquals(row.getAs("driver"), "rider-004");
}
}
// Assert that id1 record count which has been updated to rider-004 and driver-004 is 22, which is the total
// number of records with row_key id1
assertEquals(22, record1Count);
// Assert that id2 record count which has been updated to rider-004 and driver-004 is 21, which is the total
// number of records with row_key id2
assertEquals(21, record2Count);
}
@Test
public void testHoodieMergeHandleWriteStatMetrics() throws Exception {
// insert 100 records
HoodieWriteConfig config = getConfigBuilder().build();
HoodieWriteClient writeClient = getWriteClient(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();
// All records should be inserts into new parquet
Assert.assertTrue(statuses.stream()
.filter(status -> status.getStat().getPrevCommit() != HoodieWriteStat.NULL_COMMIT).count() > 0);
// Num writes should be equal to the number of records inserted
Assert.assertEquals((long) statuses.stream()
.map(status -> status.getStat().getNumWrites()).reduce((a,b) -> a + b).get(), 100);
// Num update writes should be equal to the number of records updated
Assert.assertEquals((long) statuses.stream()
.map(status -> status.getStat().getNumUpdateWrites()).reduce((a,b) -> a + b).get(), 0);
// Num update writes should be equal to the number of insert records converted to updates as part of small file
// handling
Assert.assertEquals((long) statuses.stream()
.map(status -> status.getStat().getNumInserts()).reduce((a,b) -> a + b).get(), 100);
// Update all the 100 records
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
newCommitTime = "101";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
statuses = writeClient.upsert(updatedRecordsRDD, newCommitTime).collect();
// All records should be upserts into existing parquet
Assert.assertEquals(statuses.stream()
.filter(status -> status.getStat().getPrevCommit() == HoodieWriteStat.NULL_COMMIT).count(), 0);
// Num writes should be equal to the number of records inserted
Assert.assertEquals((long) statuses.stream()
.map(status -> status.getStat().getNumWrites()).reduce((a,b) -> a + b).get(), 100);
// Num update writes should be equal to the number of records updated
Assert.assertEquals((long) statuses.stream()
.map(status -> status.getStat().getNumUpdateWrites()).reduce((a,b) -> a + b).get(), 100);
// Num update writes should be equal to the number of insert records converted to updates as part of small file
// handling
Assert.assertEquals((long) statuses.stream()
.map(status -> status.getStat().getNumInserts()).reduce((a,b) -> a + b).get(), 0);
newCommitTime = "102";
writeClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> allRecords = dataGen.generateInserts(newCommitTime, 100);
allRecords.addAll(updatedRecords);
JavaRDD<HoodieRecord> allRecordsRDD = jsc.parallelize(allRecords, 1);
statuses = writeClient.upsert(allRecordsRDD, newCommitTime).collect();
// All records should be upserts into existing parquet (with inserts as updates small file handled)
Assert.assertEquals((long) statuses.stream()
.filter(status -> status.getStat().getPrevCommit() == HoodieWriteStat.NULL_COMMIT).count(), 0);
// Num writes should be equal to the total number of records written
Assert.assertEquals((long) statuses.stream()
.map(status -> status.getStat().getNumWrites()).reduce((a,b) -> a + b).get(), 200);
// Num update writes should be equal to the number of records updated (including inserts converted as updates)
Assert.assertEquals((long) statuses.stream()
.map(status -> status.getStat().getNumUpdateWrites()).reduce((a,b) -> a + b).get(), 100);
// Num update writes should be equal to the number of insert records converted to updates as part of small file
// handling
Assert.assertEquals((long) statuses.stream()
.map(status -> status.getStat().getNumInserts()).reduce((a,b) -> a + b).get(), 100);
}
private Dataset<Row> getRecords() {
// Check the entire dataset has 8 records still
String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length];
for (int i = 0; i < fullPartitionPaths.length; i++) {
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
}
Dataset<Row> dataSet = HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs,
fullPartitionPaths);
return dataSet;
}
/**
* Assert no failures in writing hoodie files
*
* @param statuses List of Write Status
*/
void assertNoWriteErrors(List<WriteStatus> statuses) {
// Verify there are no errors
for (WriteStatus status : statuses) {
assertFalse("Errors found in write of " + status.getFileId(), status.hasErrors());
}
}
HoodieWriteConfig.Builder getConfigBuilder() {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
.forTable("test-trip-table")
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withBulkInsertParallelism(2);
}
}

View File

@@ -0,0 +1,266 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.io.strategy;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import com.beust.jcommander.internal.Lists;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import java.text.SimpleDateFormat;
import java.util.ArrayList;
import java.util.Date;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.stream.Collectors;
import org.apache.commons.lang3.time.DateUtils;
import org.apache.hudi.avro.model.HoodieCompactionOperation;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.io.compact.strategy.BoundedIOCompactionStrategy;
import org.apache.hudi.io.compact.strategy.BoundedPartitionAwareCompactionStrategy;
import org.apache.hudi.io.compact.strategy.DayBasedCompactionStrategy;
import org.apache.hudi.io.compact.strategy.LogFileSizeBasedCompactionStrategy;
import org.apache.hudi.io.compact.strategy.UnBoundedCompactionStrategy;
import org.apache.hudi.io.compact.strategy.UnBoundedPartitionAwareCompactionStrategy;
import org.junit.Assert;
import org.junit.Test;
public class TestHoodieCompactionStrategy {
private static final long MB = 1024 * 1024L;
private String[] partitionPaths = {"2017/01/01", "2017/01/02", "2017/01/03"};
@Test
public void testUnBounded() {
Map<Long, List<Long>> sizesMap = Maps.newHashMap();
sizesMap.put(120 * MB, Lists.newArrayList(60 * MB, 10 * MB, 80 * MB));
sizesMap.put(110 * MB, Lists.newArrayList());
sizesMap.put(100 * MB, Lists.newArrayList(MB));
sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB));
UnBoundedCompactionStrategy strategy = new UnBoundedCompactionStrategy();
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).build()).build();
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
assertEquals("UnBounded should not re-order or filter", operations, returned);
}
@Test
public void testBoundedIOSimple() {
Map<Long, List<Long>> sizesMap = Maps.newHashMap();
sizesMap.put(120 * MB, Lists.newArrayList(60 * MB, 10 * MB, 80 * MB));
sizesMap.put(110 * MB, Lists.newArrayList());
sizesMap.put(100 * MB, Lists.newArrayList(MB));
sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB));
BoundedIOCompactionStrategy strategy = new BoundedIOCompactionStrategy();
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build())
.build();
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
assertTrue("BoundedIOCompaction should have resulted in fewer compactions", returned.size() < operations.size());
assertEquals("BoundedIOCompaction should have resulted in 2 compactions being chosen", 2, returned.size());
// Total size of all the log files
Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB))
.map(s -> s.longValue()).reduce((size1, size2) -> size1 + size2).orElse(0L);
assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", 610,
(long) returnedSize);
}
@Test
public void testLogFileSizeCompactionSimple() {
Map<Long, List<Long>> sizesMap = Maps.newHashMap();
sizesMap.put(120 * MB, Lists.newArrayList(60 * MB, 10 * MB, 80 * MB));
sizesMap.put(110 * MB, Lists.newArrayList());
sizesMap.put(100 * MB, Lists.newArrayList(MB));
sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB));
LogFileSizeBasedCompactionStrategy strategy = new LogFileSizeBasedCompactionStrategy();
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build())
.build();
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
assertTrue("LogFileSizeBasedCompactionStrategy should have resulted in fewer compactions",
returned.size() < operations.size());
assertEquals("LogFileSizeBasedCompactionStrategy should have resulted in 1 compaction", 1, returned.size());
// Total size of all the log files
Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB))
.map(s -> s.longValue()).reduce((size1, size2) -> size1 + size2).orElse(0L);
assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", 1204,
(long) returnedSize);
}
@Test
public void testDayBasedCompactionSimple() {
Map<Long, List<Long>> sizesMap = Maps.newHashMap();
sizesMap.put(120 * MB, Lists.newArrayList(60 * MB, 10 * MB, 80 * MB));
sizesMap.put(110 * MB, Lists.newArrayList());
sizesMap.put(100 * MB, Lists.newArrayList(MB));
sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB));
Map<Long, String> keyToPartitionMap = new ImmutableMap.Builder().put(120 * MB, partitionPaths[2])
.put(110 * MB, partitionPaths[2])
.put(100 * MB, partitionPaths[1])
.put(90 * MB, partitionPaths[0])
.build();
DayBasedCompactionStrategy strategy = new DayBasedCompactionStrategy();
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy)
.withTargetPartitionsPerDayBasedCompaction(1)
.build()).build();
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap, keyToPartitionMap);
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
assertTrue("DayBasedCompactionStrategy should have resulted in fewer compactions",
returned.size() < operations.size());
Assert.assertEquals("DayBasedCompactionStrategy should have resulted in fewer compactions",
returned.size(), 2);
int comparision = strategy.getComparator().compare(returned.get(returned.size() - 1).getPartitionPath(), returned
.get(0).getPartitionPath());
// Either the partition paths are sorted in descending order or they are equal
assertTrue("DayBasedCompactionStrategy should sort partitions in descending order", comparision >= 0);
}
@Test
public void testBoundedPartitionAwareCompactionSimple() {
Map<Long, List<Long>> sizesMap = Maps.newHashMap();
sizesMap.put(120 * MB, Lists.newArrayList(60 * MB, 10 * MB, 80 * MB));
sizesMap.put(110 * MB, Lists.newArrayList());
sizesMap.put(100 * MB, Lists.newArrayList(MB));
sizesMap.put(70 * MB, Lists.newArrayList(MB));
sizesMap.put(80 * MB, Lists.newArrayList(MB));
sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB));
SimpleDateFormat format = new SimpleDateFormat("yyyy/MM/dd");
Date today = new Date();
String currentDay = format.format(today);
String currentDayMinus1 = format.format(DateUtils.addDays(today, -1));
String currentDayMinus2 = format.format(DateUtils.addDays(today, -2));
String currentDayMinus3 = format.format(DateUtils.addDays(today, -3));
String currentDayPlus1 = format.format(DateUtils.addDays(today, 1));
String currentDayPlus5 = format.format(DateUtils.addDays(today, 5));
Map<Long, String> keyToPartitionMap = new ImmutableMap.Builder()
.put(120 * MB, currentDay)
.put(110 * MB, currentDayMinus1)
.put(100 * MB, currentDayMinus2)
.put(80 * MB, currentDayMinus3)
.put(90 * MB, currentDayPlus1)
.put(70 * MB, currentDayPlus5)
.build();
BoundedPartitionAwareCompactionStrategy strategy = new BoundedPartitionAwareCompactionStrategy();
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy)
.withTargetPartitionsPerDayBasedCompaction(2)
.build()).build();
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap, keyToPartitionMap);
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
assertTrue("BoundedPartitionAwareCompactionStrategy should have resulted in fewer compactions",
returned.size() < operations.size());
Assert.assertEquals("BoundedPartitionAwareCompactionStrategy should have resulted in fewer compactions",
returned.size(), 5);
int comparision = strategy.getComparator().compare(returned.get(returned.size() - 1).getPartitionPath(), returned
.get(0).getPartitionPath());
// Either the partition paths are sorted in descending order or they are equal
assertTrue("BoundedPartitionAwareCompactionStrategy should sort partitions in descending order", comparision >= 0);
}
@Test
public void testUnboundedPartitionAwareCompactionSimple() {
Map<Long, List<Long>> sizesMap = Maps.newHashMap();
sizesMap.put(120 * MB, Lists.newArrayList(60 * MB, 10 * MB, 80 * MB));
sizesMap.put(110 * MB, Lists.newArrayList());
sizesMap.put(100 * MB, Lists.newArrayList(MB));
sizesMap.put(80 * MB, Lists.newArrayList(MB));
sizesMap.put(70 * MB, Lists.newArrayList(MB));
sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB));
SimpleDateFormat format = new SimpleDateFormat("yyyy/MM/dd");
Date today = new Date();
String currentDay = format.format(today);
String currentDayMinus1 = format.format(DateUtils.addDays(today, -1));
String currentDayMinus2 = format.format(DateUtils.addDays(today, -2));
String currentDayMinus3 = format.format(DateUtils.addDays(today, -3));
String currentDayPlus1 = format.format(DateUtils.addDays(today, 1));
String currentDayPlus5 = format.format(DateUtils.addDays(today, 5));
Map<Long, String> keyToPartitionMap = new ImmutableMap.Builder()
.put(120 * MB, currentDay)
.put(110 * MB, currentDayMinus1)
.put(100 * MB, currentDayMinus2)
.put(80 * MB, currentDayMinus3)
.put(90 * MB, currentDayPlus1)
.put(70 * MB, currentDayPlus5)
.build();
UnBoundedPartitionAwareCompactionStrategy strategy = new UnBoundedPartitionAwareCompactionStrategy();
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy)
.withTargetPartitionsPerDayBasedCompaction(2)
.build()).build();
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap, keyToPartitionMap);
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
assertTrue("UnBoundedPartitionAwareCompactionStrategy should not include last " + writeConfig
.getTargetPartitionsPerDayBasedCompaction() + " partitions or later partitions from today",
returned.size() < operations.size());
Assert.assertEquals("BoundedPartitionAwareCompactionStrategy should have resulted in 1 compaction",
returned.size(), 1);
}
private List<HoodieCompactionOperation> createCompactionOperations(HoodieWriteConfig config,
Map<Long, List<Long>> sizesMap) {
Map<Long, String> keyToPartitionMap = sizesMap.entrySet().stream().map(e ->
Pair.of(e.getKey(), partitionPaths[new Random().nextInt(partitionPaths.length - 1)]))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
return createCompactionOperations(config, sizesMap, keyToPartitionMap);
}
private List<HoodieCompactionOperation> createCompactionOperations(HoodieWriteConfig config,
Map<Long, List<Long>> sizesMap, Map<Long, String> keyToPartitionMap) {
List<HoodieCompactionOperation> operations = Lists.newArrayList(sizesMap.size());
sizesMap.forEach((k, v) -> {
HoodieDataFile df = TestHoodieDataFile.newDataFile(k);
String partitionPath = keyToPartitionMap.get(k);
List<HoodieLogFile> logFiles = v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList());
operations.add(new HoodieCompactionOperation(df.getCommitTime(),
logFiles.stream().map(s -> s.getPath().toString()).collect(Collectors.toList()),
df.getPath(),
df.getFileId(),
partitionPath,
config.getCompactionStrategy().captureMetrics(config, Option.of(df), partitionPath, logFiles)));
});
return operations;
}
}

View File

@@ -0,0 +1,56 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.io.strategy;
import java.util.UUID;
import org.apache.hudi.common.model.HoodieDataFile;
public class TestHoodieDataFile extends HoodieDataFile {
private final long size;
public TestHoodieDataFile(long size) {
super("/tmp/XYXYXYXYXYYX_11_20180918020003.parquet");
this.size = size;
}
public static HoodieDataFile newDataFile(long size) {
return new TestHoodieDataFile(size);
}
@Override
public String getPath() {
return "/tmp/test";
}
@Override
public String getFileId() {
return UUID.randomUUID().toString();
}
@Override
public String getCommitTime() {
return "100";
}
@Override
public long getFileSize() {
return size;
}
}

View File

@@ -0,0 +1,46 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.io.strategy;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.model.HoodieLogFile;
public class TestHoodieLogFile extends HoodieLogFile {
private final long size;
public TestHoodieLogFile(long size) {
super("/tmp/.ce481ee7-9e53-4a2e-9992-f9e295fa79c0_20180919184844.log.1");
this.size = size;
}
public static HoodieLogFile newLogFile(long size) {
return new TestHoodieLogFile(size);
}
@Override
public Path getPath() {
return new Path("/tmp/test-log");
}
@Override
public long getFileSize() {
return size;
}
}

View File

@@ -0,0 +1,48 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.metrics;
import static org.apache.hudi.metrics.Metrics.registerGauge;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import org.apache.commons.configuration.ConfigurationException;
import org.apache.hudi.config.HoodieWriteConfig;
import org.junit.Before;
import org.junit.Test;
public class TestHoodieMetrics {
private HoodieMetrics metrics = null;
@Before
public void start() throws ConfigurationException {
HoodieWriteConfig config = mock(HoodieWriteConfig.class);
when(config.isMetricsOn()).thenReturn(true);
when(config.getMetricsReporterType()).thenReturn(MetricsReporterType.INMEMORY);
metrics = new HoodieMetrics(config, "raw_table");
}
@Test
public void testRegisterGauge() {
registerGauge("metric1", 123L);
assertTrue(Metrics.getInstance().getRegistry().getGauges().get("metric1").getValue().toString().equals("123"));
}
}

View File

@@ -0,0 +1,511 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.table;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.File;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.UUID;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.WriteStatus;
import org.apache.hudi.common.BloomFilter;
import org.apache.hudi.common.HoodieClientTestUtils;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.common.TestRawTripPayload;
import org.apache.hudi.common.TestRawTripPayload.MetadataMergeWriteStatus;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ParquetUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.io.HoodieCreateHandle;
import org.apache.hudi.table.HoodieCopyOnWriteTable.UpsertPartitioner;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.hadoop.ParquetReader;
import org.apache.spark.TaskContext;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import scala.Tuple2;
public class TestCopyOnWriteTable {
protected static Logger log = LogManager.getLogger(TestCopyOnWriteTable.class);
private String basePath = null;
private transient JavaSparkContext jsc = null;
@Before
public void init() throws Exception {
// Initialize a local spark env
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("TestCopyOnWriteTable"));
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
this.basePath = folder.getRoot().getAbsolutePath();
HoodieTestUtils.init(jsc.hadoopConfiguration(), basePath);
}
@Test
public void testMakeNewPath() throws Exception {
String fileName = UUID.randomUUID().toString();
String partitionPath = "2016/05/04";
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieWriteConfig config = makeHoodieClientConfig();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
Pair<Path, String> newPathWithWriteToken = jsc.parallelize(Arrays.asList(1)).map(x -> {
HoodieRecord record = mock(HoodieRecord.class);
when(record.getPartitionPath()).thenReturn(partitionPath);
String writeToken = FSUtils.makeWriteToken(TaskContext.getPartitionId(), TaskContext.get().stageId(),
TaskContext.get().taskAttemptId());
HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath, fileName);
return Pair.of(io.makeNewPath(record.getPartitionPath()), writeToken);
}).collect().get(0);
Assert.assertEquals(newPathWithWriteToken.getKey().toString(),
this.basePath + "/" + partitionPath + "/"
+ FSUtils.makeDataFileName(commitTime, newPathWithWriteToken.getRight(), fileName));
}
private HoodieWriteConfig makeHoodieClientConfig() throws Exception {
return makeHoodieClientConfigBuilder().build();
}
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Exception {
// Prepare the AvroParquetIO
String schemaStr = IOUtils.toString(getClass().getResourceAsStream("/exampleSchema.txt"), "UTF-8");
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr);
}
// TODO (weiy): Add testcases for crossing file writing.
@Test
public void testUpdateRecords() throws Exception {
// Prepare the AvroParquetIO
HoodieWriteConfig config = makeHoodieClientConfig();
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
String partitionPath = "/2016/01/31";
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
// Get some records belong to the same partition (2016/01/31)
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}";
String recordStr4 = "{\"_row_key\":\"8eb5b87d-1fej-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":51}";
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));
// Insert new records
final HoodieCopyOnWriteTable cowTable = table;
jsc.parallelize(Arrays.asList(1)).map(x -> {
return cowTable.handleInsert(firstCommitTime, FSUtils.createNewFileIdPfx(), records.iterator());
}).map(x -> HoodieClientTestUtils.collectStatuses(x)).collect();
// We should have a parquet file generated (TODO: better control # files after we revise
// AvroParquetIO)
File parquetFile = null;
for (File file : new File(this.basePath + partitionPath).listFiles()) {
if (file.getName().endsWith(".parquet")) {
parquetFile = file;
break;
}
}
assertTrue(parquetFile != null);
// Read out the bloom filter and make sure filter can answer record exist or not
Path parquetFilePath = new Path(parquetFile.getAbsolutePath());
BloomFilter filter = ParquetUtils.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), parquetFilePath);
for (HoodieRecord record : records) {
assertTrue(filter.mightContain(record.getRecordKey()));
}
// Create a commit file
new File(
this.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + FSUtils.getCommitTime(parquetFile.getName())
+ ".commit").createNewFile();
// Read the parquet file, check the record content
List<GenericRecord> fileRecords = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), parquetFilePath);
GenericRecord newRecord;
int index = 0;
for (GenericRecord record : fileRecords) {
assertTrue(record.get("_row_key").toString().equals(records.get(index).getRecordKey()));
index++;
}
// We update the 1st record & add a new record
String updateRecordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
TestRawTripPayload updateRowChanges1 = new TestRawTripPayload(updateRecordStr1);
HoodieRecord updatedRecord1 = new HoodieRecord(
new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()), updateRowChanges1);
updatedRecord1.setCurrentLocation(new HoodieRecordLocation(null, FSUtils.getFileId(parquetFile.getName())));
TestRawTripPayload rowChange4 = new TestRawTripPayload(recordStr4);
HoodieRecord insertedRecord1 = new HoodieRecord(
new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
List<HoodieRecord> updatedRecords = Arrays.asList(updatedRecord1, insertedRecord1);
Thread.sleep(1000);
String newCommitTime = HoodieTestUtils.makeNewCommitTime();
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
final HoodieCopyOnWriteTable newTable = new HoodieCopyOnWriteTable(config, jsc);
List<WriteStatus> statuses =
jsc.parallelize(Arrays.asList(1)).map(x -> {
return newTable.handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(),
updatedRecords.iterator());
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
// Check the updated file
File updatedParquetFile = null;
for (File file : new File(basePath + "/2016/01/31").listFiles()) {
if (file.getName().endsWith(".parquet")) {
if (FSUtils.getFileId(file.getName()).equals(FSUtils.getFileId(parquetFile.getName()))
&& HoodieTimeline.compareTimestamps(FSUtils.getCommitTime(file.getName()),
FSUtils.getCommitTime(parquetFile.getName()), HoodieTimeline.GREATER)) {
updatedParquetFile = file;
break;
}
}
}
assertTrue(updatedParquetFile != null);
// Check whether the record has been updated
Path updatedParquetFilePath = new Path(updatedParquetFile.getAbsolutePath());
BloomFilter updatedFilter = ParquetUtils.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(),
updatedParquetFilePath);
for (HoodieRecord record : records) {
// No change to the _row_key
assertTrue(updatedFilter.mightContain(record.getRecordKey()));
}
assertTrue(updatedFilter.mightContain(insertedRecord1.getRecordKey()));
records.add(insertedRecord1);// add this so it can further check below
ParquetReader updatedReader = ParquetReader.builder(new AvroReadSupport<>(), updatedParquetFilePath).build();
index = 0;
while ((newRecord = (GenericRecord) updatedReader.read()) != null) {
assertTrue(newRecord.get("_row_key").toString().equals(records.get(index).getRecordKey()));
if (index == 0) {
assertTrue(newRecord.get("number").toString().equals("15"));
}
index++;
}
updatedReader.close();
// Also check the numRecordsWritten
WriteStatus writeStatus = statuses.get(0);
assertTrue("Should be only one file generated", statuses.size() == 1);
assertEquals(4, writeStatus.getStat().getNumWrites());//3 rewritten records + 1 new record
}
private List<HoodieRecord> newHoodieRecords(int n, String time) throws Exception {
List<HoodieRecord> records = new ArrayList<>();
for (int i = 0; i < n; i++) {
String recordStr = String.format("{\"_row_key\":\"%s\",\"time\":\"%s\",\"number\":%d}",
UUID.randomUUID().toString(), time, i);
TestRawTripPayload rowChange = new TestRawTripPayload(recordStr);
records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
}
return records;
}
// Check if record level metadata is aggregated properly at the end of write.
@Test
public void testMetadataAggregateFromWriteStatus() throws Exception {
// Prepare the AvroParquetIO
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withWriteStatusClass(MetadataMergeWriteStatus.class)
.build();
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
// Get some records belong to the same partition (2016/01/31)
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));
// Insert new records
List<WriteStatus> writeStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
return table.handleInsert(firstCommitTime, FSUtils.createNewFileIdPfx(), records.iterator());
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
Map<String, String> allWriteStatusMergedMetadataMap = MetadataMergeWriteStatus
.mergeMetadataForWriteStatuses(writeStatuses);
assertTrue(allWriteStatusMergedMetadataMap.containsKey("InputRecordCount_1506582000"));
// For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this
// should be 2 * 3
assertEquals("6", allWriteStatusMergedMetadataMap.get("InputRecordCount_1506582000"));
}
@Test
public void testInsertRecords() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
// Case 1:
// 10 records for partition 1, 1 record for partition 2.
List<HoodieRecord> records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z");
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
// Insert new records
final List<HoodieRecord> recs2 = records;
List<WriteStatus> returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
return table.handleInsert(commitTime, FSUtils.createNewFileIdPfx(), recs2.iterator());
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
// TODO: check the actual files and make sure 11 records, total were written.
assertEquals(2, returnedStatuses.size());
assertEquals("2016/01/31", returnedStatuses.get(0).getPartitionPath());
assertEquals(0, returnedStatuses.get(0).getFailedRecords().size());
assertEquals(10, returnedStatuses.get(0).getTotalRecords());
assertEquals("2016/02/01", returnedStatuses.get(1).getPartitionPath());
assertEquals(0, returnedStatuses.get(0).getFailedRecords().size());
assertEquals(1, returnedStatuses.get(1).getTotalRecords());
// Case 2:
// 1 record for partition 1, 5 record for partition 2, 1 records for partition 3.
records = newHoodieRecords(1, "2016-01-31T03:16:41.415Z");
records.addAll(newHoodieRecords(5, "2016-02-01T03:16:41.415Z"));
records.addAll(newHoodieRecords(1, "2016-02-02T03:16:41.415Z"));
// Insert new records
final List<HoodieRecord> recs3 = records;
returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
return table.handleInsert(commitTime, FSUtils.createNewFileIdPfx(), recs3.iterator());
}).flatMap(x -> HoodieClientTestUtils.collectStatuses(x).iterator()).collect();
assertEquals(3, returnedStatuses.size());
assertEquals("2016/01/31", returnedStatuses.get(0).getPartitionPath());
assertEquals(1, returnedStatuses.get(0).getTotalRecords());
assertEquals("2016/02/01", returnedStatuses.get(1).getPartitionPath());
assertEquals(5, returnedStatuses.get(1).getTotalRecords());
assertEquals("2016/02/02", returnedStatuses.get(2).getPartitionPath());
assertEquals(1, returnedStatuses.get(2).getTotalRecords());
}
@Test
public void testFileSizeUpsertRecords() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withStorageConfig(
HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024).parquetPageSize(64 * 1024)
.build()).build();
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
List<HoodieRecord> records = new ArrayList<>();
// Approx 1150 records are written for block size of 64KB
for (int i = 0; i < 2000; i++) {
String recordStr =
"{\"_row_key\":\"" + UUID.randomUUID().toString() + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i
+ "}";
TestRawTripPayload rowChange = new TestRawTripPayload(recordStr);
records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
}
// Insert new records
jsc.parallelize(Arrays.asList(1))
.map(i -> table.handleInsert(commitTime, FSUtils.createNewFileIdPfx(), records.iterator()))
.map(x -> HoodieClientTestUtils.collectStatuses(x)).collect();
// Check the updated file
int counts = 0;
for (File file : new File(basePath + "/2016/01/31").listFiles()) {
if (file.getName().endsWith(".parquet") && FSUtils.getCommitTime(file.getName()).equals(commitTime)) {
log.info(file.getName() + "-" + file.length());
counts++;
}
}
assertEquals("If the number of records are more than 1150, then there should be a new file", 3, counts);
}
private UpsertPartitioner getUpsertPartitioner(int smallFileSize, int numInserts,
int numUpdates, int fileSize, String testPartitionPath, boolean autoSplitInserts) throws Exception {
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withCompactionConfig(
HoodieCompactionConfig.newBuilder().compactionSmallFileSize(smallFileSize).insertSplitSize(100)
.autoTuneInsertSplits(autoSplitInserts).build()).withStorageConfig(
HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()).build();
HoodieClientTestUtils.fakeCommitFile(basePath, "001");
HoodieClientTestUtils.fakeDataFile(basePath, testPartitionPath, "001", "file1", fileSize);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[]{testPartitionPath});
List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", numInserts);
List<HoodieRecord> updateRecords = dataGenerator.generateUpdates("001", numUpdates);
for (HoodieRecord updateRec : updateRecords) {
updateRec.setCurrentLocation(new HoodieRecordLocation("001", "file1"));
}
List<HoodieRecord> records = new ArrayList<>();
records.addAll(insertRecords);
records.addAll(updateRecords);
WorkloadProfile profile = new WorkloadProfile(jsc.parallelize(records));
HoodieCopyOnWriteTable.UpsertPartitioner partitioner =
(HoodieCopyOnWriteTable.UpsertPartitioner) table.getUpsertPartitioner(profile);
assertEquals("Update record should have gone to the 1 update partiton", 0, partitioner.getPartition(
new Tuple2<>(updateRecords.get(0).getKey(), Option.ofNullable(updateRecords.get(0).getCurrentLocation()))));
return partitioner;
}
@Test
public void testUpsertPartitioner() throws Exception {
final String testPartitionPath = "2016/09/26";
// Inserts + Updates... Check all updates go together & inserts subsplit
UpsertPartitioner partitioner = getUpsertPartitioner(0, 200, 100, 1024,
testPartitionPath, false);
List<HoodieCopyOnWriteTable.InsertBucket> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
assertEquals("Total of 2 insert buckets", 2, insertBuckets.size());
}
@Test
public void testUpsertPartitionerWithSmallInsertHandling() throws Exception {
final String testPartitionPath = "2016/09/26";
// Inserts + Updates .. Check updates go together & inserts subsplit, after expanding
// smallest file
UpsertPartitioner partitioner = getUpsertPartitioner(1000 * 1024, 400, 100, 800 * 1024,testPartitionPath,
false);
List<HoodieCopyOnWriteTable.InsertBucket> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
assertEquals("Should have 3 partitions", 3, partitioner.numPartitions());
assertEquals("Bucket 0 is UPDATE", HoodieCopyOnWriteTable.BucketType.UPDATE,
partitioner.getBucketInfo(0).bucketType);
assertEquals("Bucket 1 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(1).bucketType);
assertEquals("Bucket 2 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(2).bucketType);
assertEquals("Total of 3 insert buckets", 3, insertBuckets.size());
assertEquals("First insert bucket must be same as update bucket", 0, insertBuckets.get(0).bucketNumber);
assertEquals("First insert bucket should have weight 0.5", 0.5, insertBuckets.get(0).weight, 0.01);
// Now with insert split size auto tuned
partitioner = getUpsertPartitioner(1000 * 1024, 2400, 100, 800 * 1024, testPartitionPath, true);
insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
assertEquals("Should have 4 partitions", 4, partitioner.numPartitions());
assertEquals("Bucket 0 is UPDATE", HoodieCopyOnWriteTable.BucketType.UPDATE,
partitioner.getBucketInfo(0).bucketType);
assertEquals("Bucket 1 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(1).bucketType);
assertEquals("Bucket 2 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(2).bucketType);
assertEquals("Bucket 3 is INSERT", HoodieCopyOnWriteTable.BucketType.INSERT,
partitioner.getBucketInfo(3).bucketType);
assertEquals("Total of 4 insert buckets", 4, insertBuckets.size());
assertEquals("First insert bucket must be same as update bucket", 0, insertBuckets.get(0).bucketNumber);
assertEquals("First insert bucket should have weight 0.5", 200.0 / 2400, insertBuckets.get(0).weight, 0.01);
}
@Test
public void testInsertUpsertWithHoodieAvroPayload() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withStorageConfig(
HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()).build();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
final HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
String commitTime = "000";
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
// Perform inserts of 100 records to test CreateHandle and BufferedExecutor
final List<HoodieRecord> inserts = dataGenerator.generateInsertsWithHoodieAvroPayload(commitTime, 100);
final List<List<WriteStatus>> ws = jsc.parallelize(Arrays.asList(1)).map(x -> {
return table.handleInsert(commitTime, UUID.randomUUID().toString(), inserts.iterator());
}).map(x -> (List<WriteStatus>)HoodieClientTestUtils.collectStatuses(x)).collect();
WriteStatus writeStatus = ws.get(0).get(0);
String fileId = writeStatus.getFileId();
metadata.getFs().create(new Path(basePath + "/.hoodie/000.commit")).close();
final HoodieCopyOnWriteTable table2 = new HoodieCopyOnWriteTable(config, jsc);
final List<HoodieRecord> updates =
dataGenerator.generateUpdatesWithHoodieAvroPayload(commitTime, writeStatus.getWrittenRecords());
jsc.parallelize(Arrays.asList(1)).map(x -> {
return table2.handleUpdate("001", fileId, updates.iterator());
}).map(x -> (List<WriteStatus>)HoodieClientTestUtils.collectStatuses(x)).collect();
}
@After
public void cleanup() {
if (basePath != null) {
new File(basePath).delete();
}
if (jsc != null) {
jsc.stop();
}
}
}

File diff suppressed because it is too large Load Diff