Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,47 +18,60 @@
package org.apache.celeborn.plugin.flink;

import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
import org.apache.flink.runtime.shuffle.PartitionDescriptor;
import org.apache.flink.runtime.shuffle.ProducerDescriptor;

import org.apache.celeborn.plugin.flink.utils.FlinkUtils;

public class FlinkResultPartitionInfo {
private final JobID jobID;
private final PartitionDescriptor partitionDescriptor;
private final ProducerDescriptor producerDescriptor;
private final ResultPartitionID resultPartitionId;
private final IntermediateResultPartitionID partitionId;
private final ExecutionAttemptID producerId;

public FlinkResultPartitionInfo(JobID jobID, ResultPartitionID resultPartitionId) {
this.jobID = jobID;
this.resultPartitionId = resultPartitionId;
this.partitionId = resultPartitionId.getPartitionId();
this.producerId = resultPartitionId.getProducerId();
}

public FlinkResultPartitionInfo(
JobID jobID, PartitionDescriptor partitionDescriptor, ProducerDescriptor producerDescriptor) {
this.jobID = jobID;
this.partitionDescriptor = partitionDescriptor;
this.producerDescriptor = producerDescriptor;
this.resultPartitionId =
new ResultPartitionID(
partitionDescriptor.getPartitionId(), producerDescriptor.getProducerExecutionId());
this.partitionId = partitionDescriptor.getPartitionId();
this.producerId = producerDescriptor.getProducerExecutionId();
}

public ResultPartitionID getResultPartitionId() {
return new ResultPartitionID(
partitionDescriptor.getPartitionId(), producerDescriptor.getProducerExecutionId());
return resultPartitionId;
}

public String getShuffleId() {
return FlinkUtils.toShuffleId(jobID, partitionDescriptor.getResultId());
return FlinkUtils.toShuffleId(jobID, partitionId.getIntermediateDataSetID());
}

public int getTaskId() {
return partitionDescriptor.getPartitionId().getPartitionNumber();
return partitionId.getPartitionNumber();
}

public String getAttemptId() {
return FlinkUtils.toAttemptId(producerDescriptor.getProducerExecutionId());
return FlinkUtils.toAttemptId(producerId);
}

@Override
public String toString() {
final StringBuilder sb = new StringBuilder("FlinkResultPartitionInfo{");
sb.append("jobID=").append(jobID);
sb.append(", partitionDescriptor=").append(partitionDescriptor.getPartitionId());
sb.append(", producerDescriptor=").append(producerDescriptor.getProducerExecutionId());
sb.append(", resultPartitionId=").append(resultPartitionId);
sb.append(", partitionId=").append(partitionId);
sb.append(", producerId=").append(producerId);
sb.append('}');
return sb.toString();
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,120 @@
/*
* 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.celeborn.plugin.flink.tiered;

import java.util.List;
import java.util.concurrent.ScheduledExecutorService;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.io.disk.BatchShuffleReadBufferPool;
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStoragePartitionId;
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.netty.TieredStorageNettyService;
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageConsumerSpec;
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageMemoryManager;
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageMemorySpec;
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.storage.TieredStorageResourceRegistry;
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierConsumerAgent;
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierFactory;
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierMasterAgent;
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierProducerAgent;
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierShuffleDescriptor;
import org.apache.flink.runtime.util.ConfigurationParserUtils;

import org.apache.celeborn.common.CelebornConf;
import org.apache.celeborn.plugin.flink.utils.FlinkUtils;

/**
* The factory class of the Celeborn client, used as a tier of flink hybrid shuffle tiered storage.
*/
public class CelebornTierFactory implements TierFactory {

private CelebornConf conf;

/**
* The bytes size of a single buffer, default value is 32KB, it will be set according to the flink
* configuration in {@link CelebornTierFactory#setup}.
*/
private int bufferSizeBytes = -1;

/**
* The max bytes size of a single segment, it will determine how many buffer can save in a single
* segment.
*/
private static int NUM_BYTES_PER_SEGMENT = 8 * 1024 * 1024;

private static final String CELEBORN_TIER_NAME = CelebornTierFactory.class.getSimpleName();

@Override
public void setup(Configuration configuration) {
conf = FlinkUtils.toCelebornConf(configuration);
this.bufferSizeBytes = ConfigurationParserUtils.getPageSize(configuration);
}

@Override
public TieredStorageMemorySpec getMasterAgentMemorySpec() {
return new TieredStorageMemorySpec(getCelebornTierName(), 0);
}

@Override
public TieredStorageMemorySpec getProducerAgentMemorySpec() {
return new TieredStorageMemorySpec(getCelebornTierName(), 1);
}

@Override
public TieredStorageMemorySpec getConsumerAgentMemorySpec() {
return new TieredStorageMemorySpec(getCelebornTierName(), 0);
}

@Override
public TierMasterAgent createMasterAgent(
TieredStorageResourceRegistry tieredStorageResourceRegistry) {
return new CelebornTierMasterAgent(conf);
}

@Override
public TierProducerAgent createProducerAgent(
int numPartitions,
int numSubpartitions,
TieredStoragePartitionId partitionId,
String dataFileBasePath,
boolean isBroadcastOnly,
TieredStorageMemoryManager storageMemoryManager,
TieredStorageNettyService nettyService,
TieredStorageResourceRegistry resourceRegistry,
BatchShuffleReadBufferPool bufferPool,
ScheduledExecutorService ioExecutor,
List<TierShuffleDescriptor> shuffleDescriptors,
int maxRequestedBuffers) {
// TODO impl this in the follow-up PR.
return null;
}

@Override
public TierConsumerAgent createConsumerAgent(
List<TieredStorageConsumerSpec> tieredStorageConsumerSpecs,
List<TierShuffleDescriptor> shuffleDescriptors,
TieredStorageNettyService nettyService) {
// TODO impl this in the follow-up PR.
return null;
}

public static String getCelebornTierName() {
return CELEBORN_TIER_NAME;
}
}
Loading