1
0

Reformatting code per Google Code Style all over

This commit is contained in:
Vinoth Chandar
2017-11-12 22:54:56 -08:00
committed by vinoth chandar
parent 5a62480a92
commit e45679f5e2
254 changed files with 21580 additions and 21108 deletions

View File

@@ -55,237 +55,240 @@ import org.junit.BeforeClass;
import org.junit.Test;
public class TestHDFSParquetImporter implements Serializable {
private static String dfsBasePath;
private static HdfsTestService hdfsTestService;
private static MiniDFSCluster dfsCluster;
private static DistributedFileSystem dfs;
private static String dfsBasePath;
private static HdfsTestService hdfsTestService;
private static MiniDFSCluster dfsCluster;
private static DistributedFileSystem dfs;
@BeforeClass
public static void initClass() throws Exception {
hdfsTestService = new HdfsTestService();
dfsCluster = hdfsTestService.start(true);
@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));
FSUtils.setFs(dfs);
// Create a temp folder as the base path
dfs = dfsCluster.getFileSystem();
dfsBasePath = dfs.getWorkingDirectory().toString();
dfs.mkdirs(new Path(dfsBasePath));
FSUtils.setFs(dfs);
}
@AfterClass
public static void cleanupClass() throws Exception {
if (hdfsTestService != null) {
hdfsTestService.stop();
}
FSUtils.setFs(null);
}
@AfterClass
public static void cleanupClass() throws Exception {
if (hdfsTestService != null) {
hdfsTestService.stop();
/**
* Test successful data import with retries.
*/
@Test
public void testDatasetImportWithRetries() throws Exception {
JavaSparkContext jsc = null;
try {
jsc = getJavaSparkContext();
// Test root folder.
String basePath = (new Path(dfsBasePath,
Thread.currentThread().getStackTrace()[1].getMethodName())).toString();
// Hoodie root folder
Path hoodieFolder = new Path(basePath, "testTarget");
// Create schema file.
String schemaFile = new Path(basePath, "file.schema").toString();
//Create generic records.
Path srcFolder = new Path(basePath, "testSrc");
createRecords(srcFolder);
HDFSParquetImporter.Config cfg = getHDFSParquetImporterConfig(srcFolder.toString(),
hoodieFolder.toString(),
"testTable", "COPY_ON_WRITE", "_row_key", "timestamp",
1, schemaFile);
AtomicInteger retry = new AtomicInteger(3);
AtomicInteger fileCreated = new AtomicInteger(0);
HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg) {
@Override
protected int dataImport(JavaSparkContext jsc) throws IOException {
int ret = super.dataImport(jsc);
if (retry.decrementAndGet() == 0) {
fileCreated.incrementAndGet();
createSchemaFile(schemaFile);
}
return ret;
}
FSUtils.setFs(null);
}
};
// Schema file is not created so this operation should fail.
assertEquals(0, dataImporter.dataImport(jsc, retry.get()));
assertEquals(retry.get(), -1);
assertEquals(fileCreated.get(), 1);
/**
* Test successful data import with retries.
*/
@Test
public void testDatasetImportWithRetries() throws Exception {
JavaSparkContext jsc = null;
try {
jsc = getJavaSparkContext();
// Check if
// 1. .commit file is present
// 2. number of records in each partition == 24
// 3. total number of partitions == 4;
boolean isCommitFilePresent = false;
Map<String, Long> recordCounts = new HashMap<String, Long>();
RemoteIterator<LocatedFileStatus> hoodieFiles = dfs.listFiles(hoodieFolder, true);
while (hoodieFiles.hasNext()) {
LocatedFileStatus f = hoodieFiles.next();
isCommitFilePresent =
isCommitFilePresent || f.getPath().toString().endsWith(HoodieTimeline.COMMIT_EXTENSION);
// Test root folder.
String basePath = (new Path(dfsBasePath,
Thread.currentThread().getStackTrace()[1].getMethodName())).toString();
// Hoodie root folder
Path hoodieFolder = new Path(basePath, "testTarget");
// Create schema file.
String schemaFile = new Path(basePath, "file.schema").toString();
//Create generic records.
Path srcFolder = new Path(basePath, "testSrc");
createRecords(srcFolder);
HDFSParquetImporter.Config cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(),
"testTable", "COPY_ON_WRITE", "_row_key", "timestamp",
1, schemaFile);
AtomicInteger retry = new AtomicInteger(3);
AtomicInteger fileCreated = new AtomicInteger(0);
HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg) {
@Override
protected int dataImport(JavaSparkContext jsc) throws IOException {
int ret = super.dataImport(jsc);
if (retry.decrementAndGet() == 0) {
fileCreated.incrementAndGet();
createSchemaFile(schemaFile);
}
return ret;
}
};
// Schema file is not created so this operation should fail.
assertEquals(0, dataImporter.dataImport(jsc, retry.get()));
assertEquals(retry.get(), -1);
assertEquals(fileCreated.get(), 1);
// Check if
// 1. .commit file is present
// 2. number of records in each partition == 24
// 3. total number of partitions == 4;
boolean isCommitFilePresent = false;
Map<String, Long> recordCounts = new HashMap<String, Long>();
RemoteIterator<LocatedFileStatus> hoodieFiles = dfs.listFiles(hoodieFolder, true);
while (hoodieFiles.hasNext()) {
LocatedFileStatus f = hoodieFiles.next();
isCommitFilePresent = isCommitFilePresent || f.getPath().toString().endsWith(HoodieTimeline.COMMIT_EXTENSION);
if (f.getPath().toString().endsWith("parquet")) {
SQLContext sc = new SQLContext(jsc);
String partitionPath = f.getPath().getParent().toString();
long count = sc.read().parquet(f.getPath().toString()).count();
if (!recordCounts.containsKey(partitionPath)) recordCounts.put(partitionPath, 0L);
recordCounts.put(partitionPath, recordCounts.get(partitionPath) + count);
}
}
assertTrue("commit file is missing", isCommitFilePresent);
assertEquals("partition is missing", 4, recordCounts.size());
for (Entry<String, Long> e : recordCounts.entrySet()) {
assertEquals( "missing records", 24, e.getValue().longValue());
}
} finally {
if (jsc != null) {
jsc.stop();
}
if (f.getPath().toString().endsWith("parquet")) {
SQLContext sc = new SQLContext(jsc);
String partitionPath = f.getPath().getParent().toString();
long count = sc.read().parquet(f.getPath().toString()).count();
if (!recordCounts.containsKey(partitionPath)) {
recordCounts.put(partitionPath, 0L);
}
recordCounts.put(partitionPath, recordCounts.get(partitionPath) + count);
}
}
assertTrue("commit file is missing", isCommitFilePresent);
assertEquals("partition is missing", 4, recordCounts.size());
for (Entry<String, Long> e : recordCounts.entrySet()) {
assertEquals("missing records", 24, e.getValue().longValue());
}
} finally {
if (jsc != null) {
jsc.stop();
}
}
}
private void createRecords(Path srcFolder) throws ParseException, IOException {
Path srcFile = new Path(srcFolder.toString(), "file1.parquet");
long startTime = HoodieActiveTimeline.COMMIT_FORMATTER.parse("20170203000000").getTime() / 1000;
List<GenericRecord> records = new ArrayList<GenericRecord>();
for (long recordNum = 0; recordNum < 96; recordNum++) {
records.add(HoodieTestDataGenerator
.generateGenericRecord(Long.toString(recordNum), "rider-" + recordNum,
"driver-" + recordNum, startTime + TimeUnit.HOURS.toSeconds(recordNum)));
}
ParquetWriter<GenericRecord> writer = AvroParquetWriter
.<GenericRecord>builder(srcFile)
.withSchema(HoodieTestDataGenerator.avroSchema)
.withConf(new Configuration())
.build();
for (GenericRecord record : records) {
writer.write(record);
}
writer.close();
private void createRecords(Path srcFolder) throws ParseException, IOException {
Path srcFile = new Path(srcFolder.toString(), "file1.parquet");
long startTime = HoodieActiveTimeline.COMMIT_FORMATTER.parse("20170203000000").getTime() / 1000;
List<GenericRecord> records = new ArrayList<GenericRecord>();
for (long recordNum = 0; recordNum < 96; recordNum++) {
records.add(HoodieTestDataGenerator
.generateGenericRecord(Long.toString(recordNum), "rider-" + recordNum,
"driver-" + recordNum, startTime + TimeUnit.HOURS.toSeconds(recordNum)));
}
private void createSchemaFile(String schemaFile) throws IOException {
FSDataOutputStream schemaFileOS = dfs.create(new Path(schemaFile));
schemaFileOS.write(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA.getBytes());
schemaFileOS.close();
ParquetWriter<GenericRecord> writer = AvroParquetWriter
.<GenericRecord>builder(srcFile)
.withSchema(HoodieTestDataGenerator.avroSchema)
.withConf(new Configuration())
.build();
for (GenericRecord record : records) {
writer.write(record);
}
writer.close();
}
/**
* Tests for scheme file.
* 1. File is missing.
* 2. File has invalid data.
*/
@Test
public void testSchemaFile() throws Exception {
JavaSparkContext jsc = null;
try {
jsc = getJavaSparkContext();
private void createSchemaFile(String schemaFile) throws IOException {
FSDataOutputStream schemaFileOS = dfs.create(new Path(schemaFile));
schemaFileOS.write(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA.getBytes());
schemaFileOS.close();
}
// Test root folder.
String basePath = (new Path(dfsBasePath,
Thread.currentThread().getStackTrace()[1].getMethodName())).toString();
// Hoodie root folder
Path hoodieFolder = new Path(basePath, "testTarget");
Path srcFolder = new Path(basePath.toString(), "srcTest");
Path schemaFile = new Path(basePath.toString(), "missingFile.schema");
HDFSParquetImporter.Config cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(),
"testTable", "COPY_ON_WRITE", "_row_key", "timestamp",
1, schemaFile.toString());
HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg);
// Should fail - return : -1.
assertEquals(-1, dataImporter.dataImport(jsc, 0));
/**
* Tests for scheme file. 1. File is missing. 2. File has invalid data.
*/
@Test
public void testSchemaFile() throws Exception {
JavaSparkContext jsc = null;
try {
jsc = getJavaSparkContext();
dfs.create(schemaFile).write("Random invalid schema data".getBytes());
// Should fail - return : -1.
assertEquals(-1, dataImporter.dataImport(jsc, 0));
// Test root folder.
String basePath = (new Path(dfsBasePath,
Thread.currentThread().getStackTrace()[1].getMethodName())).toString();
// Hoodie root folder
Path hoodieFolder = new Path(basePath, "testTarget");
Path srcFolder = new Path(basePath.toString(), "srcTest");
Path schemaFile = new Path(basePath.toString(), "missingFile.schema");
HDFSParquetImporter.Config cfg = getHDFSParquetImporterConfig(srcFolder.toString(),
hoodieFolder.toString(),
"testTable", "COPY_ON_WRITE", "_row_key", "timestamp",
1, schemaFile.toString());
HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg);
// Should fail - return : -1.
assertEquals(-1, dataImporter.dataImport(jsc, 0));
} finally {
if (jsc != null) {
jsc.stop();
}
}
dfs.create(schemaFile).write("Random invalid schema data".getBytes());
// Should fail - return : -1.
assertEquals(-1, dataImporter.dataImport(jsc, 0));
} finally {
if (jsc != null) {
jsc.stop();
}
}
}
/**
* Test for missing rowKey and partitionKey.
*/
@Test
public void testRowAndPartitionKey() throws Exception {
JavaSparkContext jsc = null;
try {
jsc = getJavaSparkContext();
/**
* Test for missing rowKey and partitionKey.
*/
@Test
public void testRowAndPartitionKey() throws Exception {
JavaSparkContext jsc = null;
try {
jsc = getJavaSparkContext();
// Test root folder.
String basePath = (new Path(dfsBasePath,
Thread.currentThread().getStackTrace()[1].getMethodName())).toString();
// Hoodie root folder
Path hoodieFolder = new Path(basePath, "testTarget");
// Test root folder.
String basePath = (new Path(dfsBasePath,
Thread.currentThread().getStackTrace()[1].getMethodName())).toString();
// Hoodie root folder
Path hoodieFolder = new Path(basePath, "testTarget");
//Create generic records.
Path srcFolder = new Path(basePath, "testSrc");
createRecords(srcFolder);
//Create generic records.
Path srcFolder = new Path(basePath, "testSrc");
createRecords(srcFolder);
// Create schema file.
Path schemaFile = new Path(basePath.toString(), "missingFile.schema");
createSchemaFile(schemaFile.toString());
// Create schema file.
Path schemaFile = new Path(basePath.toString(), "missingFile.schema");
createSchemaFile(schemaFile.toString());
HDFSParquetImporter dataImporter;
HDFSParquetImporter.Config cfg;
HDFSParquetImporter dataImporter;
HDFSParquetImporter.Config cfg;
// Check for invalid row key.
cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(),
"testTable", "COPY_ON_WRITE", "invalidRowKey", "timestamp",
1, schemaFile.toString());
dataImporter = new HDFSParquetImporter(cfg);
assertEquals(-1, dataImporter.dataImport(jsc, 0));
// Check for invalid row key.
cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(),
"testTable", "COPY_ON_WRITE", "invalidRowKey", "timestamp",
1, schemaFile.toString());
dataImporter = new HDFSParquetImporter(cfg);
assertEquals(-1, dataImporter.dataImport(jsc, 0));
// Check for invalid partition key.
cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(),
"testTable", "COPY_ON_WRITE", "_row_key", "invalidTimeStamp",
1, schemaFile.toString());
dataImporter = new HDFSParquetImporter(cfg);
assertEquals(-1, dataImporter.dataImport(jsc, 0));
// Check for invalid partition key.
cfg = getHDFSParquetImporterConfig(srcFolder.toString(), hoodieFolder.toString(),
"testTable", "COPY_ON_WRITE", "_row_key", "invalidTimeStamp",
1, schemaFile.toString());
dataImporter = new HDFSParquetImporter(cfg);
assertEquals(-1, dataImporter.dataImport(jsc, 0));
} finally {
if (jsc != null) {
jsc.stop();
}
}
} finally {
if (jsc != null) {
jsc.stop();
}
}
}
private HDFSParquetImporter.Config getHDFSParquetImporterConfig(String srcPath, String targetPath,
String tableName, String tableType, String rowKey, String partitionKey, int parallelism,
String schemaFile) {
HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config();
cfg.srcPath = srcPath;
cfg.targetPath = targetPath;
cfg.tableName = tableName;
cfg.tableType = tableType;
cfg.rowKey = rowKey;
cfg.partitionKey = partitionKey;
cfg.parallelism = parallelism;
cfg.schemaFile = schemaFile;
return cfg;
}
private HDFSParquetImporter.Config getHDFSParquetImporterConfig(String srcPath, String targetPath,
String tableName, String tableType, String rowKey, String partitionKey, int parallelism,
String schemaFile) {
HDFSParquetImporter.Config cfg = new HDFSParquetImporter.Config();
cfg.srcPath = srcPath;
cfg.targetPath = targetPath;
cfg.tableName = tableName;
cfg.tableType = tableType;
cfg.rowKey = rowKey;
cfg.partitionKey = partitionKey;
cfg.parallelism = parallelism;
cfg.schemaFile = schemaFile;
return cfg;
}
private JavaSparkContext getJavaSparkContext() {
// Initialize a local spark env
SparkConf sparkConf = new SparkConf().setAppName("TestConversionCommand").setMaster("local[1]");
sparkConf = HoodieWriteClient.registerClasses(sparkConf);
return new JavaSparkContext(HoodieReadClient.addHoodieSupport(sparkConf));
}
private JavaSparkContext getJavaSparkContext() {
// Initialize a local spark env
SparkConf sparkConf = new SparkConf().setAppName("TestConversionCommand").setMaster("local[1]");
sparkConf = HoodieWriteClient.registerClasses(sparkConf);
return new JavaSparkContext(HoodieReadClient.addHoodieSupport(sparkConf));
}
}

