|
@@ -18,37 +18,22 @@
|
|
|
|
|
|
package org.apache.hadoop.fs.s3a.commit.magic;
|
|
package org.apache.hadoop.fs.s3a.commit.magic;
|
|
|
|
|
|
-import java.io.ByteArrayInputStream;
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
-import java.io.InputStream;
|
|
|
|
-import java.util.HashMap;
|
|
|
|
import java.util.List;
|
|
import java.util.List;
|
|
-import java.util.Map;
|
|
|
|
|
|
|
|
import software.amazon.awssdk.services.s3.model.CompletedPart;
|
|
import software.amazon.awssdk.services.s3.model.CompletedPart;
|
|
-import software.amazon.awssdk.services.s3.model.PutObjectRequest;
|
|
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
|
|
|
|
-import org.apache.commons.lang3.StringUtils;
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
-import org.apache.hadoop.fs.s3a.Retries;
|
|
|
|
-import org.apache.hadoop.fs.s3a.S3ADataBlocks;
|
|
|
|
import org.apache.hadoop.fs.s3a.WriteOperationHelper;
|
|
import org.apache.hadoop.fs.s3a.WriteOperationHelper;
|
|
import org.apache.hadoop.fs.s3a.commit.PutTracker;
|
|
import org.apache.hadoop.fs.s3a.commit.PutTracker;
|
|
-import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit;
|
|
|
|
-import org.apache.hadoop.fs.s3a.impl.PutObjectOptions;
|
|
|
|
import org.apache.hadoop.fs.s3a.statistics.PutTrackerStatistics;
|
|
import org.apache.hadoop.fs.s3a.statistics.PutTrackerStatistics;
|
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
|
-import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
|
|
|
|
-import org.apache.hadoop.util.Preconditions;
|
|
|
|
|
|
|
|
import static java.util.Objects.requireNonNull;
|
|
import static java.util.Objects.requireNonNull;
|
|
-import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MAGIC_MARKER_PUT;
|
|
|
|
-import static org.apache.hadoop.fs.s3a.commit.CommitConstants.X_HEADER_MAGIC_MARKER;
|
|
|
|
-import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation;
|
|
|
|
|
|
|
|
/**
|
|
/**
|
|
* Put tracker for Magic commits.
|
|
* Put tracker for Magic commits.
|
|
@@ -56,7 +41,7 @@ import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDura
|
|
* uses any datatype in hadoop-mapreduce.
|
|
* uses any datatype in hadoop-mapreduce.
|
|
*/
|
|
*/
|
|
@InterfaceAudience.Private
|
|
@InterfaceAudience.Private
|
|
-public class MagicCommitTracker extends PutTracker {
|
|
|
|
|
|
+public abstract class MagicCommitTracker extends PutTracker {
|
|
public static final Logger LOG = LoggerFactory.getLogger(
|
|
public static final Logger LOG = LoggerFactory.getLogger(
|
|
MagicCommitTracker.class);
|
|
MagicCommitTracker.class);
|
|
|
|
|
|
@@ -65,7 +50,7 @@ public class MagicCommitTracker extends PutTracker {
|
|
private final Path path;
|
|
private final Path path;
|
|
private final WriteOperationHelper writer;
|
|
private final WriteOperationHelper writer;
|
|
private final String bucket;
|
|
private final String bucket;
|
|
- private static final byte[] EMPTY = new byte[0];
|
|
|
|
|
|
+ protected static final byte[] EMPTY = new byte[0];
|
|
private final PutTrackerStatistics trackerStatistics;
|
|
private final PutTrackerStatistics trackerStatistics;
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -127,68 +112,11 @@ public class MagicCommitTracker extends PutTracker {
|
|
* @throws IllegalArgumentException bad argument
|
|
* @throws IllegalArgumentException bad argument
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
- public boolean aboutToComplete(String uploadId,
|
|
|
|
|
|
+ public abstract boolean aboutToComplete(String uploadId,
|
|
List<CompletedPart> parts,
|
|
List<CompletedPart> parts,
|
|
long bytesWritten,
|
|
long bytesWritten,
|
|
- final IOStatistics iostatistics)
|
|
|
|
- throws IOException {
|
|
|
|
- Preconditions.checkArgument(StringUtils.isNotEmpty(uploadId),
|
|
|
|
- "empty/null upload ID: "+ uploadId);
|
|
|
|
- Preconditions.checkArgument(parts != null,
|
|
|
|
- "No uploaded parts list");
|
|
|
|
- Preconditions.checkArgument(!parts.isEmpty(),
|
|
|
|
- "No uploaded parts to save");
|
|
|
|
-
|
|
|
|
- // put a 0-byte file with the name of the original under-magic path
|
|
|
|
- // Add the final file length as a header
|
|
|
|
- // this is done before the task commit, so its duration can be
|
|
|
|
- // included in the statistics
|
|
|
|
- Map<String, String> headers = new HashMap<>();
|
|
|
|
- headers.put(X_HEADER_MAGIC_MARKER, Long.toString(bytesWritten));
|
|
|
|
- PutObjectRequest originalDestPut = writer.createPutObjectRequest(
|
|
|
|
- originalDestKey,
|
|
|
|
- 0,
|
|
|
|
- new PutObjectOptions(true, null, headers), false);
|
|
|
|
- upload(originalDestPut, new ByteArrayInputStream(EMPTY));
|
|
|
|
-
|
|
|
|
- // build the commit summary
|
|
|
|
- SinglePendingCommit commitData = new SinglePendingCommit();
|
|
|
|
- commitData.touch(System.currentTimeMillis());
|
|
|
|
- commitData.setDestinationKey(getDestKey());
|
|
|
|
- commitData.setBucket(bucket);
|
|
|
|
- commitData.setUri(path.toUri().toString());
|
|
|
|
- commitData.setUploadId(uploadId);
|
|
|
|
- commitData.setText("");
|
|
|
|
- commitData.setLength(bytesWritten);
|
|
|
|
- commitData.bindCommitData(parts);
|
|
|
|
- commitData.setIOStatistics(
|
|
|
|
- new IOStatisticsSnapshot(iostatistics));
|
|
|
|
-
|
|
|
|
- byte[] bytes = commitData.toBytes(SinglePendingCommit.serializer());
|
|
|
|
- LOG.info("Uncommitted data pending to file {};"
|
|
|
|
- + " commit metadata for {} parts in {}. size: {} byte(s)",
|
|
|
|
- path.toUri(), parts.size(), pendingPartKey, bytesWritten);
|
|
|
|
- LOG.debug("Closed MPU to {}, saved commit information to {}; data=:\n{}",
|
|
|
|
- path, pendingPartKey, commitData);
|
|
|
|
- PutObjectRequest put = writer.createPutObjectRequest(
|
|
|
|
- pendingPartKey,
|
|
|
|
- bytes.length, null, false);
|
|
|
|
- upload(put, new ByteArrayInputStream(bytes));
|
|
|
|
- return false;
|
|
|
|
-
|
|
|
|
- }
|
|
|
|
- /**
|
|
|
|
- * PUT an object.
|
|
|
|
- * @param request the request
|
|
|
|
- * @param inputStream input stream of data to be uploaded
|
|
|
|
- * @throws IOException on problems
|
|
|
|
- */
|
|
|
|
- @Retries.RetryTranslated
|
|
|
|
- private void upload(PutObjectRequest request, InputStream inputStream) throws IOException {
|
|
|
|
- trackDurationOfInvocation(trackerStatistics, COMMITTER_MAGIC_MARKER_PUT.getSymbol(),
|
|
|
|
- () -> writer.putObject(request, PutObjectOptions.keepingDirs(),
|
|
|
|
- new S3ADataBlocks.BlockUploadData(inputStream), false, null));
|
|
|
|
- }
|
|
|
|
|
|
+ IOStatistics iostatistics)
|
|
|
|
+ throws IOException;
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public String toString() {
|
|
public String toString() {
|
|
@@ -201,4 +129,28 @@ public class MagicCommitTracker extends PutTracker {
|
|
sb.append('}');
|
|
sb.append('}');
|
|
return sb.toString();
|
|
return sb.toString();
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ public String getOriginalDestKey() {
|
|
|
|
+ return originalDestKey;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public String getPendingPartKey() {
|
|
|
|
+ return pendingPartKey;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public Path getPath() {
|
|
|
|
+ return path;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public String getBucket() {
|
|
|
|
+ return bucket;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public WriteOperationHelper getWriter() {
|
|
|
|
+ return writer;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public PutTrackerStatistics getTrackerStatistics() {
|
|
|
|
+ return trackerStatistics;
|
|
|
|
+ }
|
|
}
|
|
}
|