1
0

Spark Stage retry handling

This commit is contained in:
Balaji Varadarajan
2019-03-08 15:05:33 -08:00
committed by vinoth chandar
parent 3fd2fd6e9d
commit 145034c5fa
53 changed files with 1664 additions and 967 deletions

View File

@@ -66,6 +66,7 @@ import java.util.function.Predicate;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
@@ -620,18 +621,17 @@ public class TestCleaner extends TestHoodieClientBase {
* Test Cleaning functionality of table.rollback() API.
*/
@Test
public void testCleanTemporaryDataFilesOnRollback() throws IOException {
public void testCleanMarkerDataFilesOnRollback() throws IOException {
HoodieTestUtils.createCommitFiles(basePath, "000");
List<String> tempFiles = createTempFiles("000", 10);
assertEquals("Some temp files are created.", 10, tempFiles.size());
assertEquals("Some temp files are created.", tempFiles.size(), getTotalTempFiles());
List<String> markerFiles = createMarkerFiles("000", 10);
assertEquals("Some marker files are created.", 10, markerFiles.size());
assertEquals("Some marker files are created.", markerFiles.size(), getTotalTempFiles());
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config,
jsc);
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withUseTempFolderCopyOnWriteForCreate(true)
.withUseTempFolderCopyOnWriteForMerge(false).build();
HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), config
.getBasePath(), true),
config, jsc);
table.rollback(jsc, "000", true);
assertEquals("All temp files are deleted.", 0, getTotalTempFiles());
}
@@ -901,10 +901,10 @@ public class TestCleaner extends TestHoodieClientBase {
* @return generated files
* @throws IOException in case of error
*/
private List<String> createTempFiles(String commitTime, int numFiles) throws IOException {
private List<String> createMarkerFiles(String commitTime, int numFiles) throws IOException {
List<String> files = new ArrayList<>();
for (int i = 0; i < numFiles; i++) {
files.add(HoodieTestUtils.createNewDataFile(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME, commitTime));
files.add(HoodieTestUtils.createNewMarkerFile(basePath, "2019/03/29", commitTime));
}
return files;
}
@@ -915,7 +915,13 @@ public class TestCleaner extends TestHoodieClientBase {
* @throws IOException in case of error
*/
private int getTotalTempFiles() throws IOException {
return fs.listStatus(new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME)).length;
RemoteIterator itr = fs.listFiles(new Path(basePath, HoodieTableMetaClient.TEMPFOLDER_NAME), true);
int count = 0;
while (itr.hasNext()) {
count++;
itr.next();
}
return count;
}
private Stream<Pair<String, String>> convertPathToFileIdWithCommitTime(

View File

@@ -0,0 +1,108 @@
/*
* Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.util.ConsistencyGuard;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.FailSafeConsistencyGuard;
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.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, 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, 3, 10, 10);
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, 3, 10, 10);
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, 3, 10, 10);
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, 3, 10, 10);
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
}
}

View File

@@ -16,6 +16,7 @@
package com.uber.hoodie;
import static com.uber.hoodie.common.table.HoodieTableMetaClient.MARKER_EXTN;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@@ -38,6 +39,7 @@ import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
@@ -244,19 +246,6 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
HoodieWriteClient::upsert, false);
}
/**
* Test Upsert API using temporary folders.
*/
@Test
public void testUpsertsWithFinalizeWrite() throws Exception {
HoodieWriteConfig hoodieWriteConfig = getConfigBuilder()
.withUseTempFolderCopyOnWriteForCreate(true)
.withUseTempFolderCopyOnWriteForMerge(true)
.build();
testUpsertsInternal(hoodieWriteConfig,
HoodieWriteClient::upsert, false);
}
/**
* Test UpsertPrepped API
*/
@@ -266,19 +255,6 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
HoodieWriteClient::upsertPreppedRecords, true);
}
/**
* Test UpsertPrepped API using temporary folders.
*/
@Test
public void testUpsertsPreppedWithFinalizeWrite() throws Exception {
HoodieWriteConfig hoodieWriteConfig = getConfigBuilder()
.withUseTempFolderCopyOnWriteForCreate(true)
.withUseTempFolderCopyOnWriteForMerge(true)
.build();
testUpsertsInternal(hoodieWriteConfig,
HoodieWriteClient::upsertPreppedRecords, true);
}
/**
* Test one of HoodieWriteClient upsert(Prepped) APIs
*
@@ -385,7 +361,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
assertEquals("Just 1 file needs to be added.", 1, statuses.size());
String file1 = statuses.get(0).getFileId();
assertEquals("file should contain 100 records", ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(),
new Path(basePath, testPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(), 100);
new Path(basePath, statuses.get(0).getStat().getPath())).size(), 100);
// Update + Inserts such that they just expand file1
String commitTime2 = "002";
@@ -403,7 +379,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
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, testPartitionPath + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1));
Path newFile = new Path(basePath, statuses.get(0).getStat().getPath());
assertEquals("file should contain 140 records",
ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140);
@@ -499,7 +475,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
assertEquals("Just 1 file needs to be added.", 1, statuses.size());
String file1 = statuses.get(0).getFileId();
assertEquals("file should contain 100 records", ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(),
new Path(basePath, testPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(), 100);
new Path(basePath, statuses.get(0).getStat().getPath())).size(), 100);
// Second, set of Inserts should just expand file1
String commitTime2 = "002";
@@ -513,7 +489,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
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, testPartitionPath + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1));
Path newFile = new Path(basePath, statuses.get(0).getStat().getPath());
assertEquals("file should contain 140 records",
ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140);
@@ -678,22 +654,59 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
*/
@Test
public void testConsistencyCheckDuringFinalize() throws Exception {
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
HoodieWriteClient client = getHoodieWriteClient(cfg);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
String commitTime = "000";
client.startCommitWithTime(commitTime);
JavaRDD<HoodieRecord> writeRecords = jsc
.parallelize(dataGen.generateInserts(commitTime, 200), 1);
JavaRDD<WriteStatus> result = client.bulkInsert(writeRecords, commitTime);
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
HoodieWriteClient client = getHoodieWriteClient(cfg);
Pair<Path, JavaRDD<WriteStatus>> result = testConsistencyCheck(metaClient, commitTime);
// move one of the files & commit should fail
WriteStatus status = result.take(1).get(0);
Path origPath = new Path(basePath + "/" + status.getStat().getPath());
Path hidePath = new Path(basePath + "/" + status.getStat().getPath() + "_hide");
metaClient.getFs().rename(origPath, hidePath);
// 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).withMaxConsistencyCheckIntervalMs(1)
.withInitialConsistencyCheckIntervalMs(1).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(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);
@@ -701,12 +714,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
} catch (HoodieCommitException cme) {
assertTrue(cme.getCause() instanceof HoodieIOException);
}
// Re-introduce & commit should succeed
metaClient.getFs().rename(hidePath, origPath);
assertTrue("Commit should succeed", client.commit(commitTime, result));
assertTrue("After explicit commit, commit file should be created",
HoodieTestUtils.doesCommitExist(basePath, commitTime));
return Pair.of(markerFilePath, result);
}
/**

View File

@@ -27,6 +27,7 @@ import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.spark.api.java.JavaRDD;
import org.junit.Assert;
import org.junit.Test;
import scala.Option;
@@ -107,7 +108,7 @@ public class TestHoodieReadClient extends TestHoodieClientBase {
filteredRDD = readClient.filterExists(recordsRDD);
List<HoodieRecord> result = filteredRDD.collect();
// Check results
assertTrue(result.size() == 25);
Assert.assertEquals(25, result.size());
}
/**

View File

@@ -105,7 +105,7 @@ public class HoodieClientTestUtils {
throws Exception {
String parentPath = String.format("%s/%s", basePath, partitionPath);
new File(parentPath).mkdirs();
String path = String.format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, 0, fileId));
String path = String.format("%s/%s", parentPath, FSUtils.makeDataFileName(commitTime, "1-0-1", fileId));
new File(path).createNewFile();
new RandomAccessFile(path, "rw").setLength(length);
}
@@ -236,7 +236,7 @@ public class HoodieClientTestUtils {
Thread.sleep(1000);
String commitTime = HoodieTestUtils.makeNewCommitTime();
String fileId = UUID.randomUUID().toString();
String filename = FSUtils.makeDataFileName(commitTime, 1, fileId);
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

@@ -25,24 +25,32 @@ import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.io.HoodieCreateHandle;
import com.uber.hoodie.io.HoodieMergeHandle;
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
import java.io.File;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Iterator;
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.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 {
public class TestUpdateMapFunction implements Serializable {
private String basePath = null;
private transient JavaSparkContext jsc = null;
@@ -71,51 +79,73 @@ public class TestUpdateMapFunction {
@Test
public void testSchemaEvolutionOnUpdate() throws Exception {
// Create a bunch of records with a old version of schema
HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(HoodieTestUtils.getDefaultHadoopConf(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
final HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
System.out.println("JSC =" + jsc);
final HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
List<HoodieRecord> records = new ArrayList<>();
TestRawTripPayload rowChange1 = new TestRawTripPayload(recordStr1);
records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
TestRawTripPayload rowChange2 = new TestRawTripPayload(recordStr2);
records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
TestRawTripPayload rowChange3 = new TestRawTripPayload(recordStr3);
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
Iterator<List<WriteStatus>> insertResult = table.handleInsert("100", records.iterator());
Path commitFile = new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100"));
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
config = makeHoodieClientConfig("/exampleEvolvedSchema.txt");
metaClient = new HoodieTableMetaClient(HoodieTestUtils.getDefaultHadoopConf(), basePath);
String fileId = insertResult.next().get(0).getFileId();
System.out.println(fileId);
final HoodieWriteConfig config2 = makeHoodieClientConfig("/exampleEvolvedSchema.txt");
final Schema schema = Schema.parse(config2.getSchema());
final WriteStatus insertResult = statuses.get(0);
String fileId = insertResult.getFileId();
table = new HoodieCopyOnWriteTable(config, jsc);
// New content with values for the newly added field
recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";
records = new ArrayList<>();
rowChange1 = new TestRawTripPayload(recordStr1);
HoodieRecord record1 = new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()),
rowChange1);
record1.setCurrentLocation(new HoodieRecordLocation("100", fileId));
records.add(record1);
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 {
table.handleUpdate("101", fileId, records.iterator());
} catch (ClassCastException e) {
fail("UpdateFunction could not read records written with exampleSchema.txt using the "
+ "exampleEvolvedSchema.txt");
}
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 {
@@ -123,5 +153,4 @@ public class TestUpdateMapFunction {
String schemaStr = IOUtils.toString(getClass().getResourceAsStream(schema), "UTF-8");
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr).build();
}
}
}

View File

@@ -1,91 +0,0 @@
/*
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.io;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.anyInt;
import static org.mockito.Mockito.anyList;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import com.uber.hoodie.common.HoodieClientTestUtils;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
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 TestConsistencyCheck {
private String basePath;
private JavaSparkContext jsc;
@Before
public void setup() throws IOException {
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest("ConsistencyCheckTest"));
TemporaryFolder testFolder = new TemporaryFolder();
testFolder.create();
basePath = testFolder.getRoot().getAbsolutePath();
}
@After
public void teardown() {
if (jsc != null) {
jsc.stop();
}
File testFolderPath = new File(basePath);
if (testFolderPath.exists()) {
testFolderPath.delete();
}
}
@Test
public void testExponentialBackoff() throws Exception {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
JavaSparkContext jscSpy = spy(jsc);
ConsistencyCheck failing = new ConsistencyCheck(basePath,
Arrays.asList("partition/path/f1_0_000.parquet", "partition/path/f2_0_000.parquet"),
jscSpy, 2);
long startMs = System.currentTimeMillis();
assertEquals(1, failing.check(5, 10).size());
assertTrue((System.currentTimeMillis() - startMs) > (10 + 20 + 40 + 80));
verify(jscSpy, times(5)).parallelize(anyList(), anyInt());
}
@Test
public void testCheckPassingAndFailing() throws Exception {
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f1");
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f2");
HoodieClientTestUtils.fakeDataFile(basePath, "partition/path", "000", "f3");
ConsistencyCheck passing = new ConsistencyCheck(basePath,
Arrays.asList("partition/path/f1_0_000.parquet", "partition/path/f2_0_000.parquet"),
jsc, 2);
assertEquals(0, passing.check(1, 1000).size());
ConsistencyCheck failing = new ConsistencyCheck(basePath,
Arrays.asList("partition/path/f1_0_000.parquet", "partition/path/f4_0_000.parquet"),
jsc, 2);
assertEquals(1, failing.check(1, 1000).size());
}
}

View File

@@ -74,13 +74,14 @@ public class TestHoodieCommitArchiveLog {
@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();
}
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the
// same JVM
FileSystem.closeAll();
}
@BeforeClass
@@ -245,7 +246,7 @@ public class TestHoodieCommitArchiveLog {
//read the file
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(dfs,
new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1")),
new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1_1-0-1")),
HoodieArchivedMetaEntry.getClassSchema());
int archivedRecordsCount = 0;

View File

@@ -17,7 +17,6 @@
package com.uber.hoodie.table;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -36,6 +35,7 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
@@ -44,18 +44,18 @@ import com.uber.hoodie.table.HoodieCopyOnWriteTable.UpsertPartitioner;
import java.io.File;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator;
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.FileSystem;
import org.apache.hadoop.fs.Path;
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;
@@ -85,20 +85,24 @@ public class TestCopyOnWriteTable {
public void testMakeNewPath() throws Exception {
String fileName = UUID.randomUUID().toString();
String partitionPath = "2016/05/04";
int unitNumber = (int) (Math.random() * 10);
HoodieRecord record = mock(HoodieRecord.class);
when(record.getPartitionPath()).thenReturn(partitionPath);
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieWriteConfig config = makeHoodieClientConfig();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath,
UUID.randomUUID().toString());
Path newPath = io.makeNewPath(record.getPartitionPath(), unitNumber, fileName);
assertTrue(newPath.toString().equals(
this.basePath + "/" + partitionPath + "/" + FSUtils.makeDataFileName(commitTime, unitNumber, fileName)));
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 {
@@ -141,7 +145,11 @@ public class TestCopyOnWriteTable {
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
// Insert new records
HoodieClientTestUtils.collectStatuses(table.handleInsert(firstCommitTime, records.iterator()));
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;
@@ -190,10 +198,12 @@ public class TestCopyOnWriteTable {
Thread.sleep(1000);
String newCommitTime = HoodieTestUtils.makeNewCommitTime();
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = new HoodieCopyOnWriteTable(config, jsc);
Iterator<List<WriteStatus>> iter = table
.handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(),
updatedRecords.iterator());
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;
@@ -231,7 +241,6 @@ public class TestCopyOnWriteTable {
}
updatedReader.close();
// Also check the numRecordsWritten
List<WriteStatus> statuses = HoodieClientTestUtils.collectStatuses(iter);
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
@@ -277,8 +286,10 @@ public class TestCopyOnWriteTable {
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
// Insert new records
List<WriteStatus> writeStatuses = HoodieClientTestUtils
.collectStatuses(table.handleInsert(firstCommitTime, records.iterator()));
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"));
@@ -287,41 +298,6 @@ public class TestCopyOnWriteTable {
assertEquals("6", allWriteStatusMergedMetadataMap.get("InputRecordCount_1506582000"));
}
@Test
public void testInsertWithPartialFailures() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.makeNewCommitTime();
FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
// Write a few records, and get atleast one file
// 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"));
// Simulate crash after first file
List<WriteStatus> statuses = HoodieClientTestUtils
.collectStatuses(table.handleInsert(commitTime, records.iterator()));
WriteStatus status = statuses.get(0);
Path partialFile = new Path(String.format("%s/%s/%s", basePath, status.getPartitionPath(),
FSUtils.makeDataFileName(commitTime, 0, status.getFileId())));
assertTrue(fs.exists(partialFile));
// When we retry
records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z");
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
statuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
status = statuses.get(0);
Path retriedFIle = new Path(String.format("%s/%s/%s", basePath, status.getPartitionPath(),
FSUtils.makeDataFileName(commitTime, 0, status.getFileId())));
assertTrue(fs.exists(retriedFIle));
assertFalse(fs.exists(partialFile));
}
@Test
public void testInsertRecords() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig();
@@ -335,8 +311,10 @@ public class TestCopyOnWriteTable {
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
// Insert new records
List<WriteStatus> returnedStatuses = HoodieClientTestUtils
.collectStatuses(table.handleInsert(commitTime, records.iterator()));
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());
@@ -354,7 +332,11 @@ public class TestCopyOnWriteTable {
records.addAll(newHoodieRecords(1, "2016-02-02T03:16:41.415Z"));
// Insert new records
returnedStatuses = HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
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());
@@ -388,7 +370,9 @@ public class TestCopyOnWriteTable {
}
// Insert new records
HoodieClientTestUtils.collectStatuses(table.handleInsert(commitTime, records.iterator()));
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;
@@ -487,19 +471,26 @@ public class TestCopyOnWriteTable {
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withStorageConfig(
HoodieStorageConfig.newBuilder().limitFileSize(1000 * 1024).build()).build();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
final HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
String commitTime = "000";
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
// Perform inserts of 100 records to test CreateHandle and BufferedExecutor
List<HoodieRecord> inserts = dataGenerator.generateInsertsWithHoodieAvroPayload(commitTime, 100);
Iterator<List<WriteStatus>> ws = table.handleInsert(commitTime, inserts.iterator());
WriteStatus writeStatus = ws.next().get(0);
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();
table = new HoodieCopyOnWriteTable(config, jsc);
// Perform update of 100 records to test MergeHandle and BufferedExecutor
table.handleUpdate("001", fileId,
dataGenerator.generateUpdatesWithHoodieAvroPayload(commitTime, writeStatus.getWrittenRecords()).iterator());
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
@@ -511,4 +502,4 @@ public class TestCopyOnWriteTable {
jsc.stop();
}
}
}
}