1
0

[HUDI-296] Explore use of spotless to auto fix formatting errors (#945)

- Add spotless format fixing to project
- One time reformatting for conformity
- Build fails for formatting changes and mvn spotless:apply autofixes them
This commit is contained in:
leesf
2019-10-10 20:19:40 +08:00
committed by vinoth chandar
parent 834c591955
commit b19bed442d
381 changed files with 7350 additions and 9064 deletions

View File

@@ -84,18 +84,14 @@ public class HoodieInputFormatTest {
// Before the commit
files = inputFormat.listStatus(jobConf);
assertEquals(10, files.length);
ensureFilesInCommit(
"Commit 200 has not been committed. We should not see files from this commit", files, "200",
0);
ensureFilesInCommit("Commit 200 has not been committed. We should not see files from this commit", files, "200", 0);
InputFormatTestUtil.commit(basePath, "200");
files = inputFormat.listStatus(jobConf);
assertEquals(10, files.length);
ensureFilesInCommit(
"5 files have been updated to commit 200. We should see 5 files from commit 200 and 5 "
+ "files from 100 commit", files, "200", 5);
ensureFilesInCommit(
"5 files have been updated to commit 200. We should see 5 files from commit 100 and 5 "
+ "files from 200 commit", files, "100", 5);
ensureFilesInCommit("5 files have been updated to commit 200. We should see 5 files from commit 200 and 5 "
+ "files from 100 commit", files, "200", 5);
ensureFilesInCommit("5 files have been updated to commit 200. We should see 5 files from commit 100 and 5 "
+ "files from 200 commit", files, "100", 5);
}
@Test
@@ -110,9 +106,8 @@ public class HoodieInputFormatTest {
InputFormatTestUtil.setupIncremental(jobConf, "100", 1);
FileStatus[] files = inputFormat.listStatus(jobConf);
assertEquals(
"We should exclude commit 100 when returning incremental pull with start commit time as "
+ "100", 0, files.length);
assertEquals("We should exclude commit 100 when returning incremental pull with start commit time as " + "100", 0,
files.length);
}
@Test
@@ -140,43 +135,31 @@ public class HoodieInputFormatTest {
InputFormatTestUtil.setupIncremental(jobConf, "100", 1);
FileStatus[] files = inputFormat.listStatus(jobConf);
assertEquals("Pulling 1 commit from 100, should get us the 5 files committed at 200", 5,
files.length);
ensureFilesInCommit("Pulling 1 commit from 100, should get us the 5 files committed at 200",
files, "200", 5);
assertEquals("Pulling 1 commit from 100, should get us the 5 files committed at 200", 5, files.length);
ensureFilesInCommit("Pulling 1 commit from 100, should get us the 5 files committed at 200", files, "200", 5);
InputFormatTestUtil.setupIncremental(jobConf, "100", 3);
files = inputFormat.listStatus(jobConf);
assertEquals(
"Pulling 3 commits from 100, should get us the 3 files from 400 commit, 1 file from 300 "
+ "commit and 1 file from 200 commit", 5, files.length);
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 3 files from 400 commit",
files, "400", 3);
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 300 commit",
files, "300", 1);
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 200 commit",
files, "200", 1);
assertEquals("Pulling 3 commits from 100, should get us the 3 files from 400 commit, 1 file from 300 "
+ "commit and 1 file from 200 commit", 5, files.length);
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 3 files from 400 commit", files, "400", 3);
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 300 commit", files, "300", 1);
ensureFilesInCommit("Pulling 3 commits from 100, should get us the 1 files from 200 commit", files, "200", 1);
InputFormatTestUtil.setupIncremental(jobConf, "100", HoodieHiveUtil.MAX_COMMIT_ALL);
files = inputFormat.listStatus(jobConf);
assertEquals(
"Pulling all commits from 100, should get us the 1 file from each of 200,300,400,500,400 "
+ "commits", 5, files.length);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 600 commit",
files, "600", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 500 commit",
files, "500", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 400 commit",
files, "400", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 300 commit",
files, "300", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 200 commit",
files, "200", 1);
assertEquals("Pulling all commits from 100, should get us the 1 file from each of 200,300,400,500,400 " + "commits",
5, files.length);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 600 commit", files, "600", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 500 commit", files, "500", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 400 commit", files, "400", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 300 commit", files, "300", 1);
ensureFilesInCommit("Pulling all commits from 100, should get us the 1 files from 200 commit", files, "200", 1);
}
//TODO enable this after enabling predicate pushdown
// TODO enable this after enabling predicate pushdown
public void testPredicatePushDown() throws IOException {
// initial commit
Schema schema = InputFormatTestUtil.readSchema("/sample1.avsc");
@@ -186,8 +169,7 @@ public class HoodieInputFormatTest {
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
// check whether we have 10 records at this point
ensureRecordsInCommit("We need to have 10 records at this point for commit " + commit1, commit1,
10, 10);
ensureRecordsInCommit("We need to have 10 records at this point for commit " + commit1, commit1, 10, 10);
// update 2 records in the original parquet file and save it as commit 200
String commit2 = "20160629193623";
@@ -196,27 +178,23 @@ public class HoodieInputFormatTest {
InputFormatTestUtil.setupIncremental(jobConf, commit1, 1);
// check whether we have 2 records at this point
ensureRecordsInCommit(
"We need to have 2 records that was modified at commit " + commit2 + " and no more",
commit2, 2, 2);
ensureRecordsInCommit("We need to have 2 records that was modified at commit " + commit2 + " and no more", commit2,
2, 2);
// Make sure we have the 10 records if we roll back the stattime
InputFormatTestUtil.setupIncremental(jobConf, "0", 2);
ensureRecordsInCommit(
"We need to have 8 records that was modified at commit " + commit1 + " and no more",
commit1, 8, 10);
ensureRecordsInCommit(
"We need to have 2 records that was modified at commit " + commit2 + " and no more",
commit2, 2, 10);
ensureRecordsInCommit("We need to have 8 records that was modified at commit " + commit1 + " and no more", commit1,
8, 10);
ensureRecordsInCommit("We need to have 2 records that was modified at commit " + commit2 + " and no more", commit2,
2, 10);
}
private void ensureRecordsInCommit(String msg, String commit, int expectedNumberOfRecordsInCommit,
int totalExpected) throws IOException {
private void ensureRecordsInCommit(String msg, String commit, int expectedNumberOfRecordsInCommit, int totalExpected)
throws IOException {
int actualCount = 0;
int totalCount = 0;
InputSplit[] splits = inputFormat.getSplits(jobConf, 1);
for (InputSplit split : splits) {
RecordReader<NullWritable, ArrayWritable> recordReader = inputFormat
.getRecordReader(split, jobConf, null);
RecordReader<NullWritable, ArrayWritable> recordReader = inputFormat.getRecordReader(split, jobConf, null);
NullWritable key = recordReader.createKey();
ArrayWritable writable = recordReader.createValue();
@@ -234,8 +212,7 @@ public class HoodieInputFormatTest {
assertEquals(msg, totalExpected, totalCount);
}
public static void ensureFilesInCommit(String msg, FileStatus[] files, String commit,
int expected) {
public static void ensureFilesInCommit(String msg, FileStatus[] files, String commit, int expected) {
int count = 0;
for (FileStatus file : files) {
String commitTs = FSUtils.getCommitTime(file.getPath().getName());

View File

@@ -44,21 +44,20 @@ public class InputFormatTestUtil {
private static String TEST_WRITE_TOKEN = "1-0-1";
public static File prepareDataset(TemporaryFolder basePath, int numberOfFiles,
String commitNumber) throws IOException {
public static File prepareDataset(TemporaryFolder basePath, int numberOfFiles, String commitNumber)
throws IOException {
basePath.create();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
File partitionPath = basePath.newFolder("2016", "05", "01");
for (int i = 0; i < numberOfFiles; i++) {
File dataFile = new File(partitionPath,
FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i));
File dataFile = new File(partitionPath, FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i));
dataFile.createNewFile();
}
return partitionPath;
}
public static void simulateUpdates(File directory, final String originalCommit,
int numberOfFilesUpdated, String newCommit, boolean randomize) throws IOException {
public static void simulateUpdates(File directory, final String originalCommit, int numberOfFilesUpdated,
String newCommit, boolean randomize) throws IOException {
List<File> dataFiles = Arrays.asList(directory.listFiles(new FilenameFilter() {
@Override
public boolean accept(File dir, String name) {
@@ -69,8 +68,7 @@ public class InputFormatTestUtil {
if (randomize) {
Collections.shuffle(dataFiles);
}
List<File> toUpdateList = dataFiles
.subList(0, Math.min(numberOfFilesUpdated, dataFiles.size()));
List<File> toUpdateList = dataFiles.subList(0, Math.min(numberOfFilesUpdated, dataFiles.size()));
for (File file : toUpdateList) {
String fileId = FSUtils.getFileId(file.getName());
File dataFile = new File(directory, FSUtils.makeDataFileName(newCommit, TEST_WRITE_TOKEN, fileId));
@@ -88,18 +86,17 @@ public class InputFormatTestUtil {
new File(basePath.getRoot().toString() + "/.hoodie/", commitNumber + ".deltacommit").createNewFile();
}
public static void setupIncremental(JobConf jobConf, String startCommit,
int numberOfCommitsToPull) {
String modePropertyName = String
.format(HoodieHiveUtil.HOODIE_CONSUME_MODE_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
public static void setupIncremental(JobConf jobConf, String startCommit, int numberOfCommitsToPull) {
String modePropertyName =
String.format(HoodieHiveUtil.HOODIE_CONSUME_MODE_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
jobConf.set(modePropertyName, HoodieHiveUtil.INCREMENTAL_SCAN_MODE);
String startCommitTimestampName = String
.format(HoodieHiveUtil.HOODIE_START_COMMIT_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
String startCommitTimestampName =
String.format(HoodieHiveUtil.HOODIE_START_COMMIT_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
jobConf.set(startCommitTimestampName, startCommit);
String maxCommitPulls = String
.format(HoodieHiveUtil.HOODIE_MAX_COMMIT_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
String maxCommitPulls =
String.format(HoodieHiveUtil.HOODIE_MAX_COMMIT_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
jobConf.setInt(maxCommitPulls, numberOfCommitsToPull);
}
@@ -107,8 +104,8 @@ public class InputFormatTestUtil {
return new Schema.Parser().parse(InputFormatTestUtil.class.getResourceAsStream(location));
}
public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema,
int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException {
public static File prepareParquetDataset(TemporaryFolder basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber) throws IOException {
basePath.create();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
File partitionPath = basePath.newFolder("2016", "05", "01");
@@ -117,8 +114,8 @@ public class InputFormatTestUtil {
}
public static File prepareSimpleParquetDataset(TemporaryFolder basePath, Schema schema,
int numberOfFiles, int numberOfRecords, String commitNumber) throws Exception {
public static File prepareSimpleParquetDataset(TemporaryFolder basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber) throws Exception {
basePath.create();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
File partitionPath = basePath.newFolder("2016", "05", "01");
@@ -126,8 +123,8 @@ public class InputFormatTestUtil {
return partitionPath;
}
public static File prepareNonPartitionedParquetDataset(TemporaryFolder baseDir, Schema schema,
int numberOfFiles, int numberOfRecords, String commitNumber) throws IOException {
public static File prepareNonPartitionedParquetDataset(TemporaryFolder baseDir, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber) throws IOException {
baseDir.create();
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), baseDir.getRoot().toString());
File basePath = baseDir.getRoot();
@@ -135,17 +132,15 @@ public class InputFormatTestUtil {
return basePath;
}
private static void createData(Schema schema,
File partitionPath, int numberOfFiles, int numberOfRecords, String commitNumber)
throws IOException {
private static void createData(Schema schema, File partitionPath, int numberOfFiles, int numberOfRecords,
String commitNumber) throws IOException {
AvroParquetWriter parquetWriter;
for (int i = 0; i < numberOfFiles; i++) {
String fileId = FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i);
File dataFile = new File(partitionPath, fileId);
parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema);
try {
for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber,
fileId)) {
for (GenericRecord record : generateAvroRecords(schema, numberOfRecords, commitNumber, fileId)) {
parquetWriter.write(record);
}
} finally {
@@ -154,9 +149,8 @@ public class InputFormatTestUtil {
}
}
private static void createSimpleData(Schema schema,
File partitionPath, int numberOfFiles, int numberOfRecords, String commitNumber)
throws Exception {
private static void createSimpleData(Schema schema, File partitionPath, int numberOfFiles, int numberOfRecords,
String commitNumber) throws Exception {
AvroParquetWriter parquetWriter;
for (int i = 0; i < numberOfFiles; i++) {
String fileId = FSUtils.makeDataFileName(commitNumber, "1", "fileid" + i);
@@ -179,8 +173,8 @@ public class InputFormatTestUtil {
}
}
private static Iterable<? extends GenericRecord> generateAvroRecords(Schema schema,
int numberOfRecords, String commitTime, String fileId) throws IOException {
private static Iterable<? extends GenericRecord> generateAvroRecords(Schema schema, int numberOfRecords,
String commitTime, String fileId) throws IOException {
List<GenericRecord> records = new ArrayList<>(numberOfRecords);
for (int i = 0; i < numberOfRecords; i++) {
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, commitTime, fileId));
@@ -198,17 +192,14 @@ public class InputFormatTestUtil {
})[0];
String fileId = FSUtils.getFileId(fileToUpdate.getName());
File dataFile = new File(directory, FSUtils.makeDataFileName(newCommit, TEST_WRITE_TOKEN, fileId));
AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()),
schema);
AvroParquetWriter parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema);
try {
for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, originalCommit,
fileId)) {
for (GenericRecord record : generateAvroRecords(schema, totalNumberOfRecords, originalCommit, fileId)) {
if (numberOfRecordsToUpdate > 0) {
// update this record
record.put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, newCommit);
String oldSeqNo = (String) record.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD);
record.put(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD,
oldSeqNo.replace(originalCommit, newCommit));
record.put(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, oldSeqNo.replace(originalCommit, newCommit));
numberOfRecordsToUpdate--;
}
parquetWriter.write(record);

View File

@@ -36,12 +36,12 @@ import org.junit.rules.TemporaryFolder;
/**
*/
public class TestHoodieROTablePathFilter extends HoodieCommonTestHarness {
@Before
public void setUp() throws Exception {
initMetaClient();
}
@Test
public void testHoodiePaths() throws IOException {
// Create a temp folder as the base path
@@ -61,28 +61,26 @@ public class TestHoodieROTablePathFilter extends HoodieCommonTestHarness {
Path partitionPath = new Path("file://" + basePath + File.separator + "2017/01/01");
assertTrue("Directories should be accepted", pathFilter.accept(partitionPath));
assertTrue(pathFilter.accept(new Path(
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f1"))));
assertFalse(pathFilter.accept(new Path(
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f2"))));
assertTrue(pathFilter.accept(new Path(
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f3"))));
assertTrue(pathFilter.accept(new Path(
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "002", "f2"))));
assertFalse(pathFilter.accept(new Path(
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "003", "f3"))));
assertTrue(
pathFilter.accept(new Path("file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f1"))));
assertFalse(
pathFilter.accept(new Path("file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f2"))));
assertTrue(
pathFilter.accept(new Path("file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "001", "f3"))));
assertTrue(
pathFilter.accept(new Path("file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "002", "f2"))));
assertFalse(
pathFilter.accept(new Path("file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "003", "f3"))));
assertFalse(pathFilter.accept(new Path("file:///" + HoodieTestUtils.getCommitFilePath(basePath, "001"))));
assertFalse(pathFilter.accept(new Path("file:///" + HoodieTestUtils.getCommitFilePath(basePath, "002"))));
assertFalse(pathFilter.accept(new Path("file:///"
+ HoodieTestUtils.getInflightCommitFilePath(basePath, "003"))));
assertFalse(pathFilter.accept(new Path("file:///"
+ HoodieTestUtils.getRequestedCompactionFilePath(basePath, "004"))));
assertFalse(pathFilter.accept(new Path("file:///" + basePath + "/"
+ HoodieTableMetaClient.METAFOLDER_NAME + "/")));
assertFalse(pathFilter.accept(new Path("file:///" + HoodieTestUtils.getInflightCommitFilePath(basePath, "003"))));
assertFalse(
pathFilter.accept(new Path("file:///" + HoodieTestUtils.getRequestedCompactionFilePath(basePath, "004"))));
assertFalse(pathFilter.accept(new Path("file:///" + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/")));
assertFalse(pathFilter.accept(new Path("file:///" + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME)));
assertFalse(pathFilter.accept(new Path(
"file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "003", "f3"))));
assertFalse(
pathFilter.accept(new Path("file:///" + HoodieTestUtils.getDataFilePath(basePath, "2017/01/01", "003", "f3"))));
}

View File

@@ -33,16 +33,9 @@ public class TestRecordReaderValueIterator {
@Test
public void testValueIterator() {
String[] values = new String[]{
"hoodie",
"efficient",
"new project",
"realtime",
"spark",
"dataset",
};
List<Pair<Integer, String>> entries = IntStream.range(0, values.length)
.boxed().map(idx -> Pair.of(idx, values[idx])).collect(Collectors.toList());
String[] values = new String[] {"hoodie", "efficient", "new project", "realtime", "spark", "dataset",};
List<Pair<Integer, String>> entries =
IntStream.range(0, values.length).boxed().map(idx -> Pair.of(idx, values[idx])).collect(Collectors.toList());
TestRecordReader reader = new TestRecordReader(entries);
RecordReaderValueIterator<IntWritable, Text> itr = new RecordReaderValueIterator<IntWritable, Text>(reader);
for (int i = 0; i < values.length; i++) {

View File

@@ -91,28 +91,22 @@ public class HoodieRealtimeRecordReaderTest {
@Rule
public TemporaryFolder basePath = new TemporaryFolder();
private Writer writeLogFile(File partitionDir, Schema schema, String fileId,
String baseCommit, String newCommit, int numberOfRecords)
throws InterruptedException, IOException {
private Writer writeLogFile(File partitionDir, Schema schema, String fileId, String baseCommit, String newCommit,
int numberOfRecords) throws InterruptedException, IOException {
return writeDataBlockToLogFile(partitionDir, schema, fileId, baseCommit, newCommit, numberOfRecords, 0, 0);
}
private Writer writeRollback(File partitionDir, Schema schema, String fileId,
String baseCommit, String newCommit, String rolledBackInstant, int logVersion)
throws InterruptedException, IOException {
Writer writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(partitionDir.getPath()))
.withFileId(fileId).overBaseCommit(baseCommit)
.withFs(fs)
.withLogVersion(logVersion)
.withLogWriteToken("1-0-1")
private Writer writeRollback(File partitionDir, Schema schema, String fileId, String baseCommit, String newCommit,
String rolledBackInstant, int logVersion) throws InterruptedException, IOException {
Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath())).withFileId(fileId)
.overBaseCommit(baseCommit).withFs(fs).withLogVersion(logVersion).withLogWriteToken("1-0-1")
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
// generate metadata
Map<HeaderMetadataType, String> header = Maps.newHashMap();
header.put(HeaderMetadataType.INSTANT_TIME, newCommit);
header.put(HeaderMetadataType.TARGET_INSTANT_TIME, rolledBackInstant);
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK
.ordinal()));
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
// if update belongs to an existing log file
writer = writer.appendBlock(new HoodieCommandBlock(header));
return writer;
@@ -121,12 +115,9 @@ public class HoodieRealtimeRecordReaderTest {
private HoodieLogFormat.Writer writeDataBlockToLogFile(File partitionDir, Schema schema, String fileId,
String baseCommit, String newCommit, int numberOfRecords, int offset, int logVersion)
throws InterruptedException, IOException {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId)
.withLogVersion(logVersion)
.withLogWriteToken("1-0-1")
.overBaseCommit(baseCommit).withFs(fs).build();
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId).withLogVersion(logVersion)
.withLogWriteToken("1-0-1").overBaseCommit(baseCommit).withFs(fs).build();
List<IndexedRecord> records = new ArrayList<>();
for (int i = offset; i < offset + numberOfRecords; i++) {
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0"));
@@ -141,19 +132,17 @@ public class HoodieRealtimeRecordReaderTest {
}
private HoodieLogFormat.Writer writeRollbackBlockToLogFile(File partitionDir, Schema schema, String fileId,
String baseCommit, String newCommit, String oldCommit, int logVersion)
throws InterruptedException, IOException {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId)
.overBaseCommit(baseCommit).withLogVersion(logVersion).withFs(fs).build();
String baseCommit, String newCommit, String oldCommit, int logVersion) throws InterruptedException, IOException {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder().onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId).overBaseCommit(baseCommit)
.withLogVersion(logVersion).withFs(fs).build();
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, oldCommit);
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK
.ordinal()));
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
HoodieCommandBlock rollbackBlock = new HoodieCommandBlock(header);
writer = writer.appendBlock(rollbackBlock);
return writer;
@@ -172,12 +161,10 @@ public class HoodieRealtimeRecordReaderTest {
public void testReader(boolean partitioned) throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(),
HoodieTableType.MERGE_ON_READ);
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
String baseInstant = "100";
File partitionDir =
partitioned ? InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 100, baseInstant)
: InputFormatTestUtil.prepareNonPartitionedParquetDataset(basePath, schema, 1, 100, baseInstant);
File partitionDir = partitioned ? InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, 100, baseInstant)
: InputFormatTestUtil.prepareNonPartitionedParquetDataset(basePath, schema, 1, 100, baseInstant);
InputFormatTestUtil.commit(basePath, baseInstant);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@@ -187,9 +174,9 @@ public class HoodieRealtimeRecordReaderTest {
logVersionsWithAction.add(Pair.of(HoodieTimeline.DELTA_COMMIT_ACTION, 2));
// TODO: HUDI-154 Once Hive 2.x PR (PR-674) is merged, enable this change
// logVersionsWithAction.add(Pair.of(HoodieTimeline.ROLLBACK_ACTION, 3));
FileSlice fileSlice = new FileSlice(partitioned ? FSUtils.getRelativePartitionPath(new Path(
basePath.getRoot().getAbsolutePath()), new Path(partitionDir.getAbsolutePath())) : "default",
baseInstant, "fileid0");
FileSlice fileSlice =
new FileSlice(partitioned ? FSUtils.getRelativePartitionPath(new Path(basePath.getRoot().getAbsolutePath()),
new Path(partitionDir.getAbsolutePath())) : "default", baseInstant, "fileid0");
logVersionsWithAction.stream().forEach(logVersionWithAction -> {
try {
// update files or generate new log file
@@ -197,55 +184,53 @@ public class HoodieRealtimeRecordReaderTest {
String action = logVersionWithAction.getKey();
int baseInstantTs = Integer.parseInt(baseInstant);
String instantTime = String.valueOf(baseInstantTs + logVersion);
String latestInstant = action.equals(HoodieTimeline.ROLLBACK_ACTION)
? String.valueOf(baseInstantTs + logVersion - 2) : instantTime;
String latestInstant =
action.equals(HoodieTimeline.ROLLBACK_ACTION) ? String.valueOf(baseInstantTs + logVersion - 2)
: instantTime;
HoodieLogFormat.Writer writer = null;
if (action.equals(HoodieTimeline.ROLLBACK_ACTION)) {
writer = writeRollback(partitionDir, schema, "fileid0", baseInstant,
instantTime, String.valueOf(baseInstantTs + logVersion - 1), logVersion);
writer = writeRollback(partitionDir, schema, "fileid0", baseInstant, instantTime,
String.valueOf(baseInstantTs + logVersion - 1), logVersion);
} else {
writer = writeDataBlockToLogFile(partitionDir, schema, "fileid0", baseInstant,
instantTime, 100, 0, logVersion);
writer =
writeDataBlockToLogFile(partitionDir, schema, "fileid0", baseInstant, instantTime, 100, 0, logVersion);
}
long size = writer.getCurrentSize();
writer.close();
assertTrue("block - size should be > 0", size > 0);
//create a split with baseFile (parquet file written earlier) and new log file(s)
// create a split with baseFile (parquet file written earlier) and new log file(s)
fileSlice.addLogFile(writer.getLogFile());
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + baseInstant + ".parquet"), 0, 1,
jobConf), basePath.getRoot().getPath(),
fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).map(h -> h.getPath().toString())
.collect(Collectors.toList()), instantTime);
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + baseInstant + ".parquet"), 0, 1, jobConf),
basePath.getRoot().getPath(), fileSlice.getLogFiles().sorted(HoodieLogFile.getLogFileComparator())
.map(h -> h.getPath().toString()).collect(Collectors.toList()),
instantTime);
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader =
new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
jobConf, null);
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), jobConf, null);
JobConf jobConf = new JobConf();
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(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
if (partitioned) {
jobConf.set("partition_columns", "datestr");
}
//validate record reader compaction
// validate record reader compaction
HoodieRealtimeRecordReader recordReader = new HoodieRealtimeRecordReader(split, jobConf, reader);
//use reader to read base Parquet File and log file, merge in flight and return latest commit
//here all 100 records should be updated, see above
// use reader to read base Parquet File and log file, merge in flight and return latest commit
// here all 100 records should be updated, see above
NullWritable key = recordReader.createKey();
ArrayWritable value = recordReader.createValue();
while (recordReader.next(key, value)) {
Writable[] values = value.get();
//check if the record written is with latest commit, here "101"
// check if the record written is with latest commit, here "101"
Assert.assertEquals(latestInstant, values[0].toString());
key = recordReader.createKey();
value = recordReader.createValue();
@@ -263,53 +248,48 @@ public class HoodieRealtimeRecordReaderTest {
public void testUnMergedReader() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(),
HoodieTableType.MERGE_ON_READ);
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
String commitTime = "100";
final int numRecords = 1000;
final int firstBatchLastRecordKey = numRecords - 1;
final int secondBatchLastRecordKey = 2 * numRecords - 1;
File partitionDir = InputFormatTestUtil
.prepareParquetDataset(basePath, schema, 1, numRecords, commitTime);
File partitionDir = InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, numRecords, commitTime);
InputFormatTestUtil.commit(basePath, commitTime);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
// insert new records to log file
String newCommitTime = "101";
HoodieLogFormat.Writer writer = writeDataBlockToLogFile(partitionDir, schema, "fileid0", commitTime,
newCommitTime, numRecords, numRecords, 0);
HoodieLogFormat.Writer writer =
writeDataBlockToLogFile(partitionDir, schema, "fileid0", commitTime, newCommitTime, numRecords, numRecords, 0);
long size = writer.getCurrentSize();
writer.close();
assertTrue("block - size should be > 0", size > 0);
//create a split with baseFile (parquet file written earlier) and new log file(s)
// create a split with baseFile (parquet file written earlier) and new log file(s)
String logFilePath = writer.getLogFile().getPath().toString();
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + commitTime + ".parquet"), 0, 1,
jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + commitTime + ".parquet"), 0, 1, jobConf),
basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader =
new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
jobConf, null);
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), jobConf, null);
JobConf jobConf = new JobConf();
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(","));
String postions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, postions);
jobConf.set("partition_columns", "datestr");
// Enable merge skipping.
jobConf.set("hoodie.realtime.merge.skip", "true");
//validate unmerged record reader
// validate unmerged record reader
RealtimeUnmergedRecordReader recordReader = new RealtimeUnmergedRecordReader(split, jobConf, reader);
//use reader to read base Parquet File and log file
//here all records should be present. Also ensure log records are in order.
// use reader to read base Parquet File and log file
// here all records should be present. Also ensure log records are in order.
NullWritable key = recordReader.createKey();
ArrayWritable value = recordReader.createValue();
int numRecordsAtCommit1 = 0;
@@ -347,43 +327,38 @@ public class HoodieRealtimeRecordReaderTest {
public void testReaderWithNestedAndComplexSchema() throws Exception {
// initial commit
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getComplexEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(),
HoodieTableType.MERGE_ON_READ);
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
String commitTime = "100";
int numberOfRecords = 100;
int numberOfLogRecords = numberOfRecords / 2;
File partitionDir = InputFormatTestUtil
.prepareParquetDataset(basePath, schema, 1, numberOfRecords, commitTime);
File partitionDir = InputFormatTestUtil.prepareParquetDataset(basePath, schema, 1, numberOfRecords, commitTime);
InputFormatTestUtil.commit(basePath, commitTime);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
// update files or generate new log file
String newCommitTime = "101";
HoodieLogFormat.Writer writer = writeLogFile(partitionDir, schema, "fileid0", commitTime,
newCommitTime, numberOfLogRecords);
HoodieLogFormat.Writer writer =
writeLogFile(partitionDir, schema, "fileid0", commitTime, newCommitTime, numberOfLogRecords);
long size = writer.getCurrentSize();
writer.close();
assertTrue("block - size should be > 0", size > 0);
InputFormatTestUtil.deltaCommit(basePath, newCommitTime);
//create a split with baseFile (parquet file written earlier) and new log file(s)
// create a split with baseFile (parquet file written earlier) and new log file(s)
String logFilePath = writer.getLogFile().getPath().toString();
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + commitTime + ".parquet"), 0, 1,
jobConf), basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
new FileSplit(new Path(partitionDir + "/fileid0_1-0-1_" + commitTime + ".parquet"), 0, 1, jobConf),
basePath.getRoot().getPath(), Arrays.asList(logFilePath), newCommitTime);
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader =
new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
jobConf, null);
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), jobConf, null);
JobConf jobConf = new JobConf();
List<Schema.Field> fields = schema.getFields();
String names = fields.stream().map(f -> f.name()).collect(Collectors.joining(","));
String positions = fields.stream().map(f -> String.valueOf(f.pos()))
.collect(Collectors.joining(","));
String positions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
jobConf.set("partition_columns", "datestr");
@@ -401,7 +376,7 @@ public class HoodieRealtimeRecordReaderTest {
++numRecordsRead;
Writable[] values = value.get();
String recordCommitTime;
//check if the record written is with latest commit, here "101"
// check if the record written is with latest commit, here "101"
if (numRecordsRead > numberOfLogRecords) {
recordCommitTime = commitTime;
} else {
@@ -414,12 +389,10 @@ public class HoodieRealtimeRecordReaderTest {
value = recordReader.createValue();
// Assert type STRING
Assert.assertEquals("test value for field: field1", values[5].toString(),
"field" + currentRecordNo);
Assert.assertEquals("test value for field: field1", values[5].toString(), "field" + currentRecordNo);
Assert.assertEquals("test value for field: field2", values[6].toString(),
"field" + currentRecordNo + recordCommitTimeSuffix);
Assert.assertEquals("test value for field: name", values[7].toString(),
"name" + currentRecordNo);
Assert.assertEquals("test value for field: name", values[7].toString(), "name" + currentRecordNo);
// Assert type INT
IntWritable intWritable = (IntWritable) values[8];
@@ -459,20 +432,17 @@ public class HoodieRealtimeRecordReaderTest {
Assert.assertEquals("test value for field: tags[\"mapItem2\"].item1",
((ArrayWritable) mapItemValue2value).get()[0].toString(), "item2" + currentRecordNo);
Assert.assertEquals("test value for field: tags[\"mapItem1\"].item2",
((ArrayWritable) mapItemValue1value).get()[1].toString(),
"item" + currentRecordNo + recordCommitTimeSuffix);
((ArrayWritable) mapItemValue1value).get()[1].toString(), "item" + currentRecordNo + recordCommitTimeSuffix);
Assert.assertEquals("test value for field: tags[\"mapItem2\"].item2",
((ArrayWritable) mapItemValue2value).get()[1].toString(),
"item2" + currentRecordNo + recordCommitTimeSuffix);
((ArrayWritable) mapItemValue2value).get()[1].toString(), "item2" + currentRecordNo + recordCommitTimeSuffix);
// Assert type RECORD
ArrayWritable recordItem = (ArrayWritable) values[13];
Writable[] nestedRecord = recordItem.get();
Assert.assertEquals("test value for field: testNestedRecord.isAdmin",
((BooleanWritable) nestedRecord[0]).get(), false);
Assert
.assertEquals("test value for field: testNestedRecord.userId", nestedRecord[1].toString(),
"UserId" + currentRecordNo + recordCommitTimeSuffix);
Assert.assertEquals("test value for field: testNestedRecord.isAdmin", ((BooleanWritable) nestedRecord[0]).get(),
false);
Assert.assertEquals("test value for field: testNestedRecord.userId", nestedRecord[1].toString(),
"UserId" + currentRecordNo + recordCommitTimeSuffix);
// Assert type ARRAY
ArrayWritable arrayValue = (ArrayWritable) values[14];
@@ -489,13 +459,12 @@ public class HoodieRealtimeRecordReaderTest {
// initial commit
List<String> logFilePaths = new ArrayList<>();
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(),
HoodieTableType.MERGE_ON_READ);
HoodieTestUtils.init(hadoopConf, basePath.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
String commitTime = "100";
int numberOfRecords = 100;
int numberOfLogRecords = numberOfRecords / 2;
File partitionDir = InputFormatTestUtil
.prepareSimpleParquetDataset(basePath, schema, 1, numberOfRecords, commitTime);
File partitionDir =
InputFormatTestUtil.prepareSimpleParquetDataset(basePath, schema, 1, numberOfRecords, commitTime);
InputFormatTestUtil.commit(basePath, commitTime);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@@ -504,8 +473,8 @@ public class HoodieRealtimeRecordReaderTest {
// update files and generate new log file but don't commit
schema = SchemaTestUtil.getComplexEvolvedSchema();
String newCommitTime = "101";
HoodieLogFormat.Writer writer = writeDataBlockToLogFile(partitionDir, schema, "fileid0", commitTime,
newCommitTime, numberOfLogRecords, 0, 1);
HoodieLogFormat.Writer writer =
writeDataBlockToLogFile(partitionDir, schema, "fileid0", commitTime, newCommitTime, numberOfLogRecords, 0, 1);
long size = writer.getCurrentSize();
logFilePaths.add(writer.getLogFile().getPath().toString());
writer.close();
@@ -513,23 +482,20 @@ public class HoodieRealtimeRecordReaderTest {
// write rollback for the previous block in new log file version
newCommitTime = "102";
writer = writeRollbackBlockToLogFile(partitionDir, schema, "fileid0", commitTime,
newCommitTime, "101", 1);
writer = writeRollbackBlockToLogFile(partitionDir, schema, "fileid0", commitTime, newCommitTime, "101", 1);
logFilePaths.add(writer.getLogFile().getPath().toString());
writer.close();
assertTrue("block - size should be > 0", size > 0);
InputFormatTestUtil.deltaCommit(basePath, newCommitTime);
//create a split with baseFile (parquet file written earlier) and new log file(s)
// create a split with baseFile (parquet file written earlier) and new log file(s)
HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit(
new FileSplit(new Path(partitionDir + "/fileid0_1_" + commitTime + ".parquet"), 0, 1,
jobConf), basePath.getRoot().getPath(), logFilePaths, newCommitTime);
new FileSplit(new Path(partitionDir + "/fileid0_1_" + commitTime + ".parquet"), 0, 1, jobConf),
basePath.getRoot().getPath(), logFilePaths, newCommitTime);
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader =
new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null),
jobConf, null);
// create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<NullWritable, ArrayWritable> reader = new MapredParquetInputFormat().getRecordReader(
new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()), (String[]) null), jobConf, null);
JobConf jobConf = new JobConf();
List<Schema.Field> fields = schema.getFields();
@@ -537,8 +503,7 @@ public class HoodieRealtimeRecordReaderTest {
// Try to read all the fields passed by the new schema
String names = fields.stream().map(f -> f.name()).collect(Collectors.joining(","));
String positions = fields.stream().map(f -> String.valueOf(f.pos()))
.collect(Collectors.joining(","));
String positions = fields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
jobConf.set("partition_columns", "datestr");
@@ -554,8 +519,7 @@ public class HoodieRealtimeRecordReaderTest {
// Try to read all the fields passed by the new schema
names = firstSchemaFields.stream().map(f -> f.name()).collect(Collectors.joining(","));
positions = firstSchemaFields.stream().map(f -> String.valueOf(f.pos()))
.collect(Collectors.joining(","));
positions = firstSchemaFields.stream().map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
jobConf.set("partition_columns", "datestr");