Nicer handling of timeline archival for Cloud storage
- When append() is not supported, rollover to new file always (instead of failing) - Provide way to configure archive log folder (avoids small files inside .hoodie) - Datasets written via Spark datasource archive to .hoodie/archived - HoodieClientExample will now retain only 2,3 commits to exercise archival path during dev cycles - Few tweaks to code structure around CommitArchiveLog
This commit is contained in:
committed by
vinoth chandar
parent
0cd186c899
commit
cf7f7aabb9
@@ -107,7 +107,7 @@ public class HoodieTableMetaClientTest {
|
||||
|
||||
@Test
|
||||
public void checkArchiveCommitTimeline() throws IOException {
|
||||
Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getMetaPath());
|
||||
Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath());
|
||||
SequenceFile.Writer writer = SequenceFile
|
||||
.createWriter(HoodieTestUtils.fs.getConf(), SequenceFile.Writer.file(archiveLogPath),
|
||||
SequenceFile.Writer.keyClass(Text.class),
|
||||
|
||||
@@ -24,6 +24,7 @@ import static org.junit.Assert.fail;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.uber.hoodie.common.minicluster.MiniClusterUtil;
|
||||
import com.uber.hoodie.common.model.HoodieArchivedLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
@@ -53,6 +54,7 @@ import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.After;
|
||||
@@ -241,6 +243,34 @@ public class HoodieLogFormatTest {
|
||||
writer.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAppendNotSupported()
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
// Use some fs like LocalFileSystem, that does not support appends
|
||||
Path localPartitionPath = new Path("file://" + partitionPath);
|
||||
FileSystem localFs = FSUtils
|
||||
.getFs(localPartitionPath.toString(), HoodieTestUtils.getDefaultHadoopConf());
|
||||
Path testPath = new Path(localPartitionPath, "append_test");
|
||||
localFs.mkdirs(testPath);
|
||||
|
||||
// Some data & append two times.
|
||||
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
|
||||
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
|
||||
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, "100");
|
||||
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records,
|
||||
getSimpleSchema(), metadata);
|
||||
|
||||
for (int i = 0; i < 2; i++) {
|
||||
HoodieLogFormat.newWriterBuilder().onParentPath(testPath)
|
||||
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFileId("commits.archive")
|
||||
.overBaseCommit("").withFs(localFs).build().appendBlock(dataBlock).close();
|
||||
}
|
||||
|
||||
// ensure there are two log file versions, with same data.
|
||||
FileStatus[] statuses = localFs.listStatus(testPath);
|
||||
assertEquals(2, statuses.length);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
public void testBasicWriteAndScan()
|
||||
|
||||
Reference in New Issue
Block a user