1
0
Files
hudi/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieInsertHandle.java
Prasanna Rajaperumal 8ee777a9bb Refactor hoodie-common and create right abstractions for Hoodie Storage V2.0
The following is the gist of changes done

- All low-level operation of creating a commit code was in HoodieClient which made it hard to share code if there was a compaction commit.
- HoodieTableMetadata contained a mix of metadata and filtering files. (Also few operations required FileSystem to be passed in because those were called from TaskExecutors and others had FileSystem as a global variable). Since merge-on-read requires a lot of that code, but will have to change slightly on how it operates on the metadata and how it filters the files. The two set of operation are split into HoodieTableMetaClient and TableFileSystemView.
- Everything (active commits, archived commits, cleaner log, save point log and in future delta and compaction commits) in HoodieTableMetaClient is a HoodieTimeline. Timeline is a series of instants, which has an in-built concept of inflight and completed commit markers.
- A timeline can be queries for ranges, contains and also use to create new datapoint (create a new commit etc). Commit (and all the above metadata) creation/deletion is streamlined in a timeline
- Multiple timelines can be merged into a single timeline, giving us an audit timeline to whatever happened in a hoodie dataset. This also helps with #55.
- Move to java 8 and introduce java 8 succinct syntax in refactored code
2017-02-21 16:23:53 -08:00

128 lines
4.9 KiB
Java

/*
* Copyright (c) 2016 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 com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieInsertException;
import com.uber.hoodie.io.storage.HoodieStorageWriter;
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.TaskContext;
import java.io.IOException;
import java.util.UUID;
public class HoodieInsertHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
private static Logger logger = LogManager.getLogger(HoodieInsertHandle.class);
private final WriteStatus status;
private final HoodieStorageWriter<IndexedRecord> storageWriter;
private final Path path;
private int recordsWritten = 0;
public HoodieInsertHandle(HoodieWriteConfig config, String commitTime,
HoodieTableMetaClient metadata, String partitionPath) {
super(config, commitTime, metadata);
this.status = new WriteStatus();
status.setFileId(UUID.randomUUID().toString());
status.setPartitionPath(partitionPath);
this.path = makeNewPath(partitionPath, TaskContext.getPartitionId(), status.getFileId());
try {
this.storageWriter =
HoodieStorageWriterFactory.getStorageWriter(commitTime, path, metadata, config, schema);
} catch (IOException e) {
throw new HoodieInsertException(
"Failed to initialize HoodieStorageWriter for path " + path, e);
}
logger.info("New InsertHandle for partition :" + partitionPath);
}
/**
* Determines whether we can accept the incoming records, into the current file, depending on
*
* - Whether it belongs to the same partitionPath as existing records
* - Whether the current file written bytes lt max file size
*
* @return
*/
public boolean canWrite(HoodieRecord record) {
return storageWriter.canWrite() && record.getPartitionPath()
.equals(status.getPartitionPath());
}
/**
* Perform the actual writing of the given record into the backing file.
*
* @param record
*/
public void write(HoodieRecord record) {
try {
IndexedRecord avroRecord = record.getData().getInsertValue(schema);
storageWriter.writeAvroWithMetadata(avroRecord, record);
status.markSuccess(record);
// update the new location of record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, status.getFileId()));
record.deflate();
recordsWritten++;
} catch (Throwable t) {
// Not throwing exception from here, since we don't want to fail the entire job
// for a single record
status.markFailure(record, t);
logger.error("Error writing record " + record, t);
}
}
/**
* Performs actions to durably, persist the current changes and returns a WriteStatus object
*
* @return
*/
public WriteStatus close() {
logger.info(
"Closing the file " + status.getFileId() + " as we are done with all the records "
+ recordsWritten);
try {
storageWriter.close();
HoodieWriteStat stat = new HoodieWriteStat();
stat.setNumWrites(recordsWritten);
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
stat.setFileId(status.getFileId());
stat.setFullPath(path.toString());
stat.setTotalWriteBytes(FSUtils.getFileSize(fs, path));
stat.setTotalWriteErrors(status.getFailedRecords().size());
status.setStat(stat);
return status;
} catch (IOException e) {
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path,
e);
}
}
}