├── .github └── workflows │ └── maven.yml ├── .gitignore ├── .travis.yml ├── LICENSE ├── NOTICE ├── README.md ├── build └── rss-build-info ├── docs ├── server-data-dispatcher-diagram-1.png ├── server-data-dispatcher-diagram-2.png └── server-high-level-design.md ├── pom.xml └── src ├── main ├── java │ └── com │ │ └── uber │ │ └── rss │ │ ├── RssBuildInfo.java │ │ ├── StreamServer.java │ │ ├── StreamServerConfig.java │ │ ├── clients │ │ ├── BlockingQueueReadClient.java │ │ ├── BlockingShuffleDataReader.java │ │ ├── BlockingSingleServerReadClient.java │ │ ├── BusyStatusSocketClient.java │ │ ├── ClientBase.java │ │ ├── ClientConstants.java │ │ ├── ClientRetryOptions.java │ │ ├── DataBlockSocketReadClient.java │ │ ├── DataBlockSyncWriteClient.java │ │ ├── EofRecordKeyValuePair.java │ │ ├── FailedFetchRecordKeyValuePair.java │ │ ├── HeartbeatSocketClient.java │ │ ├── LazyWriteClient.java │ │ ├── MultiServerAsyncWriteClient.java │ │ ├── MultiServerHeartbeatClient.java │ │ ├── MultiServerReadClient.java │ │ ├── MultiServerSocketReadClient.java │ │ ├── MultiServerSyncWriteClient.java │ │ ├── MultiServerWriteClient.java │ │ ├── NotifyClient.java │ │ ├── PlainShuffleDataSocketReadClient.java │ │ ├── PlainShuffleDataSyncWriteClient.java │ │ ├── PooledShuffleDataSyncWriteClient.java │ │ ├── PooledWriteClientFactory.java │ │ ├── ReadClientDataOptions.java │ │ ├── RecordSocketReadClient.java │ │ ├── RegistryClient.java │ │ ├── ReplicatedReadClient.java │ │ ├── ReplicatedWriteClient.java │ │ ├── RetriableSocketReadClient.java │ │ ├── ServerBusyRetriableWriteClient.java │ │ ├── ServerConnectionCacheUpdateRefresher.java │ │ ├── ServerConnectionRefresher.java │ │ ├── ServerConnectionStringCache.java │ │ ├── ServerConnectionStringResolver.java │ │ ├── ServerIdAwareSocketReadClient.java │ │ ├── ServerIdAwareSyncWriteClient.java │ │ ├── ServerReplicationGroupUtil.java │ │ ├── ShuffleDataReader.java │ │ ├── ShuffleDataSocketReadClient.java │ │ ├── ShuffleDataSyncWriteClient.java │ │ ├── ShuffleDataSyncWriteClientBase.java │ │ ├── ShuffleDataWriter.java │ │ ├── ShuffleWriteConfig.java │ │ ├── SingleServerReadClient.java │ │ ├── SingleServerWriteClient.java │ │ ├── StreamDecoderBase.java │ │ ├── TaskByteArrayDataBlock.java │ │ ├── TaskDataBlock.java │ │ ├── UnpooledWriteClientFactory.java │ │ └── WriteClientFactory.java │ │ ├── common │ │ ├── AppMapId.java │ │ ├── AppShuffleId.java │ │ ├── AppShufflePartitionId.java │ │ ├── AppTaskAttemptId.java │ │ ├── Compression.java │ │ ├── DataBlock.java │ │ ├── DataBlockHeader.java │ │ ├── DownloadServerVerboseInfo.java │ │ ├── FilePathAndLength.java │ │ ├── FixedLengthInputStream.java │ │ ├── LowMemoryListener.java │ │ ├── MapTaskCommitStatus.java │ │ ├── MapTaskRssInfo.java │ │ ├── MemoryMonitor.java │ │ ├── PartitionFilePathAndLength.java │ │ ├── ServerCandidate.java │ │ ├── ServerDetail.java │ │ ├── ServerDetailCollection.java │ │ ├── ServerList.java │ │ ├── ServerReplicationGroup.java │ │ ├── ServerRole.java │ │ └── ShuffleMapTaskAttemptId.java │ │ ├── decoders │ │ ├── StreamServerMessageDecoder.java │ │ └── StreamServerVersionDecoder.java │ │ ├── exceptions │ │ ├── ExceptionWrapper.java │ │ ├── RssAggregateException.java │ │ ├── RssDiskSpaceException.java │ │ ├── RssDuplicateAppTaskAttemptException.java │ │ ├── RssEndOfStreamException.java │ │ ├── RssException.java │ │ ├── RssFileCorruptedException.java │ │ ├── RssFinishUploadException.java │ │ ├── RssInconsistentReplicaException.java │ │ ├── RssInvalidDataException.java │ │ ├── RssInvalidMapStatusException.java │ │ ├── RssInvalidServerIdException.java │ │ ├── RssInvalidServerVersionException.java │ │ ├── RssInvalidStateException.java │ │ ├── RssMaxConnectionsException.java │ │ ├── RssMissingShuffleWriteConfigException.java │ │ ├── RssNetworkException.java │ │ ├── RssNoActiveReadClientException.java │ │ ├── RssNoServerAvailableException.java │ │ ├── RssNonRecoverableException.java │ │ ├── RssOperationQueueFullException.java │ │ ├── RssQueueNotReadyException.java │ │ ├── RssRetryTimeoutException.java │ │ ├── RssRetryableSparkTaskException.java │ │ ├── RssServerBusyException.java │ │ ├── RssServerDownException.java │ │ ├── RssServerResolveException.java │ │ ├── RssShuffleCorruptedException.java │ │ ├── RssShuffleDataNotAvailableException.java │ │ ├── RssShuffleStageNotStartedException.java │ │ ├── RssStaleTaskAttemptException.java │ │ ├── RssStreamReadException.java │ │ ├── RssTooMuchDataException.java │ │ ├── RssUberEnvironmentException.java │ │ ├── RssUnsupportedCompressionException.java │ │ └── RssWriteRecordException.java │ │ ├── execution │ │ ├── ExecutorAppState.java │ │ ├── ExecutorShuffleStageState.java │ │ ├── LocalFileStateStore.java │ │ ├── LocalFileStateStoreIterator.java │ │ ├── ShuffleDataWrapper.java │ │ ├── ShuffleExecutor.java │ │ ├── ShufflePartitionWriter.java │ │ ├── StagePersistentInfo.java │ │ ├── StateStore.java │ │ ├── StateStoreLoadResult.java │ │ ├── TaskAttemptCollection.java │ │ └── TaskAttemptIdAndState.java │ │ ├── handlers │ │ ├── ChannelFutureCloseListener.java │ │ ├── ChannelIdleCheck.java │ │ ├── DownloadChannelInboundHandler.java │ │ ├── DownloadServerHandler.java │ │ ├── HandlerUtil.java │ │ ├── HttpChannelInboundHandler.java │ │ ├── NotifyChannelInboundHandler.java │ │ ├── NotifyServerHandler.java │ │ ├── RegistryChannelInboundHandler.java │ │ ├── RegistryServerHandler.java │ │ ├── ResponseStatusAndMessage.java │ │ ├── UploadChannelInboundHandler.java │ │ ├── UploadChannelManager.java │ │ └── UploadServerHandler.java │ │ ├── messages │ │ ├── AppDeletionStateItem.java │ │ ├── BaseMessage.java │ │ ├── ConnectDownloadRequest.java │ │ ├── ConnectDownloadRequestMessage.java │ │ ├── ConnectDownloadResponse.java │ │ ├── ConnectNotifyRequest.java │ │ ├── ConnectNotifyResponse.java │ │ ├── ConnectRegistryRequest.java │ │ ├── ConnectRegistryResponse.java │ │ ├── ConnectUploadRequest.java │ │ ├── ConnectUploadResponse.java │ │ ├── ControlMessage.java │ │ ├── FinishApplicationAttemptRequestMessage.java │ │ ├── FinishApplicationJobRequestMessage.java │ │ ├── FinishUploadMessage.java │ │ ├── GetBusyStatusRequest.java │ │ ├── GetBusyStatusResponse.java │ │ ├── GetDataAvailabilityRequest.java │ │ ├── GetDataAvailabilityRequestMessage.java │ │ ├── GetDataAvailabilityResponse.java │ │ ├── GetServersRequestMessage.java │ │ ├── GetServersResponseMessage.java │ │ ├── HeartbeatMessage.java │ │ ├── MessageConstants.java │ │ ├── RegisterServerRequestMessage.java │ │ ├── RegisterServerResponseMessage.java │ │ ├── SerializableMessage.java │ │ ├── ServerResponseMessage.java │ │ ├── ShuffleDataWrapper.java │ │ ├── ShuffleStageStatus.java │ │ ├── StageCorruptionStateItem.java │ │ ├── StageInfoStateItem.java │ │ ├── StartUploadMessage.java │ │ └── TaskAttemptCommitStateItem.java │ │ ├── metadata │ │ ├── InMemoryServiceRegistry.java │ │ ├── ServiceRegistry.java │ │ ├── ServiceRegistryUtils.java │ │ ├── ServiceRegistryWrapper.java │ │ ├── StandaloneServiceRegistryClient.java │ │ ├── ZooKeeperFaultTolerantServiceRegistry.java │ │ ├── ZooKeeperServerNodeData.java │ │ └── ZooKeeperServiceRegistry.java │ │ ├── metrics │ │ ├── ApplicationJobStatusMetrics.java │ │ ├── ApplicationJobStatusMetricsKey.java │ │ ├── ApplicationMetrics.java │ │ ├── ApplicationMetricsKey.java │ │ ├── ClientConnectMetrics.java │ │ ├── ClientConnectMetricsKey.java │ │ ├── ExceptionMetricGroupContainer.java │ │ ├── ExceptionMetrics.java │ │ ├── ExceptionMetricsKey.java │ │ ├── M3DummyScope.java │ │ ├── M3DummyScopeBuilder.java │ │ ├── M3Stats.java │ │ ├── MetadataClientMetrics.java │ │ ├── MetadataClientMetricsContainer.java │ │ ├── MetadataClientMetricsKey.java │ │ ├── MetricGroup.java │ │ ├── MetricGroupContainer.java │ │ ├── NettyServerSideMetricGroupContainer.java │ │ ├── NettyServerSideMetricsKey.java │ │ ├── NotifyClientMetrics.java │ │ ├── NotifyClientMetricsKey.java │ │ ├── NotifyServerMetricsContainer.java │ │ ├── ReadClientMetrics.java │ │ ├── ReadClientMetricsKey.java │ │ ├── ScheduledMetricCollector.java │ │ ├── ServerHandlerMetrics.java │ │ ├── ShuffleClientStageMetrics.java │ │ ├── ShuffleClientStageMetricsKey.java │ │ ├── WriteClientMetrics.java │ │ └── WriteClientMetricsKey.java │ │ ├── storage │ │ ├── ShuffleFileOutputStream.java │ │ ├── ShuffleFileStorage.java │ │ ├── ShuffleFileUtils.java │ │ ├── ShuffleOutputStream.java │ │ └── ShuffleStorage.java │ │ ├── tools │ │ ├── FileDescriptorStressTest.java │ │ ├── FsyncPerfTest.java │ │ ├── PartitionFileChecker.java │ │ ├── SerializerBenchmark.java │ │ ├── StreamReadClientVerify.java │ │ ├── StreamServerStressTool.java │ │ ├── StreamServerStressToolLongRun.java │ │ ├── StreamServerStressToolWrite64GB.java │ │ └── TestUtils.java │ │ └── util │ │ ├── AsyncSocketCompletionHandler.java │ │ ├── AsyncSocketState.java │ │ ├── ByteBufUtils.java │ │ ├── CountedOutputStream.java │ │ ├── ExceptionUtils.java │ │ ├── FileUtils.java │ │ ├── HttpUtils.java │ │ ├── JsonUtils.java │ │ ├── LogUtils.java │ │ ├── MonitorUtils.java │ │ ├── MovingAverageCalculator.java │ │ ├── NettyUtils.java │ │ ├── NetworkUtils.java │ │ ├── ObjectWrapper.java │ │ ├── RateCounter.java │ │ ├── RetryUtils.java │ │ ├── ServerHostAndPort.java │ │ ├── SocketAsyncWriteCallback.java │ │ ├── SocketUtils.java │ │ ├── StreamUtils.java │ │ ├── StringUtils.java │ │ ├── SystemUtils.java │ │ └── ThreadUtils.java ├── resources │ ├── log4j-rss-debug.properties │ └── log4j-rss-prod.properties └── scala │ ├── com │ └── uber │ │ └── rss │ │ └── test │ │ ├── AggOrderingShuffleTestApp.scala │ │ ├── MapCombinerShuffleTestApp.scala │ │ ├── MultiStageShuffleTestApp.scala │ │ ├── ReducerAggShuffleTestApp.scala │ │ ├── RepartitionShuffleTestApp.scala │ │ ├── ShuffleTestAppBase.scala │ │ └── util │ │ └── KeyValueGenerator.scala │ └── org │ └── apache │ └── spark │ └── shuffle │ ├── MockTaskContext.scala │ ├── RssEmptyShuffleReader.scala │ ├── RssOpts.scala │ ├── RssServerSelectionResult.scala │ ├── RssServiceRegistry.scala │ ├── RssShuffleBlockResolver.scala │ ├── RssShuffleHandle.scala │ ├── RssShuffleManager.scala │ ├── RssShuffleReader.scala │ ├── RssShuffleServerHandle.scala │ ├── RssShuffleWriter.scala │ └── rss │ ├── BlockDownloaderPartitionRangeRecordIterator.scala │ ├── BlockDownloaderPartitionRecordIterator.scala │ ├── EmptyRecordIterator.scala │ ├── MapOutputRssInfo.scala │ ├── RssSparkListener.scala │ ├── RssStressTool.scala │ ├── RssUtils.scala │ ├── WriterAggregationImpl.scala │ ├── WriterAggregationManager.scala │ ├── WriterBufferManager.scala │ └── WriterNoAggregationManager.scala └── test ├── java └── com │ └── uber │ └── rss │ ├── PackageJarFileITCase.java │ ├── RssBuildInfoTest.java │ ├── StreamServerCleanupTest.java │ ├── StreamServerConfigTest.java │ ├── StreamServerHttpTest.java │ ├── StreamServerMultiAttemptTest.java │ ├── StreamServerTest.java │ ├── StreamServerWritingTooMuchDataTest.java │ ├── clients │ ├── BlockingServerReadClientTest.java │ ├── BusyStatusSocketClientTest.java │ ├── DataBlockSocketReadClientTest.java │ ├── DataBlockSyncWriteClientTest.java │ ├── DownloadRequestClient.java │ ├── HeartbeatSocketClientTest.java │ ├── MultiServerAsyncWriteClientTest.java │ ├── MultiServerHeartbeatClientTest.java │ ├── MultiServerSocketReadClientTest.java │ ├── MultiServerSyncWriteClientTest.java │ ├── NotifyClientTest.java │ ├── PooledWriteClientFactoryRandomTest.java │ ├── PooledWriteClientFactoryTest.java │ ├── RecordSocketReadClientTest.java │ ├── RegistryClientTest.java │ ├── ReplicatedReadClientTest.java │ ├── ReplicatedWriteClientTest.java │ ├── RetriableSocketReadClientTest.java │ ├── ServerIdAwareSocketReadClientTest.java │ ├── ServerIdAwareSyncWriteClientTest.java │ ├── ServerIdleTimeoutTest.java │ ├── ServerReplicationGroupUtilTest.java │ ├── ShuffleDataSocketReadClientTest.java │ └── WriteClientEdgeCaseTest.java │ ├── common │ ├── FixedLengthInputStreamTest.java │ ├── MapTaskRssInfoTest.java │ └── ServerDetailCollectionTest.java │ ├── execution │ ├── LocalFileLocalFileStateStoreIteratorTest.java │ ├── LocalFileStateStoreStressTest.java │ ├── LocalFileStateStoreTest.java │ └── ShuffleExecutorTest.java │ ├── metadata │ ├── InMemoryServiceRegistryTest.java │ ├── ServiceRegistryTestUtil.java │ ├── ServiceRegistryUtilsTest.java │ ├── StandalongServiceRegistryClientTest.java │ ├── ZooKeeperFaultTolerantServiceRegistryTest.java │ └── ZooKeeperServiceRegistryTest.java │ ├── testutil │ ├── ClientTestUtils.java │ ├── NettyMemoryExperiments.java │ ├── RssMiniCluster.java │ ├── RssZookeeperCluster.java │ ├── StreamServerTestUtils.java │ ├── TestConstants.java │ ├── TestStreamServer.java │ └── ZooKeeperTestCluster.java │ ├── tools │ └── StreamServerStressToolTest.java │ └── util │ ├── ByteBufUtilsTest.java │ ├── ExceptionUtilsTest.java │ ├── FileUtilsTest.java │ ├── MonitorUtilsTest.java │ ├── MovingAverageCalculatorTest.java │ ├── NetworkUtilsTest.java │ ├── RateCounterTest.java │ ├── RetryUtilsTest.java │ ├── ServerHostAndPortTest.java │ └── StringUtilsTest.java └── scala └── org └── apache └── spark └── shuffle ├── EmptyPartitionShuffleTest.scala ├── LazyWriteClientTest.scala ├── RssOptsTest.scala ├── RssShuffleManagerReplicateServerGroupTest.scala ├── RssShuffleManagerServerRestartTest.scala ├── RssShuffleManagerTest.scala ├── RssShuffleManagerWithStandaloneRegistryServerTest.scala ├── ShuffleWithAggregationTest.scala ├── ShuffleWithOrderingTest.scala ├── ShuffleWithSparkSqlLargeDataTest.scala ├── ShuffleWithSparkSqlRandomTest.scala ├── ShuffleWithSparkSqlTest.scala ├── ShuffleWithZstdCompressionTest.scala ├── TestUdfs.scala ├── TestUtil.scala ├── ZeroMapperShuffleTest.scala └── rss ├── MapOutputRssInfoTest.scala └── WriteBufferManagerTest.scala /.github/workflows/maven.yml: -------------------------------------------------------------------------------- 1 | # This workflow will build a Java project with Maven 2 | # For more information see: https://help.github.com/actions/language-and-framework-guides/building-and-testing-java-with-maven 3 | 4 | name: Java CI with Maven 5 | 6 | on: 7 | push: 8 | branches: [ master, spark30, spark24 ] 9 | pull_request: 10 | branches: [ master, spark30, spark24 ] 11 | 12 | jobs: 13 | build: 14 | 15 | runs-on: ubuntu-latest 16 | 17 | steps: 18 | - uses: actions/checkout@v2 19 | - name: Set up JDK 1.8 20 | uses: actions/setup-java@v1 21 | with: 22 | java-version: 1.8 23 | - name: Build with Maven 24 | run: mvn -B package --file pom.xml 25 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | !.gitignore 2 | !.gitmodules 3 | !.jenkins/ 4 | 5 | .fossa.yml 6 | 7 | # Java 8 | *.class 9 | 10 | # Intellij 11 | *.iml 12 | *.iws 13 | 14 | classes/ 15 | 16 | log/ 17 | tmp/ 18 | temp/ 19 | out/ 20 | 21 | # Others 22 | *.out 23 | *.test 24 | # *.xml 25 | *.swp 26 | .idea/ 27 | *.cov 28 | *.tmp 29 | .DS_Store 30 | *.log* 31 | *coverage* 32 | .cache 33 | .idea 34 | *.pyc 35 | *.egg-info 36 | *.orig 37 | tmp.yaml 38 | out.txt 39 | .coverage 40 | coverage.xml 41 | cover 42 | jenkins.xml 43 | .cache 44 | .vscode/ 45 | target/ 46 | 47 | # Ignore any mocks sub-directory. 48 | mocks/ 49 | 50 | # Ignore docker generated files. 51 | docker/pip.conf 52 | 53 | # Emacs 54 | *~ 55 | .\#* 56 | \#*\# 57 | .unison.*.tmp 58 | workspace/ 59 | 60 | # Vim 61 | tags 62 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: java 2 | dist: trusty 3 | jdk: 4 | - oraclejdk8 5 | before_script: 6 | - "curl -H 'Cache-Control: no-cache' https://raw.githubusercontent.com/fossas/fossa-cli/master/install.sh | sudo bash" 7 | script: 8 | - jdk_switcher use oraclejdk8 9 | - fossa init 10 | - fossa analyze 11 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Remote Shuffle Service 2 | Copyright (c) 2020 Uber Technologies, Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | 16 | Permission is hereby granted, free of charge, to any person obtaining a copy 17 | of this software and associated documentation files (the "Software"), to deal 18 | in the Software without restriction, including without limitation the rights 19 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 20 | copies of the Software, and to permit persons to whom the Software is 21 | furnished to do so, subject to the following conditions: 22 | 23 | The above copyright notice and this permission notice shall be included in all 24 | copies or substantial portions of the Software. 25 | 26 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, 27 | INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A 28 | PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT 29 | HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION 30 | OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE 31 | SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. -------------------------------------------------------------------------------- /NOTICE: -------------------------------------------------------------------------------- 1 | Remote Shuffle Service 2 | Copyright (c) 2020 Uber Technologies, Inc. 3 | 4 | I. Included Software 5 | 6 | This product may include or depend software developed at 7 | The Apache Software Foundation (https://www.apache.org/). 8 | Licensed under the Apache License. 9 | 10 | II. License Summary 11 | - Apache License 2.0 -------------------------------------------------------------------------------- /build/rss-build-info: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | RESOURCE_DIR="$1" 4 | mkdir -p "$RESOURCE_DIR" 5 | RSS_BUILD_INFO="${RESOURCE_DIR}"/rss-build-info.properties 6 | 7 | echo Generating resource file: "$RSS_BUILD_INFO" 8 | 9 | echo_properties() { 10 | echo version=$1 11 | echo revision=$(git rev-parse HEAD) 12 | } 13 | 14 | echo_properties $2 > "$RSS_BUILD_INFO" 15 | 16 | 17 | -------------------------------------------------------------------------------- /docs/server-data-dispatcher-diagram-1.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/uber/RemoteShuffleService/b773f99bc6883710de125e864c8cf8824f939669/docs/server-data-dispatcher-diagram-1.png -------------------------------------------------------------------------------- /docs/server-data-dispatcher-diagram-2.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/uber/RemoteShuffleService/b773f99bc6883710de125e864c8cf8824f939669/docs/server-data-dispatcher-diagram-2.png -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/RssBuildInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss; 16 | 17 | import com.uber.rss.exceptions.RssException; 18 | 19 | import java.io.IOException; 20 | import java.io.InputStream; 21 | import java.util.Properties; 22 | 23 | public class RssBuildInfo { 24 | public static String UnknownValue = ""; 25 | 26 | public static String Version; 27 | public static String Revision; 28 | 29 | static { 30 | String fileName = "rss-build-info.properties"; 31 | try (InputStream resourceStream = Thread.currentThread().getContextClassLoader().getResourceAsStream(fileName)) { 32 | if (resourceStream == null) { 33 | throw new RssException(String.format("Could not find info file: %s", fileName)); 34 | } 35 | 36 | Properties props = new Properties(); 37 | props.load(resourceStream); 38 | 39 | Version = props.getProperty("version", UnknownValue); 40 | Revision = props.getProperty("revision", UnknownValue); 41 | } catch (IOException e) { 42 | throw new RssException(String.format("Failed to open info file: %s", fileName)); 43 | } 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/BlockingShuffleDataReader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | /*** 18 | * Shuffle data reader. 19 | */ 20 | public interface BlockingShuffleDataReader extends AutoCloseable { 21 | 22 | TaskByteArrayDataBlock readRecord(); 23 | 24 | long getShuffleReadBytes(); 25 | 26 | void close(); 27 | } -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/BlockingSingleServerReadClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | import com.uber.rss.common.DownloadServerVerboseInfo; 18 | 19 | public interface BlockingSingleServerReadClient extends BlockingShuffleDataReader { 20 | DownloadServerVerboseInfo connect(); 21 | } 22 | 23 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/ClientConstants.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | public class ClientConstants { 18 | public static final long DEFAULT_CONNECTION_IDLE_TIMEOUT_MILLIS = 3 * 60 * 1000; 19 | } 20 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/ClientRetryOptions.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | public class ClientRetryOptions { 18 | private final long retryIntervalMillis; 19 | private final long retryMaxMillis; 20 | private final ServerConnectionRefresher retryConnectionResolver; 21 | 22 | public ClientRetryOptions(long retryIntervalMillis, long retryMaxMillis, ServerConnectionRefresher retryConnectionResolver) { 23 | this.retryIntervalMillis = retryIntervalMillis; 24 | this.retryMaxMillis = retryMaxMillis; 25 | this.retryConnectionResolver = retryConnectionResolver; 26 | } 27 | 28 | public long getRetryIntervalMillis() { 29 | return retryIntervalMillis; 30 | } 31 | 32 | public long getRetryMaxMillis() { 33 | return retryMaxMillis; 34 | } 35 | 36 | public ServerConnectionRefresher getRetryConnectionResolver() { 37 | return retryConnectionResolver; 38 | } 39 | 40 | @Override 41 | public String toString() { 42 | return "ClientRetryOptions{" + 43 | "retryIntervalMillis=" + retryIntervalMillis + 44 | ", retryMaxMillis=" + retryMaxMillis + 45 | ", retryConnectionResolver=" + retryConnectionResolver + 46 | '}'; 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/EofRecordKeyValuePair.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | public class EofRecordKeyValuePair extends TaskByteArrayDataBlock { 18 | public EofRecordKeyValuePair() { 19 | super(null, 0L); 20 | } 21 | } -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/FailedFetchRecordKeyValuePair.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | /*** 18 | * This class is a marker to put into the record queue in async read client, so it knows 19 | * there is an underlying failure. 20 | */ 21 | public class FailedFetchRecordKeyValuePair extends TaskByteArrayDataBlock { 22 | private Throwable cause; 23 | 24 | public FailedFetchRecordKeyValuePair(Throwable cause) { 25 | super(null, 0L); 26 | this.cause = cause; 27 | } 28 | 29 | public Throwable getCause() { 30 | return cause; 31 | } 32 | } -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/MultiServerReadClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | public interface MultiServerReadClient extends ShuffleDataReader { 18 | void connect(); 19 | } 20 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/MultiServerWriteClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | /*** 18 | * Shuffle write client to upload data (records) to multiple shuffle servers. 19 | */ 20 | public interface MultiServerWriteClient extends ShuffleDataWriter { 21 | 22 | void connect(); 23 | 24 | } 25 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/PlainShuffleDataSocketReadClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | import com.uber.rss.common.AppShufflePartitionId; 18 | import org.slf4j.Logger; 19 | import org.slf4j.LoggerFactory; 20 | 21 | import java.util.Collection; 22 | 23 | /*** 24 | * Shuffle read client to download data from shuffle server. 25 | */ 26 | public class PlainShuffleDataSocketReadClient extends ShuffleDataSocketReadClient { 27 | private static final Logger logger = 28 | LoggerFactory.getLogger(PlainShuffleDataSocketReadClient.class); 29 | 30 | public PlainShuffleDataSocketReadClient(String host, int port, int timeoutMillis, String user, AppShufflePartitionId appShufflePartitionId, Collection fetchTaskAttemptIds, long dataAvailablePollInterval, long dataAvailableWaitTime) { 31 | super(host, port, timeoutMillis, user, appShufflePartitionId, fetchTaskAttemptIds, dataAvailablePollInterval, dataAvailableWaitTime); 32 | } 33 | 34 | @Override 35 | public String toString() { 36 | return "PlainRecordSocketReadClient{" + 37 | super.toString() + 38 | '}'; 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/ReadClientDataOptions.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | import java.util.ArrayList; 18 | import java.util.Collection; 19 | import java.util.Collections; 20 | 21 | public class ReadClientDataOptions { 22 | private final Collection fetchTaskAttemptIds; 23 | private final long dataAvailablePollInterval; 24 | private final long dataAvailableWaitTime; 25 | 26 | public ReadClientDataOptions(Collection fetchTaskAttemptIds, long dataAvailablePollInterval, long dataAvailableWaitTime) { 27 | this.fetchTaskAttemptIds = Collections.unmodifiableCollection(new ArrayList<>(fetchTaskAttemptIds)); 28 | this.dataAvailablePollInterval = dataAvailablePollInterval; 29 | this.dataAvailableWaitTime = dataAvailableWaitTime; 30 | } 31 | 32 | public Collection getFetchTaskAttemptIds() { 33 | return fetchTaskAttemptIds; 34 | } 35 | 36 | public long getDataAvailablePollInterval() { 37 | return dataAvailablePollInterval; 38 | } 39 | 40 | public long getDataAvailableWaitTime() { 41 | return dataAvailableWaitTime; 42 | } 43 | 44 | @Override 45 | public String toString() { 46 | return "WriteClientDataOptions{" + 47 | "fetchTaskAttemptIds=" + fetchTaskAttemptIds + 48 | ", dataAvailablePollInterval=" + dataAvailablePollInterval + 49 | ", dataAvailableWaitTime=" + dataAvailableWaitTime + 50 | '}'; 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/ServerConnectionRefresher.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | import com.uber.rss.common.ServerDetail; 18 | 19 | public interface ServerConnectionRefresher { 20 | ServerDetail refreshConnection(ServerDetail serverDetail); 21 | } 22 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/ServerConnectionStringCache.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | import com.uber.rss.common.ServerDetail; 18 | import com.uber.rss.common.ServerList; 19 | 20 | import java.util.concurrent.ConcurrentHashMap; 21 | import java.util.stream.Collectors; 22 | 23 | /** 24 | * This class caches server connections by server ids. It also allow outside code to update 25 | * the cache. 26 | */ 27 | public class ServerConnectionStringCache { 28 | 29 | private final static ServerConnectionStringCache instance = new ServerConnectionStringCache(); 30 | 31 | public static ServerConnectionStringCache getInstance() { 32 | return instance; 33 | } 34 | 35 | private ConcurrentHashMap servers = new ConcurrentHashMap<>(); 36 | 37 | public ServerConnectionStringCache() { 38 | } 39 | 40 | public ServerDetail getServer(ServerDetail serverDetail) { 41 | ServerDetail cachedServerDetail = servers.get(serverDetail.getServerId()); 42 | if (cachedServerDetail != null) { 43 | return cachedServerDetail; 44 | } else { 45 | return serverDetail; 46 | } 47 | } 48 | 49 | public ServerList getServerList(ServerList serverList) { 50 | return new ServerList(serverList.getSevers().stream().map(this::getServer).collect(Collectors.toList())); 51 | } 52 | 53 | public void updateServer(String serverId, ServerDetail serverDetail) { 54 | servers.put(serverId, serverDetail); 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/ServerConnectionStringResolver.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | import com.uber.rss.common.ServerDetail; 18 | 19 | /** 20 | * An interface to query server connection string by giving a server id. 21 | */ 22 | public interface ServerConnectionStringResolver { 23 | ServerDetail resolveConnection(String serverId); 24 | } 25 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/ShuffleDataReader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | /*** 18 | * Shuffle data reader. 19 | */ 20 | public interface ShuffleDataReader extends AutoCloseable { 21 | 22 | TaskDataBlock readDataBlock(); 23 | 24 | long getShuffleReadBytes(); 25 | 26 | void close(); 27 | } 28 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/ShuffleDataSyncWriteClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | import com.uber.rss.common.AppTaskAttemptId; 18 | import com.uber.rss.messages.ConnectUploadResponse; 19 | 20 | import java.nio.ByteBuffer; 21 | 22 | /*** 23 | * Shuffle write client to upload data (records) to shuffle server. 24 | */ 25 | public interface ShuffleDataSyncWriteClient extends SingleServerWriteClient { 26 | 27 | String getHost(); 28 | 29 | int getPort(); 30 | 31 | String getUser(); 32 | 33 | String getAppId(); 34 | 35 | String getAppAttempt(); 36 | 37 | ConnectUploadResponse connect(); 38 | 39 | void startUpload(AppTaskAttemptId appTaskAttemptId, int numMaps, int numPartitions); 40 | 41 | void writeDataBlock(int partition, ByteBuffer value); 42 | 43 | void finishUpload(); 44 | 45 | long getShuffleWriteBytes(); 46 | } 47 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/ShuffleDataWriter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | import com.uber.rss.common.AppTaskAttemptId; 18 | 19 | import java.nio.ByteBuffer; 20 | 21 | /*** 22 | * Shuffle data writer. 23 | */ 24 | public interface ShuffleDataWriter extends AutoCloseable { 25 | 26 | void startUpload(AppTaskAttemptId appTaskAttemptId, int numMaps, int numPartitions); 27 | 28 | void writeDataBlock(int partition, ByteBuffer value); 29 | 30 | void finishUpload(); 31 | 32 | /** 33 | * Get write bytes since last start upload. 34 | * @return 35 | */ 36 | long getShuffleWriteBytes(); 37 | 38 | void close(); 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/ShuffleWriteConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | import java.util.Objects; 18 | 19 | public class ShuffleWriteConfig { 20 | 21 | private final short numSplits; 22 | 23 | public ShuffleWriteConfig() { 24 | this((short)1); 25 | } 26 | 27 | public ShuffleWriteConfig(short numSplits) { 28 | this.numSplits = numSplits; 29 | } 30 | 31 | public short getNumSplits() { 32 | return numSplits; 33 | } 34 | 35 | @Override 36 | public boolean equals(Object o) { 37 | if (this == o) return true; 38 | if (o == null || getClass() != o.getClass()) return false; 39 | ShuffleWriteConfig that = (ShuffleWriteConfig) o; 40 | return numSplits == that.numSplits; 41 | } 42 | 43 | @Override 44 | public int hashCode() { 45 | return Objects.hash(numSplits); 46 | } 47 | 48 | @Override 49 | public String toString() { 50 | return "ShuffleWriteConfig{" + 51 | ", numSplits=" + numSplits + 52 | '}'; 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/SingleServerReadClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | import com.uber.rss.common.DownloadServerVerboseInfo; 18 | 19 | public interface SingleServerReadClient extends ShuffleDataReader { 20 | DownloadServerVerboseInfo connect(); 21 | } 22 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/SingleServerWriteClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | import com.uber.rss.messages.ConnectUploadResponse; 18 | 19 | /*** 20 | * Shuffle write client to upload data (records) to shuffle server. 21 | */ 22 | public interface SingleServerWriteClient extends ShuffleDataWriter { 23 | 24 | ConnectUploadResponse connect(); 25 | 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/UnpooledWriteClientFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | import org.slf4j.Logger; 18 | import org.slf4j.LoggerFactory; 19 | 20 | /** 21 | * This class creates unpooled write client. 22 | */ 23 | public class UnpooledWriteClientFactory implements WriteClientFactory { 24 | private static final Logger logger = LoggerFactory.getLogger(UnpooledWriteClientFactory.class); 25 | 26 | private final static UnpooledWriteClientFactory instance = new UnpooledWriteClientFactory(); 27 | 28 | public static UnpooledWriteClientFactory getInstance() { 29 | return instance; 30 | } 31 | 32 | @Override 33 | public ShuffleDataSyncWriteClient getOrCreateClient(String host, int port, int timeoutMillis, boolean finishUploadAck, String user, String appId, String appAttempt, ShuffleWriteConfig shuffleWriteConfig) { 34 | final ShuffleDataSyncWriteClient writeClient = new PlainShuffleDataSyncWriteClient( 35 | host, 36 | port, 37 | timeoutMillis, 38 | finishUploadAck, 39 | user, 40 | appId, 41 | appAttempt, 42 | shuffleWriteConfig); 43 | return writeClient; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/clients/WriteClientFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | public interface WriteClientFactory { 18 | ShuffleDataSyncWriteClient getOrCreateClient(String host, int port, int timeoutMillis, boolean finishUploadAck, String user, String appId, String appAttempt, ShuffleWriteConfig shuffleWriteConfig); 19 | } 20 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/common/DataBlock.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.common; 16 | 17 | public class DataBlock { 18 | private DataBlockHeader header; 19 | private byte[] payload; 20 | 21 | public DataBlock(DataBlockHeader header, byte[] payload) { 22 | this.header = header; 23 | this.payload = payload; 24 | } 25 | 26 | public DataBlockHeader getHeader() { 27 | return header; 28 | } 29 | 30 | public byte[] getPayload() { 31 | return payload; 32 | } 33 | 34 | @Override 35 | public String toString() { 36 | return "DataBlock{" + 37 | "header=" + header + 38 | '}'; 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/common/DataBlockHeader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.common; 16 | 17 | import com.uber.rss.util.ByteBufUtils; 18 | 19 | public class DataBlockHeader { 20 | public static int NUM_BYTES = Long.BYTES + Integer.BYTES; 21 | 22 | public static byte[] serializeToBytes(byte[] taskAttemptIdBytes, int length) { 23 | byte[] bytes = new byte[NUM_BYTES]; 24 | System.arraycopy(taskAttemptIdBytes, 0, bytes, 0, Long.BYTES); 25 | ByteBufUtils.writeInt(bytes, Long.BYTES, length); 26 | return bytes; 27 | } 28 | 29 | public static DataBlockHeader deserializeFromBytes(byte[] bytes) { 30 | long taskAttemptId = ByteBufUtils.readLong(bytes, 0); 31 | int length = ByteBufUtils.readInt(bytes, Long.BYTES); 32 | return new DataBlockHeader(taskAttemptId, length); 33 | } 34 | 35 | private final long taskAttemptId; 36 | private final int length; 37 | 38 | public DataBlockHeader(long taskAttemptId, int length) { 39 | this.taskAttemptId = taskAttemptId; 40 | this.length = length; 41 | } 42 | 43 | public long getTaskAttemptId() { 44 | return taskAttemptId; 45 | } 46 | 47 | public int getLength() { 48 | return length; 49 | } 50 | 51 | @Override 52 | public String toString() { 53 | return "DataBlockHeader{" + 54 | "taskAttemptId=" + taskAttemptId + 55 | ", length=" + length + 56 | '}'; 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/common/DownloadServerVerboseInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.common; 16 | 17 | // TODO create a constructor to pass fields 18 | public class DownloadServerVerboseInfo { 19 | private String runningVersion; 20 | private String id; 21 | 22 | private MapTaskCommitStatus mapTaskCommitStatus; 23 | 24 | public String getRunningVersion() { 25 | return runningVersion; 26 | } 27 | 28 | public void setRunningVersion(String runningVersion) { 29 | this.runningVersion = runningVersion; 30 | } 31 | 32 | public String getId() { 33 | return id; 34 | } 35 | 36 | public void setId(String id) { 37 | this.id = id; 38 | } 39 | 40 | public MapTaskCommitStatus getMapTaskCommitStatus() { 41 | return mapTaskCommitStatus; 42 | } 43 | 44 | public void setMapTaskCommitStatus(MapTaskCommitStatus mapTaskCommitStatus) { 45 | this.mapTaskCommitStatus = mapTaskCommitStatus; 46 | } 47 | 48 | @Override 49 | public String toString() { 50 | return "DownloadServerVerboseInfo{" + 51 | "runningVersion='" + runningVersion + '\'' + 52 | ", id='" + id + '\'' + 53 | ", mapTaskCommitStatus=" + mapTaskCommitStatus + 54 | '}'; 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/common/FilePathAndLength.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.common; 16 | 17 | import java.util.Objects; 18 | 19 | public class FilePathAndLength { 20 | private String path; 21 | private long length; 22 | 23 | public FilePathAndLength(String path, long length) { 24 | this.path = path; 25 | this.length = length; 26 | } 27 | 28 | public String getPath() { 29 | return path; 30 | } 31 | 32 | public long getLength() { 33 | return length; 34 | } 35 | 36 | @Override 37 | public boolean equals(Object o) { 38 | if (this == o) return true; 39 | if (o == null || getClass() != o.getClass()) return false; 40 | FilePathAndLength that = (FilePathAndLength) o; 41 | return length == that.length && 42 | Objects.equals(path, that.path); 43 | } 44 | 45 | @Override 46 | public int hashCode() { 47 | return Objects.hash(path, length); 48 | } 49 | 50 | @Override 51 | public String toString() { 52 | return "FilePathAndLength{" + 53 | "path='" + path + '\'' + 54 | ", length=" + length + 55 | '}'; 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/common/LowMemoryListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.common; 16 | 17 | public interface LowMemoryListener { 18 | void run(); 19 | } 20 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/common/PartitionFilePathAndLength.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.common; 16 | 17 | import java.util.Objects; 18 | 19 | public class PartitionFilePathAndLength { 20 | private int partition; 21 | private String path; 22 | private long length; 23 | 24 | public PartitionFilePathAndLength(int partition, String path, long length) { 25 | this.partition = partition; 26 | this.path = path; 27 | this.length = length; 28 | } 29 | 30 | public int getPartition() { 31 | return partition; 32 | } 33 | 34 | public String getPath() { 35 | return path; 36 | } 37 | 38 | public long getLength() { 39 | return length; 40 | } 41 | 42 | @Override 43 | public boolean equals(Object o) { 44 | if (this == o) return true; 45 | if (o == null || getClass() != o.getClass()) return false; 46 | PartitionFilePathAndLength that = (PartitionFilePathAndLength) o; 47 | return partition == that.partition && 48 | length == that.length && 49 | Objects.equals(path, that.path); 50 | } 51 | 52 | @Override 53 | public int hashCode() { 54 | return Objects.hash(partition, path, length); 55 | } 56 | 57 | @Override 58 | public String toString() { 59 | return "PartitionFilePathAndLength{" + 60 | "partition=" + partition + 61 | ", path='" + path + '\'' + 62 | ", length=" + length + 63 | '}'; 64 | } 65 | } 66 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/common/ServerCandidate.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.common; 16 | 17 | public class ServerCandidate { 18 | private ServerDetail serverDetail; 19 | private long requestLatency; 20 | private long concurrentConnections; 21 | 22 | public ServerCandidate(ServerDetail serverDetail, long requestLatency, long concurrentConnections) { 23 | this.serverDetail = serverDetail; 24 | this.requestLatency = requestLatency; 25 | this.concurrentConnections = concurrentConnections; 26 | } 27 | 28 | public ServerDetail getServerDetail() { 29 | return serverDetail; 30 | } 31 | 32 | public long getRequestLatency() { 33 | return requestLatency; 34 | } 35 | 36 | public long getConcurrentConnections(){ 37 | return concurrentConnections; 38 | } 39 | 40 | @Override 41 | public String toString() { 42 | return "ServerCandidate{" + 43 | "serverDetail=" + serverDetail + 44 | ", requestLatency=" + requestLatency + 45 | ", concurrentConnections=" + concurrentConnections + 46 | '}'; 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/common/ServerReplicationGroup.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.common; 16 | 17 | import java.util.ArrayList; 18 | import java.util.List; 19 | 20 | public class ServerReplicationGroup { 21 | private final List servers; 22 | 23 | public ServerReplicationGroup(List servers) { 24 | this.servers = new ArrayList<>(servers); 25 | } 26 | 27 | public List getServers() { 28 | return new ArrayList<>(servers); 29 | } 30 | 31 | @Override 32 | public String toString() { 33 | return "ServerReplicationGroup{" + 34 | "servers=" + servers + 35 | '}'; 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/common/ServerRole.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.common; 16 | 17 | public enum ServerRole { 18 | UNKNOWN, 19 | NOTIFY_SERVER, 20 | REGISTRY_SERVER 21 | } 22 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/ExceptionWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class ExceptionWrapper { 18 | private T exception; 19 | 20 | public T getException() { 21 | return exception; 22 | } 23 | 24 | public void setException(T exception) { 25 | this.exception = exception; 26 | } 27 | 28 | @Override 29 | public String toString() { 30 | return "ExceptionWrapper{" + 31 | "exception=" + exception + 32 | '}'; 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssAggregateException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | import com.uber.rss.util.ExceptionUtils; 18 | 19 | import java.util.ArrayList; 20 | import java.util.Collection; 21 | import java.util.List; 22 | import java.util.stream.Collectors; 23 | 24 | public class RssAggregateException extends RuntimeException { 25 | private final List causes; 26 | 27 | public RssAggregateException(Collection causes) { 28 | this.causes = new ArrayList<>(causes); 29 | } 30 | 31 | public List getCauses() { 32 | return this.causes; 33 | } 34 | 35 | @Override 36 | public String getMessage() { 37 | return this.causes.stream().map(t-> ExceptionUtils.getSimpleMessage(t) + System.lineSeparator() + org.apache.commons.lang3.exception.ExceptionUtils.getStackTrace(t)) 38 | .collect(Collectors.joining(System.lineSeparator())); 39 | } 40 | 41 | @Override 42 | public String toString() { 43 | return "RssAggregateException: " + this.getMessage(); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssDiskSpaceException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssDiskSpaceException extends RssException { 18 | public RssDiskSpaceException() { 19 | } 20 | 21 | public RssDiskSpaceException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssDiskSpaceException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | 29 | public RssDiskSpaceException(Throwable cause) { 30 | super(cause); 31 | } 32 | 33 | public RssDiskSpaceException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { 34 | super(message, cause, enableSuppression, writableStackTrace); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssDuplicateAppTaskAttemptException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | /*** 18 | * This exception is thrown when a same Spark application task attempt connects to shuffle server multiple times. 19 | */ 20 | public class RssDuplicateAppTaskAttemptException extends RssException { 21 | public RssDuplicateAppTaskAttemptException() { 22 | } 23 | 24 | public RssDuplicateAppTaskAttemptException(String message) { 25 | super(message); 26 | } 27 | 28 | public RssDuplicateAppTaskAttemptException(String message, Throwable cause) { 29 | super(message, cause); 30 | } 31 | 32 | public RssDuplicateAppTaskAttemptException(Throwable cause) { 33 | super(cause); 34 | } 35 | 36 | public RssDuplicateAppTaskAttemptException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { 37 | super(message, cause, enableSuppression, writableStackTrace); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssEndOfStreamException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssEndOfStreamException extends RssException { 18 | public RssEndOfStreamException() { 19 | } 20 | 21 | public RssEndOfStreamException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssEndOfStreamException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | 29 | public RssEndOfStreamException(Throwable cause) { 30 | super(cause); 31 | } 32 | 33 | public RssEndOfStreamException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { 34 | super(message, cause, enableSuppression, writableStackTrace); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssException extends RuntimeException { 18 | public RssException() { 19 | } 20 | 21 | public RssException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | 29 | public RssException(Throwable cause) { 30 | super(cause); 31 | } 32 | 33 | public RssException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { 34 | super(message, cause, enableSuppression, writableStackTrace); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssFileCorruptedException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssFileCorruptedException extends RssException { 18 | 19 | public RssFileCorruptedException(String message) { 20 | super(message); 21 | } 22 | 23 | public RssFileCorruptedException(String message, Throwable cause) { 24 | super(message, cause); 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssFinishUploadException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssFinishUploadException extends RssException { 18 | public RssFinishUploadException() { 19 | } 20 | 21 | public RssFinishUploadException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssFinishUploadException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssInconsistentReplicaException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssInconsistentReplicaException extends RssException { 18 | public RssInconsistentReplicaException() { 19 | } 20 | 21 | public RssInconsistentReplicaException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssInconsistentReplicaException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssInvalidDataException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssInvalidDataException extends RssException { 18 | public RssInvalidDataException() { 19 | } 20 | 21 | public RssInvalidDataException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssInvalidDataException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | 29 | public RssInvalidDataException(Throwable cause) { 30 | super(cause); 31 | } 32 | 33 | public RssInvalidDataException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { 34 | super(message, cause, enableSuppression, writableStackTrace); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssInvalidMapStatusException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssInvalidMapStatusException extends RssException { 18 | public RssInvalidMapStatusException() { 19 | } 20 | 21 | public RssInvalidMapStatusException(String message) { 22 | super(message); 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssInvalidServerIdException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssInvalidServerIdException extends RssException { 18 | public RssInvalidServerIdException() { 19 | } 20 | 21 | public RssInvalidServerIdException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssInvalidServerIdException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssInvalidServerVersionException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssInvalidServerVersionException extends RssException { 18 | public RssInvalidServerVersionException() { 19 | } 20 | 21 | public RssInvalidServerVersionException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssInvalidServerVersionException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | 29 | } 30 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssInvalidStateException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | /*** 18 | * This exception is thrown when there is error with internal state in the shuffle server. 19 | */ 20 | public class RssInvalidStateException extends RssException { 21 | public RssInvalidStateException() { 22 | } 23 | 24 | public RssInvalidStateException(String message) { 25 | super(message); 26 | } 27 | 28 | public RssInvalidStateException(String message, Throwable cause) { 29 | super(message, cause); 30 | } 31 | 32 | public RssInvalidStateException(Throwable cause) { 33 | super(cause); 34 | } 35 | 36 | public RssInvalidStateException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { 37 | super(message, cause, enableSuppression, writableStackTrace); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssMaxConnectionsException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssMaxConnectionsException extends Exception { 18 | private String message; 19 | private int currentConnections = -1; 20 | private int maxConnections = -1; 21 | 22 | public RssMaxConnectionsException(int currentConnections, int maxConnections, String message) { 23 | super(message); 24 | this.message = message; 25 | this.currentConnections = currentConnections; 26 | this.maxConnections = maxConnections; 27 | } 28 | 29 | public int getCurrentConnections() { 30 | return currentConnections; 31 | } 32 | 33 | public int getMaxConnections() { 34 | return maxConnections; 35 | } 36 | 37 | @Override 38 | public String getMessage() { 39 | return String.format("%s, current: %s, max: %s", this.message, this.currentConnections, this.maxConnections); 40 | } 41 | 42 | @Override 43 | public String toString() { 44 | return "RssMaxConnectionsException{" + 45 | "currentConnections=" + currentConnections + 46 | ", maxConnections=" + maxConnections + 47 | "} " + super.toString(); 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssMissingShuffleWriteConfigException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssMissingShuffleWriteConfigException extends RssException { 18 | 19 | public RssMissingShuffleWriteConfigException(String message) { 20 | super(message); 21 | } 22 | 23 | public RssMissingShuffleWriteConfigException(String message, Throwable cause) { 24 | super(message, cause); 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssNetworkException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssNetworkException extends RssException { 18 | public RssNetworkException() { 19 | } 20 | 21 | public RssNetworkException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssNetworkException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | 29 | public RssNetworkException(Throwable cause) { 30 | super(cause); 31 | } 32 | 33 | public RssNetworkException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { 34 | super(message, cause, enableSuppression, writableStackTrace); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssNoActiveReadClientException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssNoActiveReadClientException extends RssException { 18 | public RssNoActiveReadClientException() { 19 | } 20 | 21 | public RssNoActiveReadClientException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssNoActiveReadClientException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssNoServerAvailableException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssNoServerAvailableException extends RssException { 18 | public RssNoServerAvailableException() { 19 | } 20 | 21 | public RssNoServerAvailableException(String message) { 22 | super(message); 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssNonRecoverableException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssNonRecoverableException extends RssException { 18 | public RssNonRecoverableException() { 19 | } 20 | 21 | public RssNonRecoverableException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssNonRecoverableException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssOperationQueueFullException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssOperationQueueFullException extends RssException { 18 | public RssOperationQueueFullException() { 19 | } 20 | 21 | public RssOperationQueueFullException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssOperationQueueFullException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | 29 | public RssOperationQueueFullException(Throwable cause) { 30 | super(cause); 31 | } 32 | 33 | public RssOperationQueueFullException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { 34 | super(message, cause, enableSuppression, writableStackTrace); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssQueueNotReadyException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssQueueNotReadyException extends RssException { 18 | public RssQueueNotReadyException() { 19 | } 20 | 21 | public RssQueueNotReadyException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssQueueNotReadyException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssRetryTimeoutException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssRetryTimeoutException extends RssException { 18 | public RssRetryTimeoutException() { 19 | } 20 | 21 | public RssRetryTimeoutException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssRetryTimeoutException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssRetryableSparkTaskException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssRetryableSparkTaskException extends RssException { 18 | public RssRetryableSparkTaskException() { 19 | } 20 | 21 | public RssRetryableSparkTaskException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssRetryableSparkTaskException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssServerBusyException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssServerBusyException extends RssException { 18 | public RssServerBusyException() { 19 | } 20 | 21 | public RssServerBusyException(String message) { 22 | super(message); 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssServerDownException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssServerDownException extends RssException { 18 | public RssServerDownException() { 19 | } 20 | 21 | public RssServerDownException(String message) { 22 | super(message); 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssServerResolveException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssServerResolveException extends RssException { 18 | public RssServerResolveException() { 19 | } 20 | 21 | public RssServerResolveException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssServerResolveException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssShuffleCorruptedException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssShuffleCorruptedException extends RssException { 18 | 19 | public RssShuffleCorruptedException(String message) { 20 | super(message); 21 | } 22 | 23 | public RssShuffleCorruptedException(String message, Throwable cause) { 24 | super(message, cause); 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssShuffleDataNotAvailableException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssShuffleDataNotAvailableException extends RssException { 18 | 19 | public RssShuffleDataNotAvailableException(String message) { 20 | super(message); 21 | } 22 | 23 | public RssShuffleDataNotAvailableException(String message, Throwable cause) { 24 | super(message, cause); 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssShuffleStageNotStartedException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssShuffleStageNotStartedException extends RssException { 18 | 19 | public RssShuffleStageNotStartedException(String message) { 20 | super(message); 21 | } 22 | 23 | public RssShuffleStageNotStartedException(String message, Throwable cause) { 24 | super(message, cause); 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssStaleTaskAttemptException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssStaleTaskAttemptException extends RssException { 18 | 19 | public RssStaleTaskAttemptException(String message) { 20 | super(message); 21 | } 22 | 23 | public RssStaleTaskAttemptException(String message, Throwable cause) { 24 | super(message, cause); 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssStreamReadException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssStreamReadException extends RssException { 18 | public RssStreamReadException() { 19 | } 20 | 21 | public RssStreamReadException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssStreamReadException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | 29 | public RssStreamReadException(Throwable cause) { 30 | super(cause); 31 | } 32 | 33 | public RssStreamReadException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { 34 | super(message, cause, enableSuppression, writableStackTrace); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssTooMuchDataException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssTooMuchDataException extends RssException { 18 | public RssTooMuchDataException() { 19 | } 20 | 21 | public RssTooMuchDataException(String message) { 22 | super(message); 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssUberEnvironmentException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssUberEnvironmentException extends RssException { 18 | public RssUberEnvironmentException() { 19 | } 20 | 21 | public RssUberEnvironmentException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssUberEnvironmentException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssUnsupportedCompressionException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssUnsupportedCompressionException extends RssException { 18 | public RssUnsupportedCompressionException() { 19 | } 20 | 21 | public RssUnsupportedCompressionException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssUnsupportedCompressionException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | 29 | public RssUnsupportedCompressionException(Throwable cause) { 30 | super(cause); 31 | } 32 | 33 | public RssUnsupportedCompressionException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { 34 | super(message, cause, enableSuppression, writableStackTrace); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/exceptions/RssWriteRecordException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.exceptions; 16 | 17 | public class RssWriteRecordException extends RssException { 18 | public RssWriteRecordException() { 19 | } 20 | 21 | public RssWriteRecordException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssWriteRecordException(String message, Throwable cause) { 26 | super(message, cause); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/execution/ShuffleDataWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.execution; 16 | 17 | import com.uber.rss.common.AppShuffleId; 18 | import io.netty.buffer.ByteBuf; 19 | 20 | /*** 21 | * This class wraps an operation to write a shuffle data record. 22 | */ 23 | public class ShuffleDataWrapper { 24 | private final AppShuffleId shuffleId; 25 | private final int mapId; 26 | private final long taskAttemptId; 27 | private final int partition; 28 | private final ByteBuf bytes; 29 | 30 | public ShuffleDataWrapper(AppShuffleId shuffleId, 31 | int mapId, 32 | long taskAttemptId, 33 | int partition, 34 | ByteBuf bytes) { 35 | this.shuffleId = shuffleId; 36 | this.mapId = mapId; 37 | this.taskAttemptId = taskAttemptId; 38 | this.partition = partition; 39 | this.bytes = bytes; 40 | } 41 | 42 | public AppShuffleId getShuffleId() { 43 | return shuffleId; 44 | } 45 | 46 | public int getMapId() { 47 | return mapId; 48 | } 49 | 50 | public long getTaskAttemptId() { 51 | return taskAttemptId; 52 | } 53 | 54 | public int getPartition() { 55 | return partition; 56 | } 57 | 58 | public ByteBuf getBytes() { 59 | return bytes; 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/execution/StagePersistentInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.execution; 16 | 17 | import com.uber.rss.clients.ShuffleWriteConfig; 18 | 19 | public class StagePersistentInfo { 20 | private final int numPartitions; 21 | private final int fileStartIndex; 22 | 23 | private final ShuffleWriteConfig shuffleWriteConfig; 24 | 25 | private final byte fileStatus; 26 | 27 | public StagePersistentInfo(int numPartitions, int fileStartIndex, ShuffleWriteConfig shuffleWriteConfig, byte fileStatus) { 28 | this.numPartitions = numPartitions; 29 | this.fileStartIndex = fileStartIndex; 30 | this.shuffleWriteConfig = shuffleWriteConfig; 31 | this.fileStatus = fileStatus; 32 | } 33 | 34 | public int getNumPartitions() { 35 | return numPartitions; 36 | } 37 | 38 | public int getFileStartIndex() { 39 | return fileStartIndex; 40 | } 41 | 42 | public ShuffleWriteConfig getShuffleWriteConfig() { 43 | return shuffleWriteConfig; 44 | } 45 | 46 | public byte getFileStatus() { 47 | return fileStatus; 48 | } 49 | 50 | @Override 51 | public String toString() { 52 | return "StagePersistentInfo{" + 53 | ", numPartitions=" + numPartitions + 54 | ", fileStartIndex=" + fileStartIndex + 55 | ", shuffleWriteConfig=" + shuffleWriteConfig + 56 | ", fileStatus=" + fileStatus + 57 | '}'; 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/execution/StateStore.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.execution; 16 | 17 | import com.uber.rss.common.AppShuffleId; 18 | import com.uber.rss.common.PartitionFilePathAndLength; 19 | 20 | import java.util.Collection; 21 | 22 | public interface StateStore extends AutoCloseable { 23 | 24 | void storeStageInfo(AppShuffleId appShuffleId, StagePersistentInfo info); 25 | 26 | void storeTaskAttemptCommit(AppShuffleId appShuffleId, 27 | Collection committedTaskAttempts, 28 | Collection partitionFilePathAndLengths); 29 | 30 | void storeAppDeletion(String appId); 31 | 32 | void storeStageCorruption(AppShuffleId appShuffleId); 33 | 34 | void commit(); 35 | 36 | LocalFileStateStoreIterator loadData(); 37 | 38 | void close(); 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/execution/TaskAttemptCollection.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.execution; 16 | 17 | import com.uber.rss.common.AppMapId; 18 | 19 | import java.util.Comparator; 20 | import java.util.HashMap; 21 | import java.util.List; 22 | import java.util.Map; 23 | import java.util.stream.Collectors; 24 | 25 | /*** 26 | * This class contains a collection of task attempts and their states. 27 | */ 28 | public class TaskAttemptCollection { 29 | private Map tasks = new HashMap<>(); 30 | 31 | public TaskAttemptCollection() { 32 | } 33 | 34 | public TaskAttemptIdAndState getTask(Long taskAttemptId) { 35 | TaskAttemptIdAndState task = tasks.get(taskAttemptId); 36 | if (task == null) { 37 | task = new TaskAttemptIdAndState(taskAttemptId); 38 | tasks.put(taskAttemptId, task); 39 | } 40 | return task; 41 | } 42 | 43 | public List getCommittedTaskIds() { 44 | return tasks.values().stream().filter(t->t.isCommitted()).map(t->t.getTaskAttemptId()).collect(Collectors.toList()); 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/handlers/ChannelFutureCloseListener.java: -------------------------------------------------------------------------------- 1 | package com.uber.rss.handlers; 2 | 3 | import io.netty.channel.ChannelFuture; 4 | import io.netty.channel.ChannelFutureListener; 5 | import org.slf4j.Logger; 6 | import org.slf4j.LoggerFactory; 7 | 8 | public class ChannelFutureCloseListener implements ChannelFutureListener { 9 | private static final Logger logger = LoggerFactory.getLogger(ChannelFutureCloseListener.class); 10 | 11 | private final String connectionInfo; 12 | 13 | public ChannelFutureCloseListener(String connectionInfo) { 14 | this.connectionInfo = connectionInfo; 15 | } 16 | 17 | @Override 18 | public void operationComplete(ChannelFuture future) throws Exception { 19 | logger.info("Closing connection {} {}", connectionInfo, System.nanoTime()); 20 | future.channel().close(); 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/handlers/ResponseStatusAndMessage.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.handlers; 16 | 17 | /*** 18 | * This class contains response status and response message. 19 | * @param 20 | */ 21 | public class ResponseStatusAndMessage { 22 | private byte status; 23 | private T message; 24 | 25 | public ResponseStatusAndMessage(byte status, T message) { 26 | this.status = status; 27 | this.message = message; 28 | } 29 | 30 | public byte getStatus() { 31 | return status; 32 | } 33 | 34 | public T getMessage() { 35 | return message; 36 | } 37 | 38 | @Override 39 | public String toString() { 40 | return "ResponseStatusAndMessage{" + 41 | "status=" + status + 42 | ", message=" + message + 43 | '}'; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/messages/AppDeletionStateItem.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.messages; 16 | 17 | import com.uber.rss.util.ByteBufUtils; 18 | import io.netty.buffer.ByteBuf; 19 | 20 | public class AppDeletionStateItem extends BaseMessage { 21 | private final String appId; 22 | 23 | public AppDeletionStateItem(String appId) { 24 | this.appId = appId; 25 | } 26 | 27 | @Override 28 | public int getMessageType() { 29 | return MessageConstants.MESSAGE_AppDeletionStateItem; 30 | } 31 | 32 | @Override 33 | public void serialize(ByteBuf buf) { 34 | ByteBufUtils.writeLengthAndString(buf, appId); 35 | } 36 | 37 | public static AppDeletionStateItem deserialize(ByteBuf buf) { 38 | String appId = ByteBufUtils.readLengthAndString(buf); 39 | return new AppDeletionStateItem(appId); 40 | } 41 | 42 | public String getAppId() { 43 | return appId; 44 | } 45 | @Override 46 | public String toString() { 47 | return "AppDeletionStateItem{" + 48 | "appId=" + appId + 49 | '}'; 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/messages/BaseMessage.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.messages; 16 | 17 | public abstract class BaseMessage extends SerializableMessage { 18 | public abstract int getMessageType(); 19 | } 20 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/messages/ConnectNotifyRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.messages; 16 | 17 | import com.uber.rss.util.ByteBufUtils; 18 | import io.netty.buffer.ByteBuf; 19 | 20 | public class ConnectNotifyRequest extends BaseMessage { 21 | private String user; 22 | 23 | public ConnectNotifyRequest(String user) { 24 | this.user = user; 25 | } 26 | 27 | @Override 28 | public int getMessageType() { 29 | return MessageConstants.MESSAGE_ConnectNotifyRequest; 30 | } 31 | 32 | @Override 33 | public void serialize(ByteBuf buf) { 34 | ByteBufUtils.writeLengthAndString(buf, user); 35 | } 36 | 37 | public static ConnectNotifyRequest deserialize(ByteBuf buf) { 38 | String user = ByteBufUtils.readLengthAndString(buf); 39 | return new ConnectNotifyRequest(user); 40 | } 41 | 42 | public String getUser() { 43 | return user; 44 | } 45 | 46 | @Override 47 | public String toString() { 48 | return "ConnectNotifyRequest{" + 49 | "user='" + user + '\'' + 50 | '}'; 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/messages/ConnectNotifyResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.messages; 16 | 17 | import com.uber.rss.util.ByteBufUtils; 18 | import io.netty.buffer.ByteBuf; 19 | 20 | public class ConnectNotifyResponse extends ServerResponseMessage { 21 | private String serverId; 22 | 23 | public ConnectNotifyResponse(String serverId) { 24 | this.serverId = serverId; 25 | } 26 | 27 | @Override 28 | public int getMessageType() { 29 | return MessageConstants.MESSAGE_ConnectNotifyResponse; 30 | } 31 | 32 | @Override 33 | public void serialize(ByteBuf buf) { 34 | ByteBufUtils.writeLengthAndString(buf, serverId); 35 | } 36 | 37 | public static ConnectNotifyResponse deserialize(ByteBuf buf) { 38 | String serverId = ByteBufUtils.readLengthAndString(buf); 39 | return new ConnectNotifyResponse(serverId); 40 | } 41 | 42 | public String getServerId() { 43 | return serverId; 44 | } 45 | 46 | @Override 47 | public String toString() { 48 | return "ConnectNotifyResponse{" + 49 | "serverId='" + serverId + '\'' + 50 | '}'; 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/messages/ConnectRegistryRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.messages; 16 | 17 | import com.uber.rss.util.ByteBufUtils; 18 | import io.netty.buffer.ByteBuf; 19 | 20 | public class ConnectRegistryRequest extends BaseMessage { 21 | private String user; 22 | 23 | public ConnectRegistryRequest(String user) { 24 | this.user = user; 25 | } 26 | 27 | @Override 28 | public int getMessageType() { 29 | return MessageConstants.MESSAGE_ConnectRegistryRequest; 30 | } 31 | 32 | @Override 33 | public void serialize(ByteBuf buf) { 34 | ByteBufUtils.writeLengthAndString(buf, user); 35 | } 36 | 37 | public static ConnectRegistryRequest deserialize(ByteBuf buf) { 38 | String user = ByteBufUtils.readLengthAndString(buf); 39 | return new ConnectRegistryRequest(user); 40 | } 41 | 42 | public String getUser() { 43 | return user; 44 | } 45 | 46 | @Override 47 | public String toString() { 48 | return "ConnectRegistryRequest{" + 49 | "user='" + user + '\'' + 50 | '}'; 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/messages/ConnectRegistryResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.messages; 16 | 17 | import com.uber.rss.util.ByteBufUtils; 18 | import io.netty.buffer.ByteBuf; 19 | 20 | public class ConnectRegistryResponse extends ServerResponseMessage { 21 | private String serverId; 22 | 23 | public ConnectRegistryResponse(String serverId) { 24 | this.serverId = serverId; 25 | } 26 | 27 | @Override 28 | public int getMessageType() { 29 | return MessageConstants.MESSAGE_ConnectRegistryResponse; 30 | } 31 | 32 | @Override 33 | public void serialize(ByteBuf buf) { 34 | ByteBufUtils.writeLengthAndString(buf, serverId); 35 | } 36 | 37 | public static ConnectRegistryResponse deserialize(ByteBuf buf) { 38 | String serverId = ByteBufUtils.readLengthAndString(buf); 39 | return new ConnectRegistryResponse(serverId); 40 | } 41 | 42 | public String getServerId() { 43 | return serverId; 44 | } 45 | 46 | @Override 47 | public String toString() { 48 | return "ConnectRegistryResponse{" + 49 | "serverId='" + serverId + '\'' + 50 | '}'; 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/messages/ControlMessage.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.messages; 16 | 17 | public abstract class ControlMessage extends BaseMessage { 18 | } 19 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/messages/GetBusyStatusRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.messages; 16 | 17 | import com.uber.rss.util.ByteBufUtils; 18 | import io.netty.buffer.ByteBuf; 19 | 20 | public class GetBusyStatusRequest extends BaseMessage { 21 | private String user; 22 | 23 | public GetBusyStatusRequest(String user) { 24 | this.user = user; 25 | } 26 | 27 | @Override 28 | public int getMessageType() { 29 | return MessageConstants.MESSAGE_GetBusyStatusRequest; 30 | } 31 | 32 | @Override 33 | public void serialize(ByteBuf buf) { 34 | ByteBufUtils.writeLengthAndString(buf, user); 35 | } 36 | 37 | public static GetBusyStatusRequest deserialize(ByteBuf buf) { 38 | String user = ByteBufUtils.readLengthAndString(buf); 39 | return new GetBusyStatusRequest(user); 40 | } 41 | 42 | public String getUser() { 43 | return user; 44 | } 45 | 46 | @Override 47 | public String toString() { 48 | return "GetBusyStatusRequest{" + 49 | "user='" + user + '\'' + 50 | '}'; 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/messages/GetDataAvailabilityRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.messages; 16 | 17 | import io.netty.buffer.ByteBuf; 18 | 19 | /*** 20 | * Message sent by the client to check shuffle data availability 21 | */ 22 | public class GetDataAvailabilityRequest extends BaseMessage { 23 | 24 | public GetDataAvailabilityRequest() { 25 | } 26 | 27 | @Override 28 | public int getMessageType() { 29 | return MessageConstants.MESSAGE_GetDataAvailabilityRequest; 30 | } 31 | 32 | @Override 33 | public void serialize(ByteBuf buf) { 34 | } 35 | 36 | public static GetDataAvailabilityRequest deserialize(ByteBuf buf) { 37 | return new GetDataAvailabilityRequest(); 38 | } 39 | 40 | @Override 41 | public String toString() { 42 | return "GetDataAvailabilityRequest{" + 43 | '}'; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/messages/GetDataAvailabilityRequestMessage.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.messages; 16 | 17 | import io.netty.buffer.ByteBuf; 18 | 19 | /*** 20 | * Message sent by the client to check shuffle data availability 21 | */ 22 | public class GetDataAvailabilityRequestMessage extends BaseMessage { 23 | 24 | public GetDataAvailabilityRequestMessage() { 25 | } 26 | 27 | @Override 28 | public int getMessageType() { 29 | return MessageConstants.MESSAGE_GetDataAvailabilityRequestMessage; 30 | } 31 | 32 | @Override 33 | public void serialize(ByteBuf buf) { 34 | } 35 | 36 | public static GetDataAvailabilityRequest deserialize(ByteBuf buf) { 37 | return new GetDataAvailabilityRequest(); 38 | } 39 | 40 | @Override 41 | public String toString() { 42 | return "GetDataAvailabilityRequest{" + 43 | '}'; 44 | } 45 | } -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/messages/RegisterServerResponseMessage.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.messages; 16 | 17 | import com.uber.rss.util.ByteBufUtils; 18 | import io.netty.buffer.ByteBuf; 19 | 20 | public class RegisterServerResponseMessage extends ServerResponseMessage { 21 | private String serverId; 22 | 23 | public RegisterServerResponseMessage(String serverId) { 24 | this.serverId = serverId; 25 | } 26 | 27 | @Override 28 | public int getMessageType() { 29 | return MessageConstants.MESSAGE_RegisterServerResponse; 30 | } 31 | 32 | @Override 33 | public void serialize(ByteBuf buf) { 34 | ByteBufUtils.writeLengthAndString(buf, serverId); 35 | } 36 | 37 | public static RegisterServerResponseMessage deserialize(ByteBuf buf) { 38 | String serverId = ByteBufUtils.readLengthAndString(buf); 39 | return new RegisterServerResponseMessage(serverId); 40 | } 41 | 42 | public String getServerId() { 43 | return serverId; 44 | } 45 | 46 | @Override 47 | public String toString() { 48 | return "RegisterServerResponseMessage{" + 49 | "serverId='" + serverId + '\'' + 50 | '}'; 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/messages/SerializableMessage.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.messages; 16 | 17 | import io.netty.buffer.ByteBuf; 18 | 19 | public abstract class SerializableMessage { 20 | public abstract void serialize(ByteBuf buf); 21 | } 22 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/messages/ServerResponseMessage.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.messages; 16 | 17 | /** 18 | * Base class used for all server response messages 19 | */ 20 | public abstract class ServerResponseMessage extends BaseMessage { 21 | protected byte status; 22 | 23 | public ServerResponseMessage() { 24 | this.status = MessageConstants.RESPONSE_STATUS_UNSPECIFIED; 25 | } 26 | 27 | public ServerResponseMessage(byte status) { 28 | this.status = status; 29 | } 30 | 31 | public byte getStatus() { 32 | return status; 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/messages/ShuffleDataWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.messages; 16 | 17 | /*** 18 | * This class wraps a chunk of data inside the shuffle file. The data (bytes) should 19 | * be written to shuffle file directly. 20 | */ 21 | public class ShuffleDataWrapper { 22 | private final int partitionId; 23 | private final long taskAttemptId; 24 | private final byte[] bytes; 25 | 26 | public ShuffleDataWrapper(int partitionId, long taskAttemptId, byte[] bytes) { 27 | if (bytes == null) { 28 | throw new NullPointerException("bytes"); 29 | } 30 | 31 | this.partitionId = partitionId; 32 | this.taskAttemptId = taskAttemptId; 33 | this.bytes = bytes; 34 | } 35 | 36 | public int getPartitionId() { 37 | return partitionId; 38 | } 39 | 40 | public long getTaskAttemptId() { 41 | return taskAttemptId; 42 | } 43 | 44 | public byte[] getBytes() { 45 | return bytes; 46 | } 47 | 48 | @Override 49 | public String toString() { 50 | return "ShuffleDataWrapper{" + 51 | "partitionId=" + partitionId + 52 | ", taskAttemptId=" + taskAttemptId + 53 | ", bytes.length=" + bytes.length + 54 | '}'; 55 | } 56 | } 57 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metadata/ZooKeeperServerNodeData.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metadata; 16 | 17 | public class ZooKeeperServerNodeData { 18 | private String runningVersion; 19 | private String connectionString; 20 | 21 | public String getRunningVersion() { 22 | return runningVersion; 23 | } 24 | 25 | public void setRunningVersion(String runningVersion) { 26 | this.runningVersion = runningVersion; 27 | } 28 | 29 | public String getConnectionString() { 30 | return connectionString; 31 | } 32 | 33 | public void setConnectionString(String connectionString) { 34 | this.connectionString = connectionString; 35 | } 36 | 37 | @Override 38 | public String toString() { 39 | return "ZooKeeperServerNodeData{" + 40 | "runningVersion='" + runningVersion + '\'' + 41 | ", connectionString='" + connectionString + '\'' + 42 | '}'; 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/ApplicationJobStatusMetrics.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import com.uber.m3.tally.Counter; 18 | import com.uber.m3.tally.Scope; 19 | 20 | import java.util.HashMap; 21 | import java.util.Map; 22 | 23 | public class ApplicationJobStatusMetrics extends MetricGroup { 24 | 25 | private final Counter numApplicationJobs; 26 | private final Counter numRssExceptionJobs; 27 | 28 | public ApplicationJobStatusMetrics(ApplicationJobStatusMetricsKey key) { 29 | super(key); 30 | 31 | this.numApplicationJobs = scope.counter("numApplicationJobs2"); 32 | this.numRssExceptionJobs = scope.counter("numRssExceptionJobs"); 33 | } 34 | 35 | public Counter getNumApplicationJobs() { 36 | return numApplicationJobs; 37 | } 38 | 39 | public Counter getNumRssExceptionJobs() { 40 | return numRssExceptionJobs; 41 | } 42 | 43 | @Override 44 | protected Scope createScope(ApplicationJobStatusMetricsKey key) { 45 | Map tags = new HashMap<>(); 46 | tags.put(M3Stats.TAG_NAME_USER, key.getUser()); 47 | tags.put(M3Stats.TAG_NAME_JOB_STATUS, key.getJobStatus()); 48 | 49 | return M3Stats.createSubScope(tags); 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/ApplicationJobStatusMetricsKey.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import java.util.Objects; 18 | 19 | public class ApplicationJobStatusMetricsKey { 20 | private String user; 21 | private String jobStatus; 22 | 23 | public ApplicationJobStatusMetricsKey(String user, String jobStatus) { 24 | this.user = user; 25 | this.jobStatus = jobStatus; 26 | } 27 | 28 | public String getUser() { 29 | return user; 30 | } 31 | 32 | public String getJobStatus() { 33 | return jobStatus; 34 | } 35 | 36 | @Override 37 | public boolean equals(Object o) { 38 | if (this == o) return true; 39 | if (o == null || getClass() != o.getClass()) return false; 40 | ApplicationJobStatusMetricsKey that = (ApplicationJobStatusMetricsKey) o; 41 | return Objects.equals(user, that.user) && 42 | Objects.equals(jobStatus, that.jobStatus); 43 | } 44 | 45 | @Override 46 | public int hashCode() { 47 | return Objects.hash(user, jobStatus); 48 | } 49 | 50 | @Override 51 | public String toString() { 52 | return "ApplicationJobStatusMetricsKey{" + 53 | "user='" + user + '\'' + 54 | ", jobStatus='" + jobStatus + '\'' + 55 | '}'; 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/ApplicationMetrics.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import com.uber.m3.tally.Counter; 18 | import com.uber.m3.tally.Scope; 19 | 20 | import java.util.HashMap; 21 | import java.util.Map; 22 | 23 | public class ApplicationMetrics extends MetricGroup { 24 | 25 | private final Counter numApplications; 26 | 27 | public ApplicationMetrics(ApplicationMetricsKey key) { 28 | super(key); 29 | 30 | this.numApplications = scope.counter("numApplications"); 31 | } 32 | 33 | public Counter getNumApplications() { 34 | return numApplications; 35 | } 36 | 37 | @Override 38 | protected Scope createScope(ApplicationMetricsKey key) { 39 | Map tags = new HashMap<>(); 40 | tags.put(M3Stats.TAG_NAME_USER, key.getUser()); 41 | tags.put(M3Stats.TAG_NAME_ATTEMPT_ID, key.getAttemptId()); 42 | return M3Stats.createSubScope(tags); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/ApplicationMetricsKey.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import java.util.Objects; 18 | 19 | public class ApplicationMetricsKey { 20 | private String user; 21 | private String attemptId; 22 | 23 | public ApplicationMetricsKey(String user, String attemptId) { 24 | this.user = user; 25 | this.attemptId = attemptId; 26 | } 27 | 28 | public String getUser() { 29 | return user; 30 | } 31 | 32 | public String getAttemptId() { 33 | return attemptId; 34 | } 35 | 36 | @Override 37 | public boolean equals(Object o) { 38 | if (this == o) return true; 39 | if (o == null || getClass() != o.getClass()) return false; 40 | ApplicationMetricsKey that = (ApplicationMetricsKey) o; 41 | return Objects.equals(user, that.user) && 42 | Objects.equals(attemptId, that.attemptId); 43 | } 44 | 45 | @Override 46 | public int hashCode() { 47 | 48 | return Objects.hash(user, attemptId); 49 | } 50 | 51 | @Override 52 | public String toString() { 53 | return "ApplicationMetricsKey{" + 54 | "user='" + user + '\'' + 55 | ", attemptId='" + attemptId + '\'' + 56 | '}'; 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/ClientConnectMetrics.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import com.uber.m3.tally.Gauge; 18 | import com.uber.m3.tally.Scope; 19 | import com.uber.m3.tally.Timer; 20 | 21 | import java.util.HashMap; 22 | import java.util.Map; 23 | 24 | public class ClientConnectMetrics extends MetricGroup { 25 | 26 | private final Timer socketConnectLatency; 27 | private final Gauge socketConnectRetries; 28 | 29 | public ClientConnectMetrics(ClientConnectMetricsKey key) { 30 | super(key); 31 | 32 | this.socketConnectLatency = scope.timer("socketConnectLatency"); 33 | this.socketConnectRetries = scope.gauge("socketConnectRetries"); 34 | } 35 | 36 | public Timer getSocketConnectLatency() { 37 | return socketConnectLatency; 38 | } 39 | 40 | public Gauge getSocketConnectRetries() { 41 | return socketConnectRetries; 42 | } 43 | 44 | @Override 45 | protected Scope createScope(ClientConnectMetricsKey key) { 46 | Map tags = new HashMap<>(); 47 | tags.put(M3Stats.TAG_NAME_SOURCE, key.getSource()); 48 | tags.put(M3Stats.TAG_NAME_REMOTE, key.getRemote()); 49 | return M3Stats.createSubScope(tags); 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/ClientConnectMetricsKey.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import java.util.Objects; 18 | 19 | public class ClientConnectMetricsKey { 20 | private String source; 21 | private String remote; 22 | 23 | public ClientConnectMetricsKey(String source, String remote) { 24 | this.source = source; 25 | this.remote = remote; 26 | } 27 | 28 | public String getSource() { 29 | return source; 30 | } 31 | 32 | public String getRemote() { 33 | return remote; 34 | } 35 | 36 | @Override 37 | public boolean equals(Object o) { 38 | if (this == o) return true; 39 | if (o == null || getClass() != o.getClass()) return false; 40 | ClientConnectMetricsKey that = (ClientConnectMetricsKey) o; 41 | return Objects.equals(source, that.source) && 42 | Objects.equals(remote, that.remote); 43 | } 44 | 45 | @Override 46 | public int hashCode() { 47 | return Objects.hash(source, remote); 48 | } 49 | 50 | @Override 51 | public String toString() { 52 | return "ClientConnectMetricsKey{" + 53 | "source='" + source + '\'' + 54 | ", remote='" + remote + '\'' + 55 | '}'; 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/ExceptionMetricGroupContainer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | public class ExceptionMetricGroupContainer { 18 | private MetricGroupContainer metricGroupContainer; 19 | 20 | public ExceptionMetricGroupContainer() { 21 | this.metricGroupContainer = new MetricGroupContainer<>( 22 | t->new ExceptionMetrics(t.getExceptionName(), t.getExceptionSource())); 23 | } 24 | 25 | public ExceptionMetrics getMetricGroup(Throwable ex, String exceptionSource) { 26 | ExceptionMetricsKey tag = new ExceptionMetricsKey(ex.getClass().getSimpleName(), exceptionSource); 27 | return metricGroupContainer.getMetricGroup(tag); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/ExceptionMetrics.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import com.uber.m3.tally.Counter; 18 | import com.uber.m3.tally.Scope; 19 | import org.slf4j.Logger; 20 | import org.slf4j.LoggerFactory; 21 | 22 | import java.util.HashMap; 23 | import java.util.Map; 24 | 25 | public class ExceptionMetrics extends MetricGroup { 26 | private static final Logger logger = LoggerFactory.getLogger(ExceptionMetrics.class); 27 | 28 | private final Counter numExceptions; 29 | 30 | public ExceptionMetrics(String exceptionName, String exceptionSource) { 31 | super(new ExceptionMetricsKey(exceptionName, exceptionSource)); 32 | this.numExceptions = scope.counter("numExceptions"); 33 | } 34 | 35 | public Counter getNumExceptions() { 36 | return numExceptions; 37 | } 38 | 39 | @Override 40 | public void close() { 41 | M3Stats.decreaseNumM3Scopes(); 42 | } 43 | 44 | @Override 45 | protected Scope createScope(ExceptionMetricsKey key) { 46 | Map tags = new HashMap<>(); 47 | tags.put("exception", key.getExceptionName()); 48 | tags.put("source", key.getExceptionSource()); 49 | return M3Stats.createSubScope(tags); 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/ExceptionMetricsKey.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import java.util.Objects; 18 | 19 | public class ExceptionMetricsKey { 20 | private final String exceptionName; 21 | private final String exceptionSource; 22 | 23 | public ExceptionMetricsKey(String exceptionName, String exceptionSource) { 24 | this.exceptionName = exceptionName; 25 | this.exceptionSource = exceptionSource; 26 | } 27 | 28 | public String getExceptionName() { 29 | return exceptionName; 30 | } 31 | 32 | public String getExceptionSource() { 33 | return exceptionSource; 34 | } 35 | 36 | @Override 37 | public boolean equals(Object o) { 38 | if (this == o) return true; 39 | if (o == null || getClass() != o.getClass()) return false; 40 | ExceptionMetricsKey that = (ExceptionMetricsKey) o; 41 | return Objects.equals(exceptionName, that.exceptionName) && 42 | Objects.equals(exceptionSource, that.exceptionSource); 43 | } 44 | 45 | @Override 46 | public int hashCode() { 47 | return Objects.hash(exceptionName, exceptionSource); 48 | } 49 | 50 | @Override 51 | public String toString() { 52 | return "ExceptionMetricsKey{" + 53 | "exceptionName='" + exceptionName + '\'' + 54 | ", exceptionSource='" + exceptionSource + '\'' + 55 | '}'; 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/M3DummyScopeBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import com.uber.m3.tally.Buckets; 18 | import com.uber.m3.tally.RootScopeBuilder; 19 | import com.uber.m3.tally.Scope; 20 | import com.uber.m3.tally.ScopeBuilder; 21 | import com.uber.m3.tally.StatsReporter; 22 | import com.uber.m3.util.Duration; 23 | import com.uber.m3.util.ImmutableMap; 24 | 25 | import java.util.Map; 26 | 27 | public class M3DummyScopeBuilder extends RootScopeBuilder { 28 | @Override 29 | public ScopeBuilder reporter(StatsReporter reporter) { 30 | return this; 31 | } 32 | 33 | @Override 34 | public ScopeBuilder prefix(String prefix) { 35 | return this; 36 | } 37 | 38 | @Override 39 | public ScopeBuilder separator(String separator) { 40 | return this; 41 | } 42 | 43 | @Override 44 | public ScopeBuilder tags(Map tags) { 45 | return this; 46 | } 47 | 48 | @Override 49 | public ScopeBuilder tags(ImmutableMap tags) { 50 | return this; 51 | } 52 | 53 | @SuppressWarnings({"rawtypes", "unchecked"}) 54 | @Override 55 | public ScopeBuilder defaultBuckets(Buckets defaultBuckets) { 56 | return this; 57 | } 58 | 59 | @Override 60 | public Scope reportEvery(Duration interval) { 61 | return new M3DummyScope(); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/MetadataClientMetricsContainer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | public class MetadataClientMetricsContainer { 18 | private MetricGroupContainer metricGroupContainer; 19 | 20 | public MetadataClientMetricsContainer() { 21 | this.metricGroupContainer = new MetricGroupContainer<>( 22 | t->new MetadataClientMetrics(t)); 23 | } 24 | 25 | public MetadataClientMetrics getMetricGroup(String client, String operation) { 26 | MetadataClientMetricsKey tag = new MetadataClientMetricsKey(client, operation); 27 | return metricGroupContainer.getMetricGroup(tag); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/MetricGroup.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import com.uber.m3.tally.Scope; 18 | 19 | public abstract class MetricGroup implements AutoCloseable { 20 | protected final K key; 21 | protected final Scope scope; 22 | 23 | public MetricGroup(K key) { 24 | this.key = key; 25 | this.scope = createScope(key); 26 | } 27 | 28 | abstract protected Scope createScope(K key); 29 | 30 | @Override 31 | public void close() { 32 | M3Stats.decreaseNumM3Scopes(); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/MetricGroupContainer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import java.util.concurrent.ConcurrentHashMap; 18 | import java.util.function.Function; 19 | 20 | public class MetricGroupContainer > { 21 | private final Function createFunction; 22 | 23 | private final ConcurrentHashMap metricGroups = new ConcurrentHashMap<>(); 24 | 25 | public MetricGroupContainer(Function createFunction) { 26 | this.createFunction = createFunction; 27 | } 28 | 29 | public M getMetricGroup(K key) { 30 | return metricGroups.computeIfAbsent(key, createFunction); 31 | } 32 | 33 | public void removeMetricGroup(K key) { 34 | M metricGroup = metricGroups.remove(key); 35 | if (metricGroup != null) { 36 | metricGroup.close(); 37 | } 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/NettyServerSideMetricGroupContainer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import java.util.function.Function; 18 | 19 | public class NettyServerSideMetricGroupContainer > { 20 | private MetricGroupContainer metricGroupContainer; 21 | 22 | public NettyServerSideMetricGroupContainer(Function createFunction) { 23 | this.metricGroupContainer = new MetricGroupContainer(createFunction); 24 | } 25 | 26 | public M getMetricGroup(String user) { 27 | return metricGroupContainer.getMetricGroup(new NettyServerSideMetricsKey(user)); 28 | } 29 | 30 | public void removeMetricGroup(String user) { 31 | metricGroupContainer.removeMetricGroup(new NettyServerSideMetricsKey(user)); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/NettyServerSideMetricsKey.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import java.util.Objects; 18 | 19 | public class NettyServerSideMetricsKey { 20 | private String user; 21 | 22 | public NettyServerSideMetricsKey(String user) { 23 | this.user = user; 24 | } 25 | 26 | public String getUser() { 27 | return user; 28 | } 29 | 30 | @Override 31 | public boolean equals(Object o) { 32 | if (this == o) return true; 33 | if (o == null || getClass() != o.getClass()) return false; 34 | NettyServerSideMetricsKey that = (NettyServerSideMetricsKey) o; 35 | return Objects.equals(user, that.user); 36 | } 37 | 38 | @Override 39 | public int hashCode() { 40 | return Objects.hash(user); 41 | } 42 | 43 | @Override 44 | public String toString() { 45 | return "NettyServerSideMetricsKey{" + 46 | "user='" + user + '\'' + 47 | '}'; 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/NotifyClientMetrics.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import com.uber.m3.tally.Counter; 18 | import com.uber.m3.tally.Scope; 19 | 20 | import java.util.HashMap; 21 | import java.util.Map; 22 | 23 | public class NotifyClientMetrics extends MetricGroup { 24 | 25 | private final Counter numClients; 26 | 27 | public NotifyClientMetrics(NotifyClientMetricsKey key) { 28 | super(key); 29 | 30 | this.numClients = scope.counter("numClients"); 31 | } 32 | 33 | public Counter getNumClients() { 34 | return numClients; 35 | } 36 | 37 | @Override 38 | protected Scope createScope(NotifyClientMetricsKey key) { 39 | Map tags = new HashMap<>(); 40 | tags.put(M3Stats.TAG_NAME_SOURCE, key.getSource()); 41 | tags.put(M3Stats.TAG_NAME_USER, key.getUser()); 42 | return M3Stats.createSubScope(tags); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/NotifyClientMetricsKey.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import java.util.Objects; 18 | 19 | public class NotifyClientMetricsKey { 20 | private String source; 21 | private String user; 22 | 23 | public NotifyClientMetricsKey(String source, String user) { 24 | this.source = source; 25 | this.user = user; 26 | } 27 | 28 | public String getSource() { 29 | return source; 30 | } 31 | 32 | public String getUser() { 33 | return user; 34 | } 35 | 36 | @Override 37 | public boolean equals(Object o) { 38 | if (this == o) return true; 39 | if (o == null || getClass() != o.getClass()) return false; 40 | NotifyClientMetricsKey that = (NotifyClientMetricsKey) o; 41 | return Objects.equals(source, that.source) && 42 | Objects.equals(user, that.user); 43 | } 44 | 45 | @Override 46 | public int hashCode() { 47 | return Objects.hash(source, user); 48 | } 49 | 50 | @Override 51 | public String toString() { 52 | return "NotifyClientMetricsKey{" + 53 | "source='" + source + '\'' + 54 | ", user='" + user + '\'' + 55 | '}'; 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/NotifyServerMetricsContainer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | public class NotifyServerMetricsContainer { 18 | private MetricGroupContainer applicationJobStatusMetricsContainer; 19 | 20 | private MetricGroupContainer applicationMetricsContainer; 21 | 22 | public NotifyServerMetricsContainer() { 23 | this.applicationJobStatusMetricsContainer = new MetricGroupContainer<>( 24 | t->new ApplicationJobStatusMetrics(t)); 25 | 26 | this.applicationMetricsContainer = new MetricGroupContainer<>( 27 | t->new ApplicationMetrics(t)); 28 | } 29 | 30 | public ApplicationJobStatusMetrics getApplicationJobStatusMetrics(String user, String jobStatus) { 31 | ApplicationJobStatusMetricsKey key = new ApplicationJobStatusMetricsKey(user, jobStatus); 32 | return applicationJobStatusMetricsContainer.getMetricGroup(key); 33 | } 34 | 35 | public ApplicationMetrics getApplicationMetrics(String user, String attemptId) { 36 | ApplicationMetricsKey key = new ApplicationMetricsKey(user, attemptId); 37 | return applicationMetricsContainer.getMetricGroup(key); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/ReadClientMetricsKey.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import java.util.Objects; 18 | 19 | public class ReadClientMetricsKey { 20 | private String source; 21 | private String user; 22 | 23 | public ReadClientMetricsKey(String source, String user) { 24 | this.source = source; 25 | this.user = user; 26 | } 27 | 28 | public String getSource() { 29 | return source; 30 | } 31 | 32 | public String getUser() { 33 | return user; 34 | } 35 | 36 | @Override 37 | public boolean equals(Object o) { 38 | if (this == o) return true; 39 | if (o == null || getClass() != o.getClass()) return false; 40 | ReadClientMetricsKey that = (ReadClientMetricsKey) o; 41 | return Objects.equals(source, that.source) && 42 | Objects.equals(user, that.user); 43 | } 44 | 45 | @Override 46 | public int hashCode() { 47 | return Objects.hash(source, user); 48 | } 49 | 50 | @Override 51 | public String toString() { 52 | return "ReadClientMetricsKey{" + 53 | "source='" + source + '\'' + 54 | ", user='" + user + '\'' + 55 | '}'; 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/ShuffleClientStageMetricsKey.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import java.util.Objects; 18 | 19 | public class ShuffleClientStageMetricsKey { 20 | private String user; 21 | private String queue; 22 | 23 | public ShuffleClientStageMetricsKey(String user, String queue) { 24 | this.user = user; 25 | this.queue = queue; 26 | } 27 | 28 | public String getUser() { 29 | return user; 30 | } 31 | 32 | public String getQueue() { 33 | return queue; 34 | } 35 | 36 | @Override 37 | public boolean equals(Object o) { 38 | if (this == o) return true; 39 | if (o == null || getClass() != o.getClass()) return false; 40 | ShuffleClientStageMetricsKey that = (ShuffleClientStageMetricsKey) o; 41 | return Objects.equals(user, that.user) && 42 | Objects.equals(queue, that.queue); 43 | } 44 | 45 | @Override 46 | public int hashCode() { 47 | return Objects.hash(user, queue); 48 | } 49 | 50 | @Override 51 | public String toString() { 52 | return "ShuffleClientStageMetricsKey{" + 53 | "user='" + user + '\'' + 54 | ", queue='" + queue + '\'' + 55 | '}'; 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/metrics/WriteClientMetricsKey.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.metrics; 16 | 17 | import java.util.Objects; 18 | 19 | public class WriteClientMetricsKey { 20 | private String source; 21 | private String user; 22 | 23 | public WriteClientMetricsKey(String source, String user) { 24 | this.source = source; 25 | this.user = user; 26 | } 27 | 28 | public String getSource() { 29 | return source; 30 | } 31 | 32 | public String getUser() { 33 | return user; 34 | } 35 | 36 | @Override 37 | public boolean equals(Object o) { 38 | if (this == o) return true; 39 | if (o == null || getClass() != o.getClass()) return false; 40 | WriteClientMetricsKey that = (WriteClientMetricsKey) o; 41 | return Objects.equals(source, that.source) && 42 | Objects.equals(user, that.user); 43 | } 44 | 45 | @Override 46 | public int hashCode() { 47 | return Objects.hash(source, user); 48 | } 49 | 50 | @Override 51 | public String toString() { 52 | return "WriteClientMetricsKey{" + 53 | "source='" + source + '\'' + 54 | ", user='" + user + '\'' + 55 | '}'; 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/storage/ShuffleFileUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.storage; 16 | 17 | import com.uber.rss.common.AppShuffleId; 18 | 19 | import java.nio.file.Paths; 20 | 21 | /*** 22 | * Utility methods for shuffle files. 23 | */ 24 | public class ShuffleFileUtils { 25 | public static final int MAX_SPLITS = 10000; 26 | 27 | public static String getShuffleFileName(int shuffleId, int partitionId) { 28 | return String.format("shuffle_%s_p_%s.data", shuffleId, partitionId); 29 | } 30 | 31 | public static String getShuffleFilePath(String rootDir, 32 | AppShuffleId appShuffleId, 33 | int partitionId) { 34 | String fileName = getShuffleFileName( 35 | appShuffleId.getShuffleId(), partitionId); 36 | String path = Paths.get( 37 | getAppShuffleDir(rootDir, appShuffleId.getAppId()), 38 | appShuffleId.getAppAttempt(), 39 | fileName).toString(); 40 | return path; 41 | } 42 | 43 | public static String getAppShuffleDir(String rootDir, String appId) { 44 | return Paths.get(rootDir, appId).toString(); 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/storage/ShuffleOutputStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.storage; 16 | 17 | /*** 18 | * Shuffle output stream interface. 19 | */ 20 | public interface ShuffleOutputStream extends AutoCloseable { 21 | /*** 22 | * Write data to the stream. 23 | * @param bytes 24 | */ 25 | void write(byte[] bytes); 26 | 27 | 28 | /*** 29 | * Close the stream. 30 | */ 31 | void close(); 32 | 33 | /*** 34 | * Get file location for this stream. 35 | * @return 36 | */ 37 | String getLocation(); 38 | 39 | long getWrittenBytes(); 40 | } 41 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/tools/StreamServerStressToolWrite64GB.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.tools; 16 | 17 | import com.uber.rss.metrics.M3Stats; 18 | import org.slf4j.Logger; 19 | import org.slf4j.LoggerFactory; 20 | 21 | /** 22 | * This tool repeatedly runs StreamServerStressTool with writing 64GB data which exceeds max integer value. 23 | */ 24 | public class StreamServerStressToolWrite64GB { 25 | private static final Logger logger = LoggerFactory.getLogger(StreamServerStressToolWrite64GB.class); 26 | 27 | public static void main(String[] args) { 28 | Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { 29 | @Override 30 | public void uncaughtException(Thread t, Throwable e) { 31 | logger.error(String.format("Got exception from thread %s", t.getName()), e); 32 | System.exit(-1); 33 | } 34 | }); 35 | 36 | StreamServerStressTool tool = new StreamServerStressTool(); 37 | tool.setNumBytes(64L * 1024 * 1024 * 1024); 38 | tool.setNumMaps(2); 39 | tool.setNumMapAttempts(1); 40 | tool.setNumPartitions(2); 41 | tool.setNumSplits(1); 42 | tool.setWriteClientQueueSize(1000); 43 | 44 | try { 45 | tool.run(); 46 | } finally { 47 | tool.cleanup(); 48 | } 49 | 50 | M3Stats.closeDefaultScope(); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/util/AsyncSocketState.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.util; 16 | 17 | import java.nio.ByteBuffer; 18 | import java.nio.channels.AsynchronousSocketChannel; 19 | import java.util.concurrent.ConcurrentLinkedQueue; 20 | 21 | public class AsyncSocketState { 22 | private AsynchronousSocketChannel socket; 23 | private ConcurrentLinkedQueue byteBuffers = new ConcurrentLinkedQueue<>(); 24 | 25 | public AsyncSocketState(AsynchronousSocketChannel socket) { 26 | this.socket = socket; 27 | } 28 | 29 | public AsynchronousSocketChannel getSocket() { 30 | return socket; 31 | } 32 | 33 | public void addBuffer(ByteBuffer byteBuffer) { 34 | byteBuffers.add(byteBuffer); 35 | } 36 | 37 | public ByteBuffer peekBuffer() { 38 | if (byteBuffers.isEmpty()) { 39 | return null; 40 | } 41 | 42 | return byteBuffers.peek(); 43 | } 44 | 45 | public ByteBuffer removeBuffer() { 46 | return byteBuffers.poll(); 47 | } 48 | 49 | @Override 50 | public String toString() { 51 | return "AsyncSocketState{" + 52 | "byteBuffers=" + byteBuffers.size() + 53 | '}'; 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/util/CountedOutputStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.util; 16 | 17 | import java.io.IOException; 18 | import java.io.OutputStream; 19 | 20 | public class CountedOutputStream extends OutputStream { 21 | private long writtenBytes = 0; 22 | 23 | private OutputStream underlyingStream; 24 | 25 | public CountedOutputStream(OutputStream underlyingStream) { 26 | this.underlyingStream = underlyingStream; 27 | } 28 | 29 | @Override 30 | public synchronized void write(int i) throws IOException { 31 | underlyingStream.write(i); 32 | writtenBytes++; 33 | } 34 | 35 | @Override 36 | public synchronized void write(byte[] b, int off, int len) throws IOException { 37 | underlyingStream.write(b, off, len); 38 | writtenBytes += len; 39 | } 40 | 41 | @Override 42 | public synchronized void flush() throws IOException { 43 | underlyingStream.flush(); 44 | } 45 | 46 | @Override 47 | public synchronized void close() throws IOException { 48 | underlyingStream.close(); 49 | } 50 | 51 | public synchronized long getWrittenBytes() { 52 | return writtenBytes; 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/util/LogUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.util; 16 | 17 | public class LogUtils { 18 | 19 | public static double calculateMegaBytesPerSecond(long durationMillis, long bytes) { 20 | if (durationMillis == 0) { 21 | return 0; 22 | } 23 | return ((double)bytes)/(1024.0*1024.0)/(((double)durationMillis)/1000.0); 24 | } 25 | 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/util/MonitorUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.util; 16 | 17 | import java.util.regex.Pattern; 18 | 19 | public class MonitorUtils { 20 | 21 | private static final Pattern pattern = Pattern.compile("Rss\\w*Exception"); 22 | 23 | public static boolean hasRssException(String str) { 24 | if (str == null || str.isEmpty()) { 25 | return false; 26 | } 27 | 28 | if (str.contains("OutOfMemoryError")) { 29 | return true; 30 | } 31 | 32 | if (str.contains("KryoException")) { 33 | return true; 34 | } 35 | 36 | if (str.contains("exceeding memory limits")) { 37 | return true; 38 | } 39 | 40 | return pattern.matcher(str).find(); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/util/NettyUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.util; 16 | 17 | import io.netty.channel.Channel; 18 | import io.netty.channel.ChannelHandlerContext; 19 | 20 | public class NettyUtils { 21 | public static String getRemoteAddressAsString(ChannelHandlerContext ctx) { 22 | return ctx.channel().remoteAddress().toString(); 23 | } 24 | 25 | public static String getServerConnectionInfo(ChannelHandlerContext ctx) { 26 | return getServerConnectionInfo(ctx.channel()); 27 | } 28 | 29 | public static String getServerConnectionInfo(Channel channel) { 30 | return String.format("[%s -> %s]", channel.localAddress(), channel.remoteAddress()); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/util/ObjectWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.util; 16 | 17 | import java.util.Objects; 18 | 19 | public class ObjectWrapper { 20 | private volatile T object; 21 | 22 | public ObjectWrapper() { 23 | } 24 | 25 | public void setObject(T object) { 26 | this.object = object; 27 | } 28 | 29 | public T getObject() { 30 | return object; 31 | } 32 | 33 | @Override 34 | public boolean equals(Object o) { 35 | if (this == o) return true; 36 | if (o == null || getClass() != o.getClass()) return false; 37 | ObjectWrapper that = (ObjectWrapper) o; 38 | return Objects.equals(object, that.object); 39 | } 40 | 41 | @Override 42 | public int hashCode() { 43 | return Objects.hash(object); 44 | } 45 | 46 | @Override 47 | public String toString() { 48 | return "ObjectWrapper{" + 49 | "object=" + object + 50 | '}'; 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/util/SocketAsyncWriteCallback.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.util; 16 | 17 | public interface SocketAsyncWriteCallback { 18 | } 19 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/util/SystemUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.util; 16 | 17 | import com.sun.management.UnixOperatingSystemMXBean; 18 | 19 | import java.lang.management.ManagementFactory; 20 | import java.lang.management.OperatingSystemMXBean; 21 | 22 | public class SystemUtils { 23 | 24 | public static long getFileDescriptorCount() { 25 | OperatingSystemMXBean mbean = ManagementFactory.getOperatingSystemMXBean(); 26 | if(mbean instanceof UnixOperatingSystemMXBean){ 27 | return ((UnixOperatingSystemMXBean)mbean).getOpenFileDescriptorCount(); 28 | } else { 29 | return 0L; 30 | } 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /src/main/java/com/uber/rss/util/ThreadUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.util; 16 | 17 | import org.slf4j.Logger; 18 | import org.slf4j.LoggerFactory; 19 | 20 | public class ThreadUtils { 21 | private static final Logger logger = 22 | LoggerFactory.getLogger(ThreadUtils.class); 23 | 24 | public static final long SHORT_WAIT_TIME = 500; 25 | 26 | public static void sleep(long millis) { 27 | try { 28 | Thread.sleep(millis); 29 | } catch (InterruptedException e) { 30 | logger.info("Interrupted during sleeping", e); 31 | } 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/resources/log4j-rss-debug.properties: -------------------------------------------------------------------------------- 1 | log4j.rootCategory=DEBUG, CONSOLE, FILE 2 | 3 | log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender 4 | log4j.appender.CONSOLE.target=System.out 5 | log4j.appender.CONSOLE.Threshold=debug 6 | log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout 7 | log4j.appender.CONSOLE.layout.ConversionPattern=%d{MM/dd HH:mm:ss} %p %t %c{1}: %m%n 8 | 9 | log4j.appender.FILE=org.apache.log4j.FileAppender 10 | log4j.appender.FILE.File=rss.log 11 | log4j.appender.FILE.ImmediateFlush=true 12 | log4j.appender.FILE.Threshold=debug 13 | log4j.appender.FILE.Append=false 14 | log4j.appender.FILE.layout=org.apache.log4j.PatternLayout 15 | log4j.appender.FILE.layout.conversionPattern=%d{MM/dd HH:mm:ss} %p %t %c{1}: %m%n 16 | -------------------------------------------------------------------------------- /src/main/resources/log4j-rss-prod.properties: -------------------------------------------------------------------------------- 1 | log4j.rootCategory=INFO, CONSOLE, FILE 2 | 3 | log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender 4 | log4j.appender.CONSOLE.target=System.out 5 | log4j.appender.CONSOLE.Threshold=info 6 | log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout 7 | log4j.appender.CONSOLE.layout.ConversionPattern=%d{MM/dd HH:mm:ss} %p %t %c{1}: %m%n 8 | 9 | log4j.appender.FILE=org.apache.log4j.FileAppender 10 | log4j.appender.FILE.File=rss.log 11 | log4j.appender.FILE.ImmediateFlush=false 12 | log4j.appender.FILE.Threshold=info 13 | log4j.appender.FILE.Append=false 14 | log4j.appender.FILE.layout=org.apache.log4j.PatternLayout 15 | log4j.appender.FILE.layout.conversionPattern=%d{MM/dd HH:mm:ss} %p %t %c{1}: %m%n 16 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/shuffle/RssEmptyShuffleReader.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package org.apache.spark.shuffle 16 | 17 | import com.uber.rss.common.AppShuffleId 18 | import org.apache.hadoop.conf.Configuration 19 | import org.apache.spark.internal.Logging 20 | import org.apache.spark.serializer.SerializerInstance 21 | import org.apache.spark.shuffle.rss.BlockDownloaderPartitionRangeRecordIterator 22 | import org.apache.spark.util.CompletionIterator 23 | import org.apache.spark.util.collection.ExternalSorter 24 | import org.apache.spark.{InterruptibleIterator, ShuffleDependency, TaskContext} 25 | 26 | /*** 27 | * This is a shuffle reader returning zero record. 28 | * It is used when there is zero partitions for mapper side. So the reader could return 29 | * empty record iterator directly without connecting to shuffle server. 30 | * @tparam K 31 | * @tparam C 32 | */ 33 | class RssEmptyShuffleReader[K, C]( 34 | shuffleInfo: AppShuffleId, 35 | startPartition: Int, 36 | endPartition: Int) extends ShuffleReader[K, C] with Logging { 37 | 38 | logInfo(s"Using RssEmptyShuffleReader: ${this.getClass.getSimpleName}, shuffleInfo: $shuffleInfo, startPartition: $startPartition, endPartition: $endPartition") 39 | 40 | override def read(): Iterator[Product2[K, C]] = { 41 | logInfo(s"RssEmptyShuffleReader read") 42 | Iterator.empty 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/shuffle/RssServerSelectionResult.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package org.apache.spark.shuffle 16 | 17 | import com.uber.rss.common.ServerDetail 18 | 19 | case class RssServerSelectionResult(servers: Array[ServerDetail], replicas: Int, partitionFanout: Int) 20 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/shuffle/RssShuffleBlockResolver.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package org.apache.spark.shuffle 16 | 17 | import org.apache.spark.network.buffer.ManagedBuffer 18 | import org.apache.spark.storage.ShuffleBlockId 19 | 20 | class RssShuffleBlockResolver extends ShuffleBlockResolver { 21 | override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { 22 | throw new RuntimeException("RssShuffleBlockResolver.getBlockData not implemented") 23 | } 24 | 25 | override def stop(): Unit = { 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/shuffle/RssShuffleHandle.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package org.apache.spark.shuffle 16 | 17 | import com.uber.rss.common.ServerList 18 | import org.apache.spark.ShuffleDependency 19 | 20 | private[spark] class RssShuffleHandle[K, V, C]( 21 | shuffleId: Int, 22 | val appId: String, 23 | val appAttempt: String, 24 | val numMaps: Int, 25 | val user: String, 26 | val queue: String, 27 | val dependency: ShuffleDependency[K, V, C], 28 | val rssServers: Array[RssShuffleServerHandle], 29 | val partitionFanout: Int = 1) 30 | extends ShuffleHandle(shuffleId) { 31 | 32 | def getServerList: ServerList = { 33 | new ServerList(rssServers.map(_.toServerDetail())) 34 | } 35 | 36 | override def toString: String = s"RssShuffleHandle (shuffleId $shuffleId, numMaps: $numMaps, rssServers: ${rssServers.length} servers), partitionFanout: $partitionFanout" 37 | } -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/shuffle/RssShuffleServerHandle.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package org.apache.spark.shuffle 16 | 17 | import com.uber.rss.common.ServerDetail 18 | 19 | case class RssShuffleServerHandle(serverId: String, runningVersion: String, connectionString: String){ 20 | def toServerDetail(): ServerDetail = { 21 | new ServerDetail(serverId, runningVersion, connectionString) 22 | } 23 | } 24 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/shuffle/rss/EmptyRecordIterator.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package org.apache.spark.shuffle.rss 16 | 17 | import com.uber.rss.exceptions.RssException 18 | import org.apache.spark.internal.Logging 19 | 20 | class EmptyRecordIterator[K, C]() extends Iterator[Product2[K, C]] with Logging { 21 | 22 | override def hasNext: Boolean = { 23 | false 24 | } 25 | 26 | override def next(): Product2[K, C] = { 27 | throw new RssException("Cannot get next element on empty iterator") 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /src/main/scala/org/apache/spark/shuffle/rss/MapOutputRssInfo.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package org.apache.spark.shuffle.rss 16 | 17 | 18 | import scala.collection.JavaConverters._ 19 | 20 | import com.uber.rss.util.StringUtils 21 | 22 | /** 23 | * This class stores RSS information which is retrieved from map output tracker 24 | * 25 | * @param numMaps 26 | * @param numRssServers 27 | * @param taskAttemptIds 28 | */ 29 | case class MapOutputRssInfo(numMaps: Int, numRssServers: Int, taskAttemptIds: Array[Long]) { 30 | override def toString: String = { 31 | val taskAttemptIdsStr = StringUtils.toString4SortedIntList[java.lang.Long](taskAttemptIds.sorted.map(long2Long).toList.asJava) 32 | s"MapOutputRssInfo(numMaps: $numMaps, numRssServers: $numRssServers, taskAttemptIds: $taskAttemptIdsStr)" 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/test/java/com/uber/rss/RssBuildInfoTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss; 16 | 17 | import org.apache.commons.lang3.StringUtils; 18 | import org.testng.Assert; 19 | import org.testng.annotations.Test; 20 | 21 | public class RssBuildInfoTest { 22 | @Test 23 | public void versionInfo() { 24 | Assert.assertFalse(StringUtils.isBlank(RssBuildInfo.Version)); 25 | Assert.assertNotEquals(RssBuildInfo.Version, RssBuildInfo.UnknownValue); 26 | 27 | Assert.assertFalse(StringUtils.isBlank(RssBuildInfo.Revision)); 28 | Assert.assertNotEquals(RssBuildInfo.Revision, RssBuildInfo.UnknownValue); 29 | } 30 | 31 | } 32 | -------------------------------------------------------------------------------- /src/test/java/com/uber/rss/StreamServerConfigTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss; 16 | 17 | import org.testng.Assert; 18 | import org.testng.annotations.Test; 19 | 20 | import java.io.IOException; 21 | 22 | public class StreamServerConfigTest { 23 | @Test 24 | public void buildFromArgs() throws IOException { 25 | String[] args = new String[]{}; 26 | StreamServerConfig config = StreamServerConfig.buildFromArgs(args); 27 | Assert.assertEquals(config.getCluster(), "default"); 28 | 29 | args = new String[]{"-cluster", "staging"}; 30 | config = StreamServerConfig.buildFromArgs(args); 31 | Assert.assertEquals(config.getCluster(), "staging"); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/test/java/com/uber/rss/StreamServerHttpTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss; 16 | 17 | import com.uber.rss.testutil.TestStreamServer; 18 | import com.uber.rss.util.HttpUtils; 19 | import org.slf4j.Logger; 20 | import org.slf4j.LoggerFactory; 21 | import org.testng.Assert; 22 | import org.testng.annotations.Test; 23 | 24 | public class StreamServerHttpTest { 25 | private static final Logger logger = LoggerFactory.getLogger(StreamServerHttpTest.class); 26 | 27 | @Test 28 | public void health() { 29 | TestStreamServer testServer = TestStreamServer.createRunningServer(); 30 | 31 | try { 32 | String url = String.format("http://localhost:%s/health", testServer.getHttpPort()); 33 | String response = HttpUtils.getUrl(url); 34 | Assert.assertEquals(response, "OK"); 35 | } finally { 36 | testServer.shutdown(); 37 | } 38 | } 39 | 40 | @Test 41 | public void threadDump() { 42 | TestStreamServer testServer = TestStreamServer.createRunningServer(); 43 | 44 | try { 45 | String url = String.format("http://localhost:%s/threadDump", testServer.getHttpPort()); 46 | String response = HttpUtils.getUrl(url); 47 | Assert.assertTrue(response.contains("ThreadStackTrace")); 48 | Assert.assertTrue(response.contains("main")); 49 | } finally { 50 | testServer.shutdown(); 51 | } 52 | } 53 | 54 | } 55 | -------------------------------------------------------------------------------- /src/test/java/com/uber/rss/clients/BusyStatusSocketClientTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.clients; 16 | 17 | import com.uber.rss.messages.GetBusyStatusResponse; 18 | import com.uber.rss.messages.MessageConstants; 19 | import com.uber.rss.testutil.TestConstants; 20 | import com.uber.rss.testutil.TestStreamServer; 21 | import org.testng.Assert; 22 | import org.testng.annotations.Test; 23 | 24 | public class BusyStatusSocketClientTest { 25 | @Test 26 | public void connectServer() { 27 | TestStreamServer testServer1 = TestStreamServer.createRunningServer(); 28 | 29 | try (BusyStatusSocketClient client = new BusyStatusSocketClient( 30 | "localhost", testServer1.getShufflePort(), TestConstants.NETWORK_TIMEOUT, "user1")) { 31 | client.close(); 32 | client.close(); 33 | } 34 | 35 | testServer1.shutdown(); 36 | } 37 | 38 | @Test 39 | public void getBusyStatus() { 40 | TestStreamServer testServer1 = TestStreamServer.createRunningServer(); 41 | 42 | try (BusyStatusSocketClient client = new BusyStatusSocketClient( 43 | "localhost", testServer1.getShufflePort(), TestConstants.NETWORK_TIMEOUT, "user1")) { 44 | GetBusyStatusResponse response = client.getBusyStatus(); 45 | Assert.assertNotNull(response); 46 | } 47 | 48 | testServer1.shutdown(); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /src/test/java/com/uber/rss/common/MapTaskRssInfoTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.common; 16 | 17 | import org.testng.Assert; 18 | import org.testng.annotations.Test; 19 | 20 | public class MapTaskRssInfoTest { 21 | @Test 22 | public void serializeToString() { 23 | int mapId = 1; 24 | long taskAttemptId = 2; 25 | int numRssServers = 3; 26 | 27 | MapTaskRssInfo mapTaskRssInfo = new MapTaskRssInfo( 28 | mapId, taskAttemptId, 3); 29 | String str = mapTaskRssInfo.serializeToString(); 30 | 31 | MapTaskRssInfo deserializedMapTaskRssInfo = MapTaskRssInfo.deserializeFromString(str); 32 | Assert.assertEquals(deserializedMapTaskRssInfo.getMapId(), mapTaskRssInfo.getMapId()); 33 | Assert.assertEquals(deserializedMapTaskRssInfo.getTaskAttemptId(), mapTaskRssInfo.getTaskAttemptId()); 34 | Assert.assertEquals(deserializedMapTaskRssInfo.getNumRssServers(), mapTaskRssInfo.getNumRssServers()); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /src/test/java/com/uber/rss/common/ServerDetailCollectionTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.common; 16 | 17 | import org.testng.Assert; 18 | import org.testng.annotations.Test; 19 | 20 | import java.util.Arrays; 21 | import java.util.Comparator; 22 | import java.util.List; 23 | 24 | public class ServerDetailCollectionTest { 25 | @Test 26 | public void addServer() { 27 | ServerDetailCollection serverDetailCollection = new ServerDetailCollection(); 28 | 29 | Assert.assertEquals(serverDetailCollection.getServers("dc1", "cluster1").size(), 0); 30 | 31 | serverDetailCollection.addServer("dc1", "cluster1", new ServerDetail("server1", "v1", "node1:1")); 32 | serverDetailCollection.addServer("dc1", "cluster1", new ServerDetail("server1", "v1", "node2:2")); 33 | 34 | Assert.assertEquals(serverDetailCollection.getServers("dc1", "cluster1"), 35 | Arrays.asList(new ServerDetail("server1", "v1", "node2:2"))); 36 | 37 | serverDetailCollection.addServer("dc1", "cluster1", new ServerDetail("server2", "v1", "node2:2")); 38 | 39 | List servers = serverDetailCollection.getServers("dc1", "cluster1"); 40 | servers.sort(Comparator.comparing(ServerDetail::getServerId)); 41 | Assert.assertEquals(servers, 42 | Arrays.asList(new ServerDetail("server1", "v1", "node2:2"), new ServerDetail("server2", "v1", "node2:2"))); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /src/test/java/com/uber/rss/testutil/NettyMemoryExperiments.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.testutil; 16 | 17 | import io.netty.buffer.ByteBuf; 18 | import io.netty.buffer.PooledByteBufAllocator; 19 | 20 | public class NettyMemoryExperiments { 21 | 22 | private static void testMemorySize() { 23 | long allocatedBytes = 0; 24 | long totalBytes = 4L * 1024 * 1024 * 1024; 25 | while (allocatedBytes < totalBytes) { 26 | int bufferSize = 4096; 27 | ByteBuf buf = PooledByteBufAllocator.DEFAULT.buffer(bufferSize); 28 | allocatedBytes += bufferSize; 29 | System.out.println(String.format( 30 | "Allocated bytes: %s, Netty heap memory: %s, Netty directy memory: %s", 31 | allocatedBytes, 32 | PooledByteBufAllocator.DEFAULT.metric().usedHeapMemory(), 33 | PooledByteBufAllocator.DEFAULT.metric().usedDirectMemory())); 34 | buf.release(); 35 | } 36 | 37 | } 38 | 39 | public static void main(String[] args) { 40 | testMemorySize(); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /src/test/java/com/uber/rss/testutil/TestConstants.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.testutil; 16 | 17 | import com.uber.rss.clients.ShuffleWriteConfig; 18 | 19 | public class TestConstants { 20 | public static final int NETWORK_TIMEOUT = 30000; 21 | 22 | public static final int DATA_AVAILABLE_POLL_INTERVAL = 10; 23 | public static final int DATA_AVAILABLE_TIMEOUT = 30000; 24 | 25 | public static final int COMPRESSION_BUFFER_SIZE = 64*1024; 26 | 27 | public static final ShuffleWriteConfig SHUFFLE_WRITE_CONFIG = new ShuffleWriteConfig((short)3); 28 | 29 | public static final long CONNECTION_IDLE_TIMEOUT_MILLIS = 30 * 1000; 30 | } 31 | -------------------------------------------------------------------------------- /src/test/java/com/uber/rss/tools/StreamServerStressToolTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.tools; 16 | 17 | import org.testng.annotations.Test; 18 | 19 | import java.nio.file.Files; 20 | 21 | public class StreamServerStressToolTest { 22 | @Test 23 | public void runTool() throws Exception { 24 | String workDir = Files.createTempDirectory("StreamServerStressToolTest_").toString(); 25 | 26 | StreamServerStressTool tool = new StreamServerStressTool(); 27 | tool.setWorkDir(workDir); 28 | try { 29 | tool.run(); 30 | } finally { 31 | tool.cleanup(); 32 | } 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/test/java/com/uber/rss/util/ByteBufUtilsTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.util; 16 | 17 | import io.netty.buffer.ByteBuf; 18 | import io.netty.buffer.PooledByteBufAllocator; 19 | import org.testng.Assert; 20 | import org.testng.annotations.Test; 21 | import scala.Int; 22 | 23 | public class ByteBufUtilsTest { 24 | 25 | @Test 26 | public void convertIntToBytes() { 27 | byte[] bytes = ByteBufUtils.convertIntToBytes(1); 28 | Assert.assertEquals(bytes.length, 4); 29 | Assert.assertEquals(bytes, new byte[] {0, 0, 0, 1}); 30 | 31 | bytes = ByteBufUtils.convertIntToBytes(Int.MinValue()); 32 | Assert.assertEquals(bytes.length, 4); 33 | 34 | ByteBuf buf = PooledByteBufAllocator.DEFAULT.buffer(4); 35 | buf.writeBytes(bytes); 36 | 37 | Assert.assertEquals(buf.readInt(), Int.MinValue()); 38 | 39 | bytes = ByteBufUtils.convertIntToBytes(Int.MaxValue()); 40 | Assert.assertEquals(bytes.length, 4); 41 | 42 | buf = PooledByteBufAllocator.DEFAULT.buffer(4); 43 | buf.writeBytes(bytes); 44 | 45 | Assert.assertEquals(buf.readInt(), Int.MaxValue()); 46 | 47 | buf.release(); 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /src/test/java/com/uber/rss/util/MonitorUtilsTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.util; 16 | 17 | import org.testng.Assert; 18 | import org.testng.annotations.Test; 19 | 20 | public class MonitorUtilsTest { 21 | 22 | @Test 23 | public void hasRssException() { 24 | Assert.assertFalse(MonitorUtils.hasRssException(null)); 25 | Assert.assertFalse(MonitorUtils.hasRssException("")); 26 | Assert.assertFalse(MonitorUtils.hasRssException(" ")); 27 | 28 | Assert.assertFalse(MonitorUtils.hasRssException(" abc \n def \r from com.uber \n\r")); 29 | 30 | Assert.assertFalse(MonitorUtils.hasRssException(" abc \n def \r from com.uber.rss.xyz \n\r")); 31 | 32 | Assert.assertFalse(MonitorUtils.hasRssException(" abc \n def \r abc.RuntimeException() \n\r")); 33 | Assert.assertTrue(MonitorUtils.hasRssException(" abc \n def \r abc.RssException() \n\r")); 34 | Assert.assertTrue(MonitorUtils.hasRssException(" abc \n def \r abc.RssXyzException() \n\r")); 35 | 36 | Assert.assertTrue(MonitorUtils.hasRssException(" abc \n def \r abc.OutOfMemoryError() \n\r")); 37 | Assert.assertTrue(MonitorUtils.hasRssException(" abc \n def \r abc.KryoException() \n\r")); 38 | } 39 | 40 | } 41 | -------------------------------------------------------------------------------- /src/test/java/com/uber/rss/util/NetworkUtilsTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.util; 16 | 17 | import org.testng.Assert; 18 | import org.testng.annotations.Test; 19 | 20 | import java.net.ConnectException; 21 | 22 | public class NetworkUtilsTest { 23 | private final int timeout = 100; 24 | 25 | @Test 26 | public void validHostName() { 27 | Assert.assertTrue(NetworkUtils.isReachable("localhost", timeout)); 28 | Assert.assertTrue(NetworkUtils.isReachable(NetworkUtils.getLocalHostName(), timeout)); 29 | Assert.assertTrue(NetworkUtils.isReachable(NetworkUtils.getLocalFQDN(), timeout)); 30 | } 31 | 32 | @Test 33 | public void validIpAddress() { 34 | Assert.assertTrue(NetworkUtils.isReachable("127.0.0.1", timeout)); 35 | Assert.assertTrue(NetworkUtils.isReachable("0000:0000:0000:0000:0000:0000:0000:0001", timeout)); 36 | Assert.assertTrue(NetworkUtils.isReachable("::1", timeout)); 37 | } 38 | 39 | @Test 40 | public void invalidHostName() { 41 | Assert.assertFalse(NetworkUtils.isReachable(null, timeout)); 42 | Assert.assertFalse(NetworkUtils.isReachable("", timeout)); 43 | Assert.assertFalse(NetworkUtils.isReachable(" ", timeout)); 44 | Assert.assertFalse(NetworkUtils.isReachable("not_exist_host_abc_123", timeout)); 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /src/test/java/com/uber/rss/util/RetryUtilsTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package com.uber.rss.util; 16 | 17 | import org.testng.Assert; 18 | import org.testng.annotations.Test; 19 | 20 | import java.util.concurrent.atomic.AtomicInteger; 21 | 22 | public class RetryUtilsTest { 23 | 24 | @Test 25 | public void firstTrySucceeds() { 26 | String result = RetryUtils.retry(100, 100, 0, "test", () -> { 27 | return "ok"; 28 | }); 29 | Assert.assertEquals(result, "ok"); 30 | } 31 | 32 | @Test 33 | public void multipleRetry() { 34 | int retryMaxMillis = 100; 35 | AtomicInteger counter = new AtomicInteger(); 36 | long startTime = System.currentTimeMillis(); 37 | RetryUtils.retry(0, 60000, retryMaxMillis, "test", () -> { 38 | int value = counter.incrementAndGet(); 39 | if (System.currentTimeMillis() - startTime <= retryMaxMillis/2) { 40 | throw new RuntimeException("simulated exception"); 41 | } 42 | return value; 43 | }); 44 | } 45 | 46 | @Test(expectedExceptions = RuntimeException.class) 47 | public void testThrowException() { 48 | int retryMaxMillis = 100; 49 | AtomicInteger counter = new AtomicInteger(); 50 | RetryUtils.retry(1, 10, retryMaxMillis, "test", () -> { 51 | counter.incrementAndGet(); 52 | throw new RuntimeException("simulated exception"); 53 | }); 54 | } 55 | 56 | } 57 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/shuffle/RssOptsTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package org.apache.spark.shuffle 16 | 17 | import org.apache.spark.SparkConf 18 | import org.scalatest.Assertions._ 19 | import org.testng.annotations.Test 20 | 21 | @Test 22 | class RssOptsTest { 23 | private val conf = new SparkConf() 24 | 25 | def testDefaultValues(): Unit = { 26 | assert(conf.get(RssOpts.dataCenter) === "") 27 | } 28 | 29 | } 30 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/shuffle/TestUdfs.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package org.apache.spark.shuffle 16 | 17 | import java.util.Random 18 | 19 | import org.apache.commons.lang3.StringUtils 20 | 21 | object TestUdfs { 22 | val random = new Random() 23 | 24 | val testValues = (1 to 1000).map(n => { 25 | StringUtils.repeat('a', random.nextInt(n)) 26 | }).toList 27 | 28 | def intToString(intValue: Int): String = { 29 | "%09d".format(intValue) 30 | } 31 | 32 | def generateString(size: Int): String = { 33 | StringUtils.repeat('a', size) 34 | } 35 | 36 | def randomString(): String = { 37 | val index = random.nextInt(testValues.size) 38 | testValues(index) 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/test/scala/org/apache/spark/shuffle/rss/MapOutputRssInfoTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2020 Uber Technologies, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * Unless required by applicable law or agreed to in writing, software 9 | * distributed under the License is distributed on an "AS IS" BASIS, 10 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 11 | * See the License for the specific language governing permissions and 12 | * limitations under the License. 13 | */ 14 | 15 | package org.apache.spark.shuffle.rss 16 | 17 | import org.testng.Assert 18 | import org.testng.annotations.Test 19 | 20 | class MapOutputRssInfoTest { 21 | 22 | @Test 23 | def toStringTest() = { 24 | var info = MapOutputRssInfo(1, 2, Array()) 25 | Assert.assertTrue(info.toString.size > 0) 26 | 27 | info = MapOutputRssInfo(1, 2, Array(1L)) 28 | Assert.assertTrue(info.toString.size > 0) 29 | 30 | info = MapOutputRssInfo(1, 2, Array(1L, 3L, 9L, 11L, 10L, 5L)) 31 | Assert.assertTrue(info.toString.size > 0) 32 | } 33 | } 34 | --------------------------------------------------------------------------------