jerqi commented on code in PR #947: URL: https://github.com/apache/incubator-uniffle/pull/947#discussion_r1236190686
########## client-tez/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/RssShuffleManager.java: ########## @@ -0,0 +1,1236 @@ +/** + * 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.tez.runtime.library.common.shuffle.impl; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.text.DecimalFormat; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.BitSet; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RawLocalFileSystem; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.util.Time; +import org.apache.tez.common.CallableWithNdc; +import org.apache.tez.common.InputContextUtils; +import org.apache.tez.common.TezRuntimeFrameworkConfigs; +import org.apache.tez.common.TezUtilsInternal; +import org.apache.tez.common.UmbilicalUtils; +import org.apache.tez.common.counters.TaskCounter; +import org.apache.tez.common.counters.TezCounter; +import org.apache.tez.dag.api.TezConfiguration; +import org.apache.tez.dag.api.TezUncheckedException; +import org.apache.tez.dag.records.TezTaskAttemptID; +import org.apache.tez.http.HttpConnectionParams; +import org.apache.tez.runtime.api.Event; +import org.apache.tez.runtime.api.InputContext; +import org.apache.tez.runtime.api.TaskFailureType; +import org.apache.tez.runtime.api.events.InputReadErrorEvent; +import org.apache.tez.runtime.library.api.TezRuntimeConfiguration; +import org.apache.tez.runtime.library.common.CompositeInputAttemptIdentifier; +import org.apache.tez.runtime.library.common.InputAttemptIdentifier; +import org.apache.tez.runtime.library.common.TezRuntimeUtils; +import org.apache.tez.runtime.library.common.shuffle.FetchResult; +import org.apache.tez.runtime.library.common.shuffle.FetchedInput; +import org.apache.tez.runtime.library.common.shuffle.FetchedInputAllocator; +import org.apache.tez.runtime.library.common.shuffle.Fetcher; +import org.apache.tez.runtime.library.common.shuffle.Fetcher.FetcherBuilder; +import org.apache.tez.runtime.library.common.shuffle.HostPort; +import org.apache.tez.runtime.library.common.shuffle.InputHost; +import org.apache.tez.runtime.library.common.shuffle.InputHost.PartitionToInputs; +import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils; +import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils.FetchStatsLogger; +import org.roaringbitmap.longlong.Roaring64NavigableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.uniffle.common.ShuffleServerInfo; + +// This only knows how to deal with a single srcIndex for a given targetIndex. +// In case the src task generates multiple outputs for the same target Index +// (multiple src-indices), modifications will be required. +public class RssShuffleManager extends ShuffleManager { + + private static final Logger LOG = LoggerFactory.getLogger(RssShuffleManager.class); + private static final Logger LOG_FETCH = LoggerFactory.getLogger(LOG.getName() + ".fetch"); + private static final FetchStatsLogger fetchStatsLogger = new FetchStatsLogger(LOG_FETCH, LOG); + + private final InputContext inputContext; + private final int numInputs; + + private final DecimalFormat mbpsFormat = new DecimalFormat("0.00"); + + private final FetchedInputAllocator inputManager; + + @VisibleForTesting + final ListeningExecutorService fetcherExecutor; + + /** + * Executor for ReportCallable. + */ + private ExecutorService reporterExecutor; + + /** + * Lock to sync failedEvents. + */ + private final ReentrantLock reportLock = new ReentrantLock(); + + /** + * Condition to wake up the thread notifying when events fail. + */ + private final Condition reportCondition = reportLock.newCondition(); + + /** + * Events reporting fetcher failed. + */ + private final HashMap<InputReadErrorEvent, Integer> failedEvents = new HashMap<>(); + + private final ListeningExecutorService schedulerExecutor; + private final RssRunShuffleCallable rssSchedulerCallable; + + private final BlockingQueue<FetchedInput> completedInputs; + private final AtomicBoolean inputReadyNotificationSent = new AtomicBoolean(false); + @VisibleForTesting + final BitSet completedInputSet; + private final ConcurrentMap<HostPort, InputHost> knownSrcHosts; + private final BlockingQueue<InputHost> pendingHosts; + private final Set<InputAttemptIdentifier> obsoletedInputs; + private Set<RssTezFetcherTask> rssRunningFetchers; + + private final AtomicInteger numCompletedInputs = new AtomicInteger(0); + private final AtomicInteger numFetchedSpills = new AtomicInteger(0); + + private final long startTime; + private long lastProgressTime; + private long totalBytesShuffledTillNow; + + // Required to be held when manipulating pendingHosts + private final ReentrantLock lock = new ReentrantLock(); + private final Condition wakeLoop = lock.newCondition(); + + private final int numFetchers; + private final boolean asyncHttp; + + // Parameters required by Fetchers + private final CompressionCodec codec; + private final Configuration conf; + private final boolean localDiskFetchEnabled; + private final boolean sharedFetchEnabled; + private final boolean verifyDiskChecksum; + private final boolean compositeFetch; + + private final int ifileBufferSize; + private final boolean ifileReadAhead; + private final int ifileReadAheadLength; + + /** + * Holds the time to wait for failures to batch them and send less events. + */ + private final int maxTimeToWaitForReportMillis; + + private final String srcNameTrimmed; + + private final int maxTaskOutputAtOnce; + + private final AtomicBoolean isShutdown = new AtomicBoolean(false); + + private final TezCounter shuffledInputsCounter; + private final TezCounter failedShufflesCounter; + private final TezCounter bytesShuffledCounter; + private final TezCounter decompressedDataSizeCounter; + private final TezCounter bytesShuffledToDiskCounter; + private final TezCounter bytesShuffledToMemCounter; + private final TezCounter bytesShuffledDirectDiskCounter; + + private volatile Throwable shuffleError; + private final HttpConnectionParams httpConnectionParams; + + private final LocalDirAllocator localDirAllocator; + private final RawLocalFileSystem localFs; + private final Path[] localDisks; + private final String localhostName; + private final int shufflePort; + + private final TezCounter shufflePhaseTime; + private final TezCounter firstEventReceived; + private final TezCounter lastEventReceived; + + //To track shuffleInfo events when finalMerge is disabled OR pipelined shuffle is enabled in source. + @VisibleForTesting + final Map<Integer, ShuffleEventInfo> shuffleInfoEventsMap; + + private Map<Integer, List<ShuffleServerInfo>> partitionToServers; + private final Set<Integer> successRssPartitionSet = new HashSet<>(); + private final Set<Integer> runningRssPartitionMap = new HashSet<>(); + private final Set<Integer> allRssPartition = Sets.newConcurrentHashSet(); + private final BlockingQueue<Integer> pendingPartition = new LinkedBlockingQueue<>(); + Map<Integer, List<InputAttemptIdentifier>> partitionToInput = new HashMap<>(); + private final Map<Integer, Roaring64NavigableMap> rssAllBlockIdBitmapMap = new ConcurrentHashMap<>(); + private final Map<Integer, Roaring64NavigableMap> rssSuccessBlockIdBitmapMap = new ConcurrentHashMap<>(); + private final AtomicInteger numNoDataInput = new AtomicInteger(0); + private final AtomicInteger numWithDataInput = new AtomicInteger(0); + + @SuppressFBWarnings("NP_NULL_ON_SOME_PATH_FROM_RETURN_VALUE") Review Comment: Could we solve this issue instead of ignoring it? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
