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
@@ -52,11 +52,13 @@ public class HoodieTableConfig implements Serializable {
|
||||
public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME =
|
||||
"hoodie.table.rt.file.format";
|
||||
public static final String HOODIE_PAYLOAD_CLASS_PROP_NAME = "hoodie.compaction.payload.class";
|
||||
public static final String HOODIE_ARCHIVELOG_FOLDER_PROP_NAME = "hoodie.archivelog.folder";
|
||||
|
||||
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
|
||||
public static final HoodieFileFormat DEFAULT_RO_FILE_FORMAT = HoodieFileFormat.PARQUET;
|
||||
public static final HoodieFileFormat DEFAULT_RT_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG;
|
||||
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
|
||||
public static final String DEFAULT_ARCHIVELOG_FOLDER = "";
|
||||
private Properties props;
|
||||
|
||||
public HoodieTableConfig(FileSystem fs, String metaPath) {
|
||||
@@ -105,6 +107,9 @@ public class HoodieTableConfig implements Serializable {
|
||||
&& !properties.containsKey(HOODIE_PAYLOAD_CLASS_PROP_NAME)) {
|
||||
properties.setProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS);
|
||||
}
|
||||
if (!properties.containsKey(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME)) {
|
||||
properties.setProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER);
|
||||
}
|
||||
properties
|
||||
.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
|
||||
} finally {
|
||||
@@ -161,4 +166,10 @@ public class HoodieTableConfig implements Serializable {
|
||||
return DEFAULT_RT_FILE_FORMAT;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the relative path of archive log folder under metafolder, for this dataset
|
||||
*/
|
||||
public String getArchivelogFolder() {
|
||||
return props.getProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -131,6 +131,18 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
return metaPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return path where archived timeline is stored
|
||||
*/
|
||||
public String getArchivePath() {
|
||||
String archiveFolder = tableConfig.getArchivelogFolder();
|
||||
if (archiveFolder.equals(HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER)) {
|
||||
return getMetaPath();
|
||||
} else {
|
||||
return getMetaPath() + "/" + archiveFolder;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @return Table Config
|
||||
*/
|
||||
@@ -208,6 +220,18 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
if (!fs.exists(metaPathDir)) {
|
||||
fs.mkdirs(metaPathDir);
|
||||
}
|
||||
|
||||
// if anything other than default archive log folder is specified, create that too
|
||||
String archiveLogPropVal = props
|
||||
.getProperty(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME,
|
||||
HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER);
|
||||
if (!archiveLogPropVal.equals(HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER)) {
|
||||
Path archiveLogDir = new Path(metaPathDir, archiveLogPropVal);
|
||||
if (!fs.exists(archiveLogDir)) {
|
||||
fs.mkdirs(archiveLogDir);
|
||||
}
|
||||
}
|
||||
|
||||
HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath);
|
||||
log.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType()
|
||||
|
||||
@@ -40,7 +40,7 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
|
||||
private final static Logger log = LogManager.getLogger(HoodieLogFormatWriter.class);
|
||||
|
||||
private final HoodieLogFile logFile;
|
||||
private HoodieLogFile logFile;
|
||||
private final FileSystem fs;
|
||||
private final long sizeThreshold;
|
||||
private final Integer bufferSize;
|
||||
@@ -83,6 +83,15 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
if (ioe.getMessage().equalsIgnoreCase("Not supported")) {
|
||||
log.info("Append not supported. Opening a new log file..");
|
||||
this.logFile = logFile.rollOver(fs);
|
||||
this.output = fs.create(this.logFile.getPath(), false, bufferSize, replication,
|
||||
WriterBuilder.DEFAULT_SIZE_THRESHOLD, null);
|
||||
} else {
|
||||
throw ioe;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
log.info(logFile + " does not exist. Create a new file");
|
||||
|
||||
@@ -50,7 +50,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
|
||||
|
||||
public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) {
|
||||
// Read back the commits to make sure
|
||||
Path archiveLogPath = getArchiveLogPath(metaClient.getMetaPath());
|
||||
Path archiveLogPath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath());
|
||||
try (SequenceFile.Reader reader =
|
||||
new SequenceFile.Reader(metaClient.getHadoopConf(),
|
||||
SequenceFile.Reader.file(archiveLogPath))) {
|
||||
@@ -92,8 +92,8 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
|
||||
|
||||
public static Path getArchiveLogPath(String metaPath) {
|
||||
return new Path(metaPath, HOODIE_COMMIT_ARCHIVE_LOG_FILE);
|
||||
public static Path getArchiveLogPath(String archiveFolder) {
|
||||
return new Path(archiveFolder, HOODIE_COMMIT_ARCHIVE_LOG_FILE);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -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