View File

@@ -16,9 +16,15 @@
package com.uber.hoodie.utilities;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.util.FSUtils;
import java.io.File;
import java.io.IOException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.SparkConf;
@@ -28,124 +34,129 @@ import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import static org.junit.Assert.*;
public class TestHoodieSnapshotCopier {
private String rootPath = null;
private String basePath = null;
private String outputPath = null;
private FileSystem fs = null;
private JavaSparkContext jsc = null;
@Before
public void init() throws IOException {
// Prepare directories
TemporaryFolder folder = new TemporaryFolder();
folder.create();
rootPath = folder.getRoot().getAbsolutePath();
basePath = rootPath + "/" + HoodieTestUtils.RAW_TRIPS_TEST_NAME;
HoodieTestUtils.init(basePath);
outputPath = rootPath + "/output";
fs = FSUtils.getFs();
// Start a local Spark job
SparkConf conf = new SparkConf().setAppName("snapshot-test-job").setMaster("local[2]");
jsc = new JavaSparkContext(conf);
private String rootPath = null;
private String basePath = null;
private String outputPath = null;
private FileSystem fs = null;
private JavaSparkContext jsc = null;
@Before
public void init() throws IOException {
// Prepare directories
TemporaryFolder folder = new TemporaryFolder();
folder.create();
rootPath = folder.getRoot().getAbsolutePath();
basePath = rootPath + "/" + HoodieTestUtils.RAW_TRIPS_TEST_NAME;
HoodieTestUtils.init(basePath);
outputPath = rootPath + "/output";
fs = FSUtils.getFs();
// Start a local Spark job
SparkConf conf = new SparkConf().setAppName("snapshot-test-job").setMaster("local[2]");
jsc = new JavaSparkContext(conf);
}
@Test
public void testEmptySnapshotCopy() throws IOException {
// There is no real data (only .hoodie directory)
assertEquals(fs.listStatus(new Path(basePath)).length, 1);
assertFalse(fs.exists(new Path(outputPath)));
// Do the snapshot
HoodieSnapshotCopier copier = new HoodieSnapshotCopier();
copier.snapshot(jsc, basePath, outputPath, true);
// Nothing changed; we just bail out
assertEquals(fs.listStatus(new Path(basePath)).length, 1);
assertFalse(fs.exists(new Path(outputPath + "/_SUCCESS")));
}
//TODO - uncomment this after fixing test failures
//@Test
public void testSnapshotCopy() throws Exception {
// Generate some commits and corresponding parquets
String commitTime1 = "20160501010101";
String commitTime2 = "20160502020601";
String commitTime3 = "20160506030611";
new File(basePath + "/.hoodie").mkdirs();
new File(basePath + "/.hoodie/hoodie.properties").createNewFile();
// Only first two have commit files
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".inflight").createNewFile();
// Some parquet files
new File(basePath + "/2016/05/01/").mkdirs();
new File(basePath + "/2016/05/02/").mkdirs();
new File(basePath + "/2016/05/06/").mkdirs();
HoodieTestDataGenerator.writePartitionMetadata(fs,
new String[]{"2016/05/01", "2016/05/02", "2016/05/06"},
basePath);
// Make commit1
File file11 = new File(
basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime1, 1, "id11"));
file11.createNewFile();
File file12 = new File(
basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime1, 1, "id12"));
file12.createNewFile();
File file13 = new File(
basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime1, 1, "id13"));
file13.createNewFile();
// Make commit2
File file21 = new File(
basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime2, 1, "id21"));
file21.createNewFile();
File file22 = new File(
basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime2, 1, "id22"));
file22.createNewFile();
File file23 = new File(
basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime2, 1, "id23"));
file23.createNewFile();
// Make commit3
File file31 = new File(
basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime3, 1, "id31"));
file31.createNewFile();
File file32 = new File(
basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime3, 1, "id32"));
file32.createNewFile();
File file33 = new File(
basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime3, 1, "id33"));
file33.createNewFile();
// Do a snapshot copy
HoodieSnapshotCopier copier = new HoodieSnapshotCopier();
copier.snapshot(jsc, basePath, outputPath, false);
// Check results
assertTrue(fs.exists(new Path(outputPath + "/2016/05/01/" + file11.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/02/" + file12.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/06/" + file13.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/01/" + file21.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/02/" + file22.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/06/" + file23.getName())));
assertFalse(fs.exists(new Path(outputPath + "/2016/05/01/" + file31.getName())));
assertFalse(fs.exists(new Path(outputPath + "/2016/05/02/" + file32.getName())));
assertFalse(fs.exists(new Path(outputPath + "/2016/05/06/" + file33.getName())));
assertTrue(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime1 + ".commit")));
assertTrue(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime2 + ".commit")));
assertFalse(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime3 + ".commit")));
assertFalse(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime3 + ".inflight")));
assertTrue(fs.exists(new Path(outputPath + "/.hoodie/hoodie.properties")));
assertTrue(fs.exists(new Path(outputPath + "/_SUCCESS")));
}
@After
public void cleanup() {
if (rootPath != null) {
new File(rootPath).delete();
}
@Test
public void testEmptySnapshotCopy() throws IOException {
// There is no real data (only .hoodie directory)
assertEquals(fs.listStatus(new Path(basePath)).length, 1);
assertFalse(fs.exists(new Path(outputPath)));
// Do the snapshot
HoodieSnapshotCopier copier = new HoodieSnapshotCopier();
copier.snapshot(jsc, basePath, outputPath, true);
// Nothing changed; we just bail out
assertEquals(fs.listStatus(new Path(basePath)).length, 1);
assertFalse(fs.exists(new Path(outputPath + "/_SUCCESS")));
}
//TODO - uncomment this after fixing test failures
//@Test
public void testSnapshotCopy() throws Exception {
// Generate some commits and corresponding parquets
String commitTime1 = "20160501010101";
String commitTime2 = "20160502020601";
String commitTime3 = "20160506030611";
new File(basePath + "/.hoodie").mkdirs();
new File(basePath + "/.hoodie/hoodie.properties").createNewFile();
// Only first two have commit files
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
new File(basePath + "/.hoodie/" + commitTime3 + ".inflight").createNewFile();
// Some parquet files
new File(basePath + "/2016/05/01/").mkdirs();
new File(basePath + "/2016/05/02/").mkdirs();
new File(basePath + "/2016/05/06/").mkdirs();
HoodieTestDataGenerator.writePartitionMetadata(fs,
new String[] {"2016/05/01", "2016/05/02", "2016/05/06"},
basePath);
// Make commit1
File file11 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime1, 1, "id11"));
file11.createNewFile();
File file12 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime1, 1, "id12"));
file12.createNewFile();
File file13 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime1, 1, "id13"));
file13.createNewFile();
// Make commit2
File file21 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime2, 1, "id21"));
file21.createNewFile();
File file22 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime2, 1, "id22"));
file22.createNewFile();
File file23 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime2, 1, "id23"));
file23.createNewFile();
// Make commit3
File file31 = new File(basePath + "/2016/05/01/" + FSUtils.makeDataFileName(commitTime3, 1, "id31"));
file31.createNewFile();
File file32 = new File(basePath + "/2016/05/02/" + FSUtils.makeDataFileName(commitTime3, 1, "id32"));
file32.createNewFile();
File file33 = new File(basePath + "/2016/05/06/" + FSUtils.makeDataFileName(commitTime3, 1, "id33"));
file33.createNewFile();
// Do a snapshot copy
HoodieSnapshotCopier copier = new HoodieSnapshotCopier();
copier.snapshot(jsc, basePath, outputPath, false);
// Check results
assertTrue(fs.exists(new Path(outputPath + "/2016/05/01/" + file11.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/02/" + file12.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/06/" + file13.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/01/" + file21.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/02/" + file22.getName())));
assertTrue(fs.exists(new Path(outputPath + "/2016/05/06/" + file23.getName())));
assertFalse(fs.exists(new Path(outputPath + "/2016/05/01/" + file31.getName())));
assertFalse(fs.exists(new Path(outputPath + "/2016/05/02/" + file32.getName())));
assertFalse(fs.exists(new Path(outputPath + "/2016/05/06/" + file33.getName())));
assertTrue(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime1 + ".commit")));
assertTrue(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime2 + ".commit")));
assertFalse(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime3 + ".commit")));
assertFalse(fs.exists(new Path(outputPath + "/.hoodie/" + commitTime3 + ".inflight")));
assertTrue(fs.exists(new Path(outputPath + "/.hoodie/hoodie.properties")));
assertTrue(fs.exists(new Path(outputPath + "/_SUCCESS")));
}
@After
public void cleanup() {
if (rootPath != null) {
new File(rootPath).delete();
}
if (jsc != null) {
jsc.stop();
}
if (jsc != null) {
jsc.stop();
}
}
}

View File

@@ -16,7 +16,6 @@
log4j.rootLogger=WARN, A1
log4j.category.com.uber=INFO
log4j.category.org.apache.parquet.hadoop=WARN
# A1 is set to be a ConsoleAppender.
log4j.appender.A1=org.apache.log4j.ConsoleAppender
# A1 uses PatternLayout.