org.apache.flink.runtime.io.network.partition.external.ExternalBlockResultPartitionMeta Maven / Gradle / Ivy
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.flink.runtime.io.network.partition.external;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.core.fs.FSDataInputStream;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataInputViewStreamWrapper;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.annotation.concurrent.GuardedBy;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkState;
class ExternalBlockResultPartitionMeta {
private static final Logger LOG = LoggerFactory.getLogger(ExternalBlockResultPartitionMeta.class);
/** Supported version of result partition file in case of incompatible partition file. */
public static final int SUPPORTED_PROTOCOL_VERSION = 1;
private final ResultPartitionID resultPartitionID;
/** File system of data and index file for this partition */
private final FileSystem fileSystem;
private final OsCachePolicy osCachePolicy;
private final long maxReadAheadLength;
/** Make sure result partition meta will be initialized only once. */
@GuardedBy("this")
private volatile boolean hasInitialized = false;
private PersistentFileType externalFileType = PersistentFileType.UNDEFINED;
private final LocalResultPartitionResolver.ResultPartitionFileInfo fileInfo;
/**
* Spill count actually shows the count of logical partitioned files.
* We can treat a logical partitioned file as a logical file that contains
* data from each subpartition. Each subpartition in a logical file should
* be continuously stored.
*
* In SINGLE_SUBPARTITION_FILE there will be only one logical partitioned file.
* So its spill count is one.
*
*
In MULTI_SUBPARTITION_FILE, each spill phase will generate a logical
* partitioned file and may be multiple spill files left for shuffle service
* to do the final concatenation for each subpartition.
*/
private int spillCount = 1;
private int subpartitionNum = 0;
/**
* Whole partition indices list, each {@code PartitionIndices} represent one logical partitioned file.
* The size of {@code partitionIndicesList} should be equal to {@code spillCount}.
*/
private List partitionIndicesList;
/** How many subpartitions left unconsumed */
private final AtomicInteger unconsumedSubpartitionCount = new AtomicInteger(Integer.MAX_VALUE);
/** How many subpartition views alive */
private final AtomicInteger refCount = new AtomicInteger(0);
private final AtomicReference lastActiveTimeInMs = new AtomicReference<>(-1L);
ExternalBlockResultPartitionMeta(
ResultPartitionID resultPartitionID,
FileSystem fileSystem,
LocalResultPartitionResolver.ResultPartitionFileInfo fileInfo,
OsCachePolicy osCachePolicy,
long maxReadAheadLength) {
this.resultPartitionID = resultPartitionID;
this.fileSystem = fileSystem;
this.fileInfo = fileInfo;
this.osCachePolicy = osCachePolicy;
this.maxReadAheadLength = maxReadAheadLength;
}
boolean hasInitialized() {
return hasInitialized;
}
synchronized void initialize() throws IOException {
if (hasInitialized) {
return;
}
// Then read finish file to get external file type
initializeByFinishFile();
// Initialize partition indices
initializeByIndexFile();
// Set reference count according to subpartition number
unconsumedSubpartitionCount.set(subpartitionNum);
lastActiveTimeInMs.set(System.currentTimeMillis());
// As this method should read from storage, make sure it to be done once at most
hasInitialized = true;
}
public String getRootDir() {
return fileInfo.getRootDir();
}
public String getResultPartitionDir() {
return fileInfo.getPartitionDir();
}
public OsCachePolicy getOsCachePolicy() {
return osCachePolicy;
}
public long getMaxReadAheadLength() {
return maxReadAheadLength;
}
public synchronized List getSubpartitionMeta(int subpartitionIndex) throws IOException {
checkState(hasInitialized, "The meta info has not been initialized.");
checkArgument(subpartitionIndex >= 0 && subpartitionIndex < subpartitionNum, "Invalid subpartition index.");
List subpartitionMeta = new ArrayList<>(spillCount);
for (PartitionIndices partitionIndices : partitionIndicesList) {
subpartitionMeta.add(partitionIndices.getSubpartitionMeta(subpartitionIndex));
}
return subpartitionMeta;
}
PersistentFileType getExternalBlockFileType() {
if (hasInitialized()) {
return externalFileType;
} else {
throw new RuntimeException("This method should be called after initialize()");
}
}
long getConsumedPartitionTTL() {
return fileInfo.getConsumedPartitionTTL();
}
long getPartialConsumedPartitionTTL() {
return fileInfo.getPartialConsumedPartitionTTL();
}
void notifySubpartitionStartConsuming(int subpartitionIndex) {
// Increase reference count
lastActiveTimeInMs.set(System.currentTimeMillis());
refCount.addAndGet(1);
}
/**
* Notify one subpartition finishes consuming this result partition.
* @param subpartitionIndex The index of the consumed subpartition.
*/
void notifySubpartitionConsumed(int subpartitionIndex) {
// TODO: Maybe we need a bitmap to judge whether all the subpartitions have been consumed
// since one subpartition may trigger multiple decrements due to downstream retries.
// As a result, its reference will reach zero even if some subpartitions are unconsumed.
// UnconsumedSubpartitionCount can be negative due to the same reason.
// Currently we rely on consumedPartitionTTL to try to avoid such bad cases.
long currTime = System.currentTimeMillis();
if (unconsumedSubpartitionCount.decrementAndGet() < 1) {
if (LOG.isDebugEnabled()) {
LOG.debug("Partition {} 's reference count turn to zero at {}", resultPartitionID, currTime);
}
}
// Decrease reference count.
lastActiveTimeInMs.set(currTime);
refCount.decrementAndGet();
}
int getReferenceCount() {
return refCount.get();
}
int getUnconsumedSubpartitionCount() {
// unconsumedSubpartitionCount can become a negative value due to reduce task rerun
int count = unconsumedSubpartitionCount.get();
return count >= 0 ? count : 0;
}
long getLastActiveTimeInMs() {
return lastActiveTimeInMs.get();
}
@VisibleForTesting
ResultPartitionID getResultPartitionID() {
return resultPartitionID;
}
long getShrinkableMemoryFootprint() {
long shrinkableMemoryFootprint = 0L;
for (PartitionIndices partitionIndices : partitionIndicesList) {
shrinkableMemoryFootprint += partitionIndices.getShrinkableMemoryFootprint();
}
return shrinkableMemoryFootprint;
}
long shrinkMemoryFootprint() {
long shrunkMemoryFootprint = 0L;
for (PartitionIndices partitionIndices : partitionIndicesList) {
shrunkMemoryFootprint += partitionIndices.shrinkMemoryFootprint();
}
return shrunkMemoryFootprint;
}
// -------------------------------- Internal Utilities ------------------------------------
private void initializeByFinishFile() throws IOException {
String finishedPathStr = ExternalBlockShuffleUtils.generateFinishedPath(fileInfo.getPartitionDir());
Path finishFilePath = new Path(finishedPathStr);
if (!fileSystem.exists(finishFilePath)) {
throw new IOException("Finish file doesn't exist, file path: " + finishFilePath.getPath());
}
FSDataInputStream finishIn = null;
try {
finishIn = fileSystem.open(finishFilePath);
if (null == finishIn) {
throw new IOException("Cannot open finish file, file path: " + finishedPathStr);
}
DataInputView finishView = new DataInputViewStreamWrapper(finishIn);
/**
* Content of finish file:
*
* |<- protocol version (4 bytes, an integer) ->|
* |<- length of "Type" field (4 bytes, an integer) ->|
* |<- variable-length based on previous integer (the value of "Type") ->|
* |<- spill count (4 bytes, an integer) ->|
* |<- subpartition number (4 bytes, an integer, since Version_02) ->|
*/
// Read protocol version
int version = finishView.readInt();
if (version > SUPPORTED_PROTOCOL_VERSION) {
throw new RuntimeException("Unsupported External Data version: " + version
+ ", supported version: " + SUPPORTED_PROTOCOL_VERSION);
}
// Read "Type" field
int typeLength = finishView.readInt();
byte[] typeContent = new byte[typeLength];
finishView.read(typeContent);
externalFileType = PersistentFileType.valueOf(new String(typeContent));
if (externalFileType != PersistentFileType.MERGED_PARTITION_FILE
&& externalFileType != PersistentFileType.HASH_PARTITION_FILE) {
throw new RuntimeException("Unknown external file type: " + typeContent);
}
// Read spill count.
spillCount = finishView.readInt();
if (externalFileType == PersistentFileType.HASH_PARTITION_FILE) {
if (spillCount != 1) {
throw new RuntimeException("Spill count [" + spillCount
+ "] should be 1 in HASH_PARTITION_FILE mode.");
}
} else {
// In MULTI_SUBPARTITION_FILE mode, spill count can be zero if there is
// no records in this result partition.
if (spillCount < 0) {
throw new RuntimeException("Spill count [" + spillCount
+ "] should not be negative in MULTI_SUBPARTITION_FILE mode");
}
}
subpartitionNum = finishView.readInt();
if (subpartitionNum <= 0) {
throw new RuntimeException("Subpartition number [" + subpartitionNum + "] should be positive");
}
} finally {
// Close file handler
if (null != finishIn) {
finishIn.close();
}
}
}
private void initializeByIndexFile() throws IOException {
partitionIndicesList = new ArrayList<>(spillCount);
for (int i = 0; i < spillCount; i++) {
PartitionIndices partitionIndices = null;
if (externalFileType == PersistentFileType.MERGED_PARTITION_FILE) {
partitionIndices = new MergedPartitionIndices(
fileSystem, fileInfo.getPartitionDir(), i, subpartitionNum);
} else { // externalFileType == PersistentFileType.HASH_PARTITION_FILE
partitionIndices = new HashPartitionIndices(
fileSystem, fileInfo.getPartitionDir(), subpartitionNum);
}
partitionIndices.initialize();
partitionIndicesList.add(partitionIndices);
}
}
// Utility for debug.
private static String convertSubpartitionMetasToString(
List subpartitionMeta) {
final StringBuilder stringBuilder = new StringBuilder();
stringBuilder.append("subpartition metas' detail: {");
for (ExternalSubpartitionMeta meta : subpartitionMeta) {
stringBuilder.append(meta.toString());
}
stringBuilder.append("}");
return stringBuilder.toString();
}
static final class ExternalSubpartitionMeta {
private final Path dataFile;
private final long offset;
private final long length;
ExternalSubpartitionMeta(
Path dataFile, long offset, long length) {
checkArgument(dataFile != null);
checkArgument(offset >= 0);
checkArgument(length >= 0);
this.dataFile = dataFile;
this.offset = offset;
this.length = length;
}
Path getDataFile() {
return dataFile;
}
long getOffset() {
return offset;
}
long getLength() {
return length;
}
@Override
public String toString() {
return "{ dataFilePath = " + dataFile + ", offset = " + offset
+ ", buffNum = " + length + " }";
}
}
}