Skip to content

Commit

Permalink
[FLINK-36880][network] Hybrid shuffle supports job master failover if…
Browse files Browse the repository at this point in the history
… only external tier used.
  • Loading branch information
reswqa committed Dec 23, 2024
1 parent 8c6aa0b commit c166cc3
Show file tree
Hide file tree
Showing 19 changed files with 906 additions and 43 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,9 @@
package org.apache.flink.runtime.executiongraph;

import java.io.Serializable;
import java.util.List;

import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;

/** This class represents a snapshot of the result partition bytes metrics. */
Expand All @@ -34,4 +36,23 @@ public ResultPartitionBytes(long[] subpartitionBytes) {
public long[] getSubpartitionBytes() {
return subpartitionBytes;
}

/** Merge all {@link ResultPartitionBytes} by sum up them per-subpartition. */
public static ResultPartitionBytes mergeAll(List<ResultPartitionBytes> partitions) {
checkArgument(!partitions.isEmpty());
int expectedLength = partitions.get(0).getSubpartitionBytes().length;
for (ResultPartitionBytes resultPartitionByte : partitions) {
if (resultPartitionByte.getSubpartitionBytes().length != expectedLength) {
throw new IllegalArgumentException(
"only all ResultPartitionBytes with the same length can be merged");
}
}
long[] mergedSubpartitionBytes = new long[expectedLength];
for (int i = 0; i < expectedLength; i++) {
for (ResultPartitionBytes resultPartitionByte : partitions) {
mergedSubpartitionBytes[i] += resultPartitionByte.getSubpartitionBytes()[i];
}
}
return new ResultPartitionBytes(mergedSubpartitionBytes);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
/*
* 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.hybrid.tiered.shuffle;

import org.apache.flink.api.java.tuple.Tuple2;

import java.io.Serializable;
import java.util.Collection;
import java.util.Collections;

/**
* This is a collection of all {@link TieredShuffleMasterSnapshot}s from every tier in one snapshot
* round.
*/
public class AllTieredShuffleMasterSnapshots implements Serializable {
/**
* Snapshots of all tires. For each tier, it is a tuple of
* (identifier,TieredShuffleMasterSnapshot)
*/
private final Collection<Tuple2<String, TieredShuffleMasterSnapshot>> snapshots;

public AllTieredShuffleMasterSnapshots(
Collection<Tuple2<String, TieredShuffleMasterSnapshot>> snapshots) {
this.snapshots = snapshots;
}

public Collection<Tuple2<String, TieredShuffleMasterSnapshot>> getSnapshots() {
return snapshots;
}

public static AllTieredShuffleMasterSnapshots empty() {
return new AllTieredShuffleMasterSnapshots(Collections.emptyList());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* 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.hybrid.tiered.shuffle;

/**
* A singleton implementation of {@link TieredShuffleMasterSnapshot} that represents an empty
* snapshot of tiered shuffle master.
*/
public class EmptyTieredShuffleMasterSnapshot implements TieredShuffleMasterSnapshot {
private static final EmptyTieredShuffleMasterSnapshot INSTANCE =
new EmptyTieredShuffleMasterSnapshot();

public static EmptyTieredShuffleMasterSnapshot getInstance() {
return INSTANCE;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* 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.hybrid.tiered.shuffle;

import org.apache.flink.api.common.JobID;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.shuffle.ShuffleDescriptor;

import java.util.Optional;

/** The retriever for shuffle descriptor. */
public interface ShuffleDescriptorRetriever {
/**
* Get shuffle descriptor by JobID and ResultPartitionId.
*
* @return shuffle descriptor or empty if not exist.
*/
Optional<ShuffleDescriptor> getShuffleDescriptor(
JobID jobID, ResultPartitionID resultPartitionID);
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,9 @@
package org.apache.flink.runtime.io.network.partition.hybrid.tiered.shuffle;

import org.apache.flink.api.common.JobID;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageConfiguration;
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.common.TieredStorageIdMappingUtils;
Expand All @@ -30,11 +32,16 @@
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierShuffleDescriptor;
import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierShuffleHandler;
import org.apache.flink.runtime.shuffle.JobShuffleContext;
import org.apache.flink.runtime.shuffle.PartitionWithMetrics;
import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
import org.apache.flink.runtime.shuffle.ShuffleMasterContext;
import org.apache.flink.runtime.shuffle.ShuffleMasterSnapshotContext;

import java.time.Duration;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.stream.Collectors;

Expand All @@ -48,17 +55,79 @@ public class TieredInternalShuffleMaster {

private final ShuffleMasterContext shuffleMasterContext;

public TieredInternalShuffleMaster(ShuffleMasterContext shuffleMasterContext) {
private final boolean useExternalTier;

public TieredInternalShuffleMaster(
ShuffleMasterContext shuffleMasterContext,
ShuffleDescriptorRetriever shuffleDescriptorRetriever) {
this.shuffleMasterContext = shuffleMasterContext;
Configuration conf = shuffleMasterContext.getConfiguration();
String externalTierFactoryClass =
conf.get(
NettyShuffleEnvironmentOptions
.NETWORK_HYBRID_SHUFFLE_EXTERNAL_REMOTE_TIER_FACTORY_CLASS_NAME);
this.useExternalTier = externalTierFactoryClass != null;
TieredStorageConfiguration tieredStorageConfiguration =
TieredStorageConfiguration.fromConfiguration(conf);
TieredStorageResourceRegistry resourceRegistry = new TieredStorageResourceRegistry();
List<TierMasterAgent> tierFactories =
List<Tuple2<String, TierMasterAgent>> tierFactories =
tieredStorageConfiguration.getTierFactories().stream()
.map(tierFactory -> tierFactory.createMasterAgent(resourceRegistry))
.map(
tierFactory ->
Tuple2.of(
tierFactory.identifier(),
tierFactory.createMasterAgent(resourceRegistry)))
.collect(Collectors.toList());
this.tieredStorageMasterClient = new TieredStorageMasterClient(tierFactories);
this.tieredStorageMasterClient =
new TieredStorageMasterClient(tierFactories, shuffleDescriptorRetriever);
}

public boolean supportsBatchSnapshot() {
return useExternalTier;
}

public void snapshotState(
CompletableFuture<AllTieredShuffleMasterSnapshots> snapshotFuture,
ShuffleMasterSnapshotContext context,
JobID jobId) {
// only external tier supports snapshot for now.
if (useExternalTier) {
tieredStorageMasterClient.snapshotState(snapshotFuture, context, jobId);
} else {
snapshotFuture.complete(AllTieredShuffleMasterSnapshots.empty());
}
}

public void snapshotState(CompletableFuture<AllTieredShuffleMasterSnapshots> snapshotFuture) {
if (useExternalTier) {
tieredStorageMasterClient.snapshotState(snapshotFuture);
} else {
snapshotFuture.complete(AllTieredShuffleMasterSnapshots.empty());
}
}

public void restoreState(List<TieredInternalShuffleMasterSnapshot> snapshots, JobID jobId) {
if (useExternalTier) {
tieredStorageMasterClient.restoreState(snapshots, jobId);
}
}

public void restoreState(TieredInternalShuffleMasterSnapshot clusterSnapshot) {
if (useExternalTier) {
tieredStorageMasterClient.restoreState(clusterSnapshot);
}
}

public CompletableFuture<Collection<PartitionWithMetrics>> getPartitionWithMetrics(
JobShuffleContext jobShuffleContext,
Duration timeout,
Set<ResultPartitionID> expectedPartitions) {
if (useExternalTier) {
return tieredStorageMasterClient.getPartitionWithMetrics(
jobShuffleContext, timeout, expectedPartitions);
} else {
return CompletableFuture.completedFuture(Collections.emptyList());
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* 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.hybrid.tiered.shuffle;

import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.shuffle.ShuffleDescriptor;
import org.apache.flink.runtime.shuffle.ShuffleMasterSnapshot;

import java.util.Map;

/**
* The internal {@link ShuffleMasterSnapshot} for hybrid shuffle. This bump shuffle descriptors and
* all tiers snapshot.
*/
public class TieredInternalShuffleMasterSnapshot implements ShuffleMasterSnapshot {
private final Map<ResultPartitionID, ShuffleDescriptor> shuffleDescriptors;

private final AllTieredShuffleMasterSnapshots allTierSnapshots;

public TieredInternalShuffleMasterSnapshot(
Map<ResultPartitionID, ShuffleDescriptor> shuffleDescriptors,
AllTieredShuffleMasterSnapshots allTierSnapshots) {
this.shuffleDescriptors = shuffleDescriptors;
this.allTierSnapshots = allTierSnapshots;
}

public Map<ResultPartitionID, ShuffleDescriptor> getShuffleDescriptors() {
return shuffleDescriptors;
}

public AllTieredShuffleMasterSnapshots getAllTierSnapshots() {
return allTierSnapshots;
}

@Override
public boolean isIncremental() {
return true;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
/*
* 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.hybrid.tiered.shuffle;

import org.apache.flink.runtime.io.network.partition.hybrid.tiered.tier.TierShuffleDescriptor;
import org.apache.flink.runtime.shuffle.ShuffleMetrics;

import static org.apache.flink.util.Preconditions.checkNotNull;

/** Partition with shuffle metrics for tiered storage. */
public class TieredPartitionWithMetrics {
private final TierShuffleDescriptor shuffleDescriptor;
private final ShuffleMetrics partitionMetrics;

public TieredPartitionWithMetrics(
TierShuffleDescriptor shuffleDescriptor, ShuffleMetrics partitionMetrics) {
this.shuffleDescriptor = checkNotNull(shuffleDescriptor);
this.partitionMetrics = checkNotNull(partitionMetrics);
}

public ShuffleMetrics getPartitionMetrics() {
return partitionMetrics;
}

public TierShuffleDescriptor getPartition() {
return shuffleDescriptor;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* 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.hybrid.tiered.shuffle;

import java.io.Serializable;

/**
* This class represents a snapshot of tiered shuffle master, which can be used to restore the
* internal state of the shuffle master.
*
* <p>IMPORTANT: It is incremental.
*/
public interface TieredShuffleMasterSnapshot extends Serializable {}
Loading

0 comments on commit c166cc3

Please sign in to comment.