├── .asf.yaml ├── .baseline └── copyright │ └── apache-license-header.txt ├── .github ├── ISSUE_TEMPLATE │ ├── bug-report.yml │ ├── config.yml │ ├── doc-improvement-report.yml │ ├── feature-request.yml │ ├── flaky-test-report.yml │ ├── improvement.yml │ ├── subtask.yml │ └── umbrella.yml ├── PULL_REQUEST_TEMPLATE └── workflows │ ├── build.yml │ ├── changes.yml │ ├── docker.yml │ ├── parallel.yml │ ├── sequential.yml │ ├── single.yml │ └── test-results.yml ├── .gitignore ├── .mvn └── wrapper │ └── maven-wrapper.properties ├── CONTRIBUTING.md ├── DISCLAIMER ├── LICENSE ├── LICENSE-binary ├── NOTICE ├── NOTICE-binary ├── README.md ├── bin ├── get-metrics.py ├── restart-coordinator.sh ├── restart-shuffle-server.sh ├── start-coordinator.sh ├── start-dashboard.sh ├── start-shuffle-server.sh ├── stop-coordinator.sh ├── stop-dashboard.sh ├── stop-shuffle-server.sh ├── uniffle ├── uniffle-function.sh └── utils.sh ├── build_distribution.sh ├── checkstyle.xml ├── cli ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── uniffle │ │ ├── AbstractCustomCommandLine.java │ │ ├── CustomCommandLine.java │ │ ├── UniffleCliArgsException.java │ │ ├── api │ │ └── AdminRestApi.java │ │ ├── cli │ │ ├── CLIContentUtils.java │ │ ├── UniffleAdminCLI.java │ │ └── UniffleCLI.java │ │ ├── client │ │ ├── HttpClientFactory.java │ │ ├── RestClient.java │ │ ├── RestClientConf.java │ │ ├── RestClientImpl.java │ │ ├── UniffleRestClient.java │ │ └── exception │ │ │ └── UniffleRestException.java │ │ └── entity │ │ └── ApplicationResponse.java │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── uniffle │ │ └── cli │ │ ├── AdminRestApiTest.java │ │ ├── CLIContentUtilsTest.java │ │ ├── UniffleTestAdminCLI.java │ │ └── UniffleTestCLI.java │ └── resources │ └── CLIContentResult ├── client-mr ├── core │ ├── pom.xml │ └── src │ │ ├── main │ │ └── java │ │ │ └── org │ │ │ └── apache │ │ │ ├── hadoop │ │ │ ├── mapred │ │ │ │ ├── RssCombineOutputCollector.java │ │ │ │ ├── RssMapOutputCollector.java │ │ │ │ ├── SortWriteBuffer.java │ │ │ │ └── SortWriteBufferManager.java │ │ │ └── mapreduce │ │ │ │ ├── MRIdHelper.java │ │ │ │ ├── RssMRConfig.java │ │ │ │ ├── RssMRUtils.java │ │ │ │ ├── task │ │ │ │ └── reduce │ │ │ │ │ ├── MRMetricsReporter.java │ │ │ │ │ ├── RMRssShuffle.java │ │ │ │ │ ├── RssBypassWriter.java │ │ │ │ │ ├── RssEventFetcher.java │ │ │ │ │ ├── RssFetcher.java │ │ │ │ │ ├── RssInMemoryRemoteMerger.java │ │ │ │ │ ├── RssRemoteMergeManagerImpl.java │ │ │ │ │ └── RssShuffle.java │ │ │ │ └── v2 │ │ │ │ └── app │ │ │ │ └── RssMRAppMaster.java │ │ │ └── uniffle │ │ │ └── client │ │ │ └── shuffle │ │ │ ├── MRCombiner.java │ │ │ └── RecordCollector.java │ │ └── test │ │ ├── java │ │ └── org │ │ │ └── apache │ │ │ ├── hadoop │ │ │ ├── mapred │ │ │ │ ├── SortWriteBufferManagerTest.java │ │ │ │ └── SortWriteBufferTest.java │ │ │ └── mapreduce │ │ │ │ ├── RssMRUtilsTest.java │ │ │ │ └── task │ │ │ │ └── reduce │ │ │ │ ├── EventFetcherTest.java │ │ │ │ ├── FetcherTest.java │ │ │ │ ├── RMRssShuffleTest.java │ │ │ │ ├── RssInMemoryRemoteMergerTest.java │ │ │ │ └── RssRemoteMergeManagerTest.java │ │ │ └── uniffle │ │ │ └── client │ │ │ └── shuffle │ │ │ ├── MRCombinerTest.java │ │ │ └── RecordCollectorTest.java │ │ └── resources │ │ └── log4j2.xml ├── hadoop2.8 │ ├── pom.xml │ └── src │ │ └── main │ │ └── java │ │ └── org │ │ └── apache │ │ └── uniffle │ │ └── hadoop │ │ └── shim │ │ └── HadoopShimImpl.java └── hadoop3.2 │ ├── pom.xml │ └── src │ └── main │ └── java │ └── org │ └── apache │ └── uniffle │ └── hadoop │ └── shim │ └── HadoopShimImpl.java ├── client-spark ├── common │ ├── pom.xml │ └── src │ │ ├── main │ │ └── java │ │ │ └── org │ │ │ └── apache │ │ │ ├── spark │ │ │ └── shuffle │ │ │ │ ├── DelegationRssShuffleManagerUtils.java │ │ │ │ ├── RssShuffleHandle.java │ │ │ │ ├── RssSparkConfig.java │ │ │ │ ├── RssSparkShuffleUtils.java │ │ │ │ ├── RssStageInfo.java │ │ │ │ ├── RssStageResubmitManager.java │ │ │ │ ├── ShuffleHandleInfoManager.java │ │ │ │ ├── SparkVersionUtils.java │ │ │ │ ├── events │ │ │ │ ├── ShuffleAssignmentInfoEvent.java │ │ │ │ ├── ShuffleMetric.java │ │ │ │ ├── ShuffleReadMetric.java │ │ │ │ ├── ShuffleWriteMetric.java │ │ │ │ ├── TaskShuffleReadInfoEvent.java │ │ │ │ ├── TaskShuffleWriteInfoEvent.java │ │ │ │ └── UniffleEvent.java │ │ │ │ ├── handle │ │ │ │ ├── MutableShuffleHandleInfo.java │ │ │ │ ├── ShuffleHandleInfo.java │ │ │ │ ├── ShuffleHandleInfoBase.java │ │ │ │ ├── SimpleShuffleHandleInfo.java │ │ │ │ ├── StageAttemptShuffleHandleInfo.java │ │ │ │ └── split │ │ │ │ │ └── PartitionSplitInfo.java │ │ │ │ ├── reader │ │ │ │ ├── RssFetchFailedIterator.java │ │ │ │ └── RssShuffleDataIterator.java │ │ │ │ └── writer │ │ │ │ ├── AddBlockEvent.java │ │ │ │ ├── BlockFailureCallback.java │ │ │ │ ├── BlockSuccessCallback.java │ │ │ │ ├── BufferManagerOptions.java │ │ │ │ ├── DataPusher.java │ │ │ │ ├── TaskAttemptAssignment.java │ │ │ │ ├── WrappedByteArrayOutputStream.java │ │ │ │ ├── WriteBufferManager.java │ │ │ │ └── WriterBuffer.java │ │ │ └── uniffle │ │ │ └── shuffle │ │ │ ├── BlockIdManager.java │ │ │ ├── BlockIdSelfManagedShuffleWriteClient.java │ │ │ ├── RssShuffleClientFactory.java │ │ │ ├── ShuffleIdMappingManager.java │ │ │ └── manager │ │ │ ├── RssShuffleManagerBase.java │ │ │ ├── RssShuffleManagerInterface.java │ │ │ ├── ShuffleManagerGrpcService.java │ │ │ └── ShuffleManagerServerFactory.java │ │ └── test │ │ └── java │ │ └── org │ │ └── apache │ │ ├── spark │ │ └── shuffle │ │ │ ├── RssSparkShuffleUtilsTest.java │ │ │ ├── handle │ │ │ └── MutableShuffleHandleInfoTest.java │ │ │ ├── reader │ │ │ ├── AbstractRssReaderTest.java │ │ │ └── RssShuffleDataIteratorTest.java │ │ │ └── writer │ │ │ ├── DataPusherTest.java │ │ │ ├── WriteBufferManagerTest.java │ │ │ └── WriteBufferTest.java │ │ └── uniffle │ │ └── shuffle │ │ └── manager │ │ ├── DummyRssShuffleManager.java │ │ ├── RssShuffleManagerBaseTest.java │ │ ├── ShuffleManagerGrpcServiceTest.java │ │ └── ShuffleManagerServerFactoryTest.java ├── extension │ ├── pom.xml │ └── src │ │ └── main │ │ ├── resources │ │ └── META-INF │ │ │ └── services │ │ │ └── org.apache.spark.status.AppHistoryServerPlugin │ │ └── scala │ │ └── org │ │ └── apache │ │ └── spark │ │ ├── UniffleHistoryServerPlugin.scala │ │ ├── UniffleListener.scala │ │ ├── UnifflePlugin.scala │ │ ├── UniffleStatusStore.scala │ │ └── ui │ │ ├── ShufflePage.scala │ │ └── ShuffleTab.scala ├── spark2-shaded │ └── pom.xml ├── spark2 │ ├── pom.xml │ └── src │ │ ├── main │ │ └── java │ │ │ └── org │ │ │ └── apache │ │ │ └── spark │ │ │ ├── network │ │ │ └── util │ │ │ │ └── NettyUtils.java │ │ │ └── shuffle │ │ │ ├── DelegationRssShuffleManager.java │ │ │ ├── RssShuffleManager.java │ │ │ ├── reader │ │ │ └── RssShuffleReader.java │ │ │ └── writer │ │ │ └── RssShuffleWriter.java │ │ └── test │ │ └── java │ │ └── org │ │ └── apache │ │ └── spark │ │ └── shuffle │ │ ├── DelegationRssShuffleManagerTest.java │ │ ├── RssSpark2ShuffleUtilsTest.java │ │ ├── SparkVersionUtilsTest.java │ │ ├── reader │ │ └── RssShuffleReaderTest.java │ │ └── writer │ │ └── RssShuffleWriterTest.java ├── spark3-shaded │ └── pom.xml └── spark3 │ ├── pom.xml │ └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── spark │ │ └── shuffle │ │ ├── DelegationRssShuffleManager.java │ │ ├── FunctionUtils.java │ │ ├── RssShuffleDataIo.java │ │ ├── RssShuffleDriverComponents.java │ │ ├── RssShuffleManager.java │ │ ├── Spark3VersionUtils.java │ │ ├── reader │ │ └── RssShuffleReader.java │ │ └── writer │ │ └── RssShuffleWriter.java │ └── test │ └── java │ └── org │ └── apache │ └── spark │ └── shuffle │ ├── DelegationRssShuffleManagerTest.java │ ├── FunctionUtilsTests.java │ ├── RssShuffleManagerTest.java │ ├── RssShuffleManagerTestBase.java │ ├── RssSpark3ShuffleUtilsTest.java │ ├── SparkVersionUtilsTest.java │ ├── TestUtils.java │ ├── reader │ └── RssShuffleReaderTest.java │ └── writer │ └── RssShuffleWriterTest.java ├── client-tez ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── tez │ │ ├── common │ │ ├── GetShuffleServerRequest.java │ │ ├── GetShuffleServerResponse.java │ │ ├── IdUtils.java │ │ ├── InputContextUtils.java │ │ ├── RssTezConfig.java │ │ ├── RssTezUtils.java │ │ ├── ShuffleAssignmentsInfoWritable.java │ │ ├── TezClassLoader.java │ │ ├── TezIdHelper.java │ │ ├── TezRemoteShuffleUmbilicalProtocol.java │ │ └── UmbilicalUtils.java │ │ ├── dag │ │ └── app │ │ │ ├── RssDAGAppMaster.java │ │ │ ├── TezRemoteShuffleManager.java │ │ │ └── security │ │ │ └── authorize │ │ │ └── RssTezAMPolicyProvider.java │ │ └── runtime │ │ └── library │ │ ├── common │ │ ├── shuffle │ │ │ ├── RemoteFetchedInput.java │ │ │ ├── impl │ │ │ │ ├── RssShuffleManager.java │ │ │ │ ├── RssSimpleFetchedInputAllocator.java │ │ │ │ ├── RssTezFetcher.java │ │ │ │ └── RssTezFetcherTask.java │ │ │ └── orderedgrouped │ │ │ │ ├── RMRssShuffle.java │ │ │ │ ├── RMRssShuffleScheduler.java │ │ │ │ ├── RssInMemoryMerger.java │ │ │ │ ├── RssMergeManager.java │ │ │ │ ├── RssShuffle.java │ │ │ │ ├── RssShuffleScheduler.java │ │ │ │ ├── RssTezBypassWriter.java │ │ │ │ └── RssTezShuffleDataFetcher.java │ │ └── sort │ │ │ ├── buffer │ │ │ ├── WriteBuffer.java │ │ │ └── WriteBufferManager.java │ │ │ └── impl │ │ │ ├── RssSorter.java │ │ │ ├── RssTezPerPartitionRecord.java │ │ │ └── RssUnSorter.java │ │ ├── input │ │ ├── RMRssOrderedGroupedKVInput.java │ │ ├── RssConcatenatedMergedKeyValueInput.java │ │ ├── RssConcatenatedMergedKeyValuesInput.java │ │ ├── RssOrderedGroupedInputLegacy.java │ │ ├── RssOrderedGroupedKVInput.java │ │ ├── RssOrderedGroupedMergedKVInput.java │ │ └── RssUnorderedKVInput.java │ │ └── output │ │ ├── RssOrderedPartitionedKVOutput.java │ │ ├── RssUnorderedKVOutput.java │ │ └── RssUnorderedPartitionedKVOutput.java │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── tez │ │ ├── common │ │ ├── GetShuffleServerRequestTest.java │ │ ├── GetShuffleServerResponseTest.java │ │ ├── IdUtilsTest.java │ │ ├── InputContextUtilsTest.java │ │ ├── RssTezUtilsTest.java │ │ ├── ShuffleAssignmentsInfoWritableTest.java │ │ └── TezIdHelperTest.java │ │ ├── dag │ │ └── app │ │ │ ├── RssDAGAppMasterTest.java │ │ │ └── TezRemoteShuffleManagerTest.java │ │ └── runtime │ │ └── library │ │ ├── common │ │ ├── shuffle │ │ │ ├── impl │ │ │ │ ├── RssShuffleManagerTest.java │ │ │ │ ├── RssSimpleFetchedInputAllocatorTest.java │ │ │ │ └── RssTezFetcherTest.java │ │ │ └── orderedgrouped │ │ │ │ ├── RMRssShuffleTest.java │ │ │ │ ├── RssInMemoryMergerTest.java │ │ │ │ ├── RssMergeManagerTest.java │ │ │ │ ├── RssShuffleSchedulerTest.java │ │ │ │ ├── RssShuffleTest.java │ │ │ │ ├── RssTezBypassWriterTest.java │ │ │ │ └── RssTezShuffleDataFetcherTest.java │ │ └── sort │ │ │ ├── buffer │ │ │ ├── WriteBufferManagerTest.java │ │ │ └── WriteBufferTest.java │ │ │ └── impl │ │ │ ├── RssSorterTest.java │ │ │ ├── RssTezPerPartitionRecordTest.java │ │ │ └── RssUnSorterTest.java │ │ ├── input │ │ ├── RMRssOrderedGroupedKVInputTest.java │ │ ├── RssOrderedGroupedKVInputTest.java │ │ └── RssSortedGroupedMergedInputTest.java │ │ └── output │ │ ├── OutputTestHelpers.java │ │ ├── RssOrderedPartitionedKVOutputTest.java │ │ ├── RssUnorderedKVOutputTest.java │ │ └── RssUnorderedPartitionedKVOutputTest.java │ └── resources │ └── log4j2.xml ├── client ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── uniffle │ │ └── client │ │ ├── PartitionDataReplicaRequirementTracking.java │ │ ├── api │ │ ├── ShuffleReadClient.java │ │ └── ShuffleWriteClient.java │ │ ├── factory │ │ └── ShuffleClientFactory.java │ │ ├── impl │ │ ├── FailedBlockSendTracker.java │ │ ├── ShuffleReadClientImpl.java │ │ ├── ShuffleWriteClientImpl.java │ │ ├── TrackingBlockStatus.java │ │ └── TrackingPartitionStatus.java │ │ ├── record │ │ ├── Record.java │ │ ├── RecordBlob.java │ │ ├── RecordBuffer.java │ │ ├── RecordCollection.java │ │ ├── metrics │ │ │ └── MetricsReporter.java │ │ ├── reader │ │ │ ├── BufferedSegment.java │ │ │ ├── KeyValueReader.java │ │ │ ├── KeyValuesReader.java │ │ │ └── RMRecordsReader.java │ │ └── writer │ │ │ └── Combiner.java │ │ ├── response │ │ ├── CompressedShuffleBlock.java │ │ └── SendShuffleDataResult.java │ │ └── util │ │ ├── ClientUtils.java │ │ ├── DefaultIdHelper.java │ │ └── RssClientConfig.java │ └── test │ └── java │ └── org │ └── apache │ └── uniffle │ └── client │ ├── ClientUtilsTest.java │ ├── PartitionDataReplicaRequirementTrackingTest.java │ ├── TestUtils.java │ ├── impl │ ├── FailedBlockSendTrackerTest.java │ ├── ShuffleReadClientImplTest.java │ └── ShuffleWriteClientImplTest.java │ └── record │ ├── reader │ ├── BufferedSegmentTest.java │ ├── MockedShuffleServerClient.java │ ├── MockedShuffleWriteClient.java │ └── RMRecordsReaderTest.java │ └── writer │ ├── RecordCollectionTest.java │ └── SumByKeyCombiner.java ├── common ├── pom.xml └── src │ ├── main │ ├── java-templates │ │ └── org │ │ │ └── apache │ │ │ └── uniffle │ │ │ └── common │ │ │ └── ProjectConstants.java │ ├── java │ │ ├── io │ │ │ └── netty │ │ │ │ └── util │ │ │ │ └── CompositeFileRegion.java │ │ └── org │ │ │ └── apache │ │ │ └── uniffle │ │ │ └── common │ │ │ ├── Application.java │ │ │ ├── Arguments.java │ │ │ ├── AuditType.java │ │ │ ├── BlockCompletionCallback.java │ │ │ ├── BufferSegment.java │ │ │ ├── ClientType.java │ │ │ ├── PartitionInfo.java │ │ │ ├── PartitionRange.java │ │ │ ├── PartitionSplitMode.java │ │ │ ├── ReceivingFailureServer.java │ │ │ ├── ReconfigurableConfManager.java │ │ │ ├── ReconfigurableRegistry.java │ │ │ ├── RemoteStorageInfo.java │ │ │ ├── ServerStatus.java │ │ │ ├── ShuffleAssignmentsInfo.java │ │ │ ├── ShuffleBlockInfo.java │ │ │ ├── ShuffleDataDistributionType.java │ │ │ ├── ShuffleDataResult.java │ │ │ ├── ShuffleDataSegment.java │ │ │ ├── ShuffleIndexResult.java │ │ │ ├── ShufflePartitionedBlock.java │ │ │ ├── ShufflePartitionedData.java │ │ │ ├── ShuffleRegisterInfo.java │ │ │ ├── ShuffleServerInfo.java │ │ │ ├── StorageType.java │ │ │ ├── UnionKey.java │ │ │ ├── audit │ │ │ └── RpcAuditContext.java │ │ │ ├── compression │ │ │ ├── Codec.java │ │ │ ├── Lz4Codec.java │ │ │ ├── NoOpCodec.java │ │ │ ├── SnappyCodec.java │ │ │ └── ZstdCodec.java │ │ │ ├── config │ │ │ ├── ConfigOption.java │ │ │ ├── ConfigOptions.java │ │ │ ├── ConfigUtils.java │ │ │ ├── FallbackKey.java │ │ │ ├── RssBaseConf.java │ │ │ ├── RssClientConf.java │ │ │ └── RssConf.java │ │ │ ├── exception │ │ │ ├── ExceedHugePartitionHardLimitException.java │ │ │ ├── FileNotFoundException.java │ │ │ ├── InvalidRequestException.java │ │ │ ├── NoBufferException.java │ │ │ ├── NoBufferForHugePartitionException.java │ │ │ ├── NoRegisterException.java │ │ │ ├── NotRetryException.java │ │ │ ├── RssException.java │ │ │ ├── RssFetchFailedException.java │ │ │ ├── RssSendFailedException.java │ │ │ └── RssWaitFailedException.java │ │ │ ├── executor │ │ │ ├── MeasurableRejectedExecutionHandler.java │ │ │ └── ThreadPoolManager.java │ │ │ ├── filesystem │ │ │ └── HadoopFilesystemProvider.java │ │ │ ├── function │ │ │ ├── ConsumerWithException.java │ │ │ └── TupleConsumer.java │ │ │ ├── future │ │ │ ├── CompletableFutureExtension.java │ │ │ └── CompletableFutureUtils.java │ │ │ ├── merger │ │ │ ├── KeyValueIterator.java │ │ │ ├── MergeState.java │ │ │ ├── Merger.java │ │ │ ├── Segment.java │ │ │ └── StreamedSegment.java │ │ │ ├── metrics │ │ │ ├── AbstractMetricReporter.java │ │ │ ├── CommonMetrics.java │ │ │ ├── EmptyGRPCMetrics.java │ │ │ ├── GRPCMetrics.java │ │ │ ├── JvmMetrics.java │ │ │ ├── MetricReporter.java │ │ │ ├── MetricReporterFactory.java │ │ │ ├── MetricsManager.java │ │ │ ├── NettyMetrics.java │ │ │ ├── RPCMetrics.java │ │ │ ├── SupplierGauge.java │ │ │ └── prometheus │ │ │ │ └── PrometheusPushGatewayMetricReporter.java │ │ │ ├── netty │ │ │ ├── DecodeException.java │ │ │ ├── EncodeException.java │ │ │ ├── FrameDecoder.java │ │ │ ├── IOMode.java │ │ │ ├── MessageEncoder.java │ │ │ ├── TransportFrameDecoder.java │ │ │ ├── buffer │ │ │ │ ├── FileSegmentManagedBuffer.java │ │ │ │ ├── ManagedBuffer.java │ │ │ │ ├── MultiFileSegmentManagedBuffer.java │ │ │ │ └── NettyManagedBuffer.java │ │ │ ├── client │ │ │ │ ├── RpcResponseCallback.java │ │ │ │ ├── TransportClient.java │ │ │ │ ├── TransportClientFactory.java │ │ │ │ ├── TransportConf.java │ │ │ │ └── TransportContext.java │ │ │ ├── handle │ │ │ │ ├── BaseMessageHandler.java │ │ │ │ ├── MessageHandler.java │ │ │ │ ├── TransportChannelHandler.java │ │ │ │ ├── TransportRequestHandler.java │ │ │ │ └── TransportResponseHandler.java │ │ │ └── protocol │ │ │ │ ├── AbstractFileRegion.java │ │ │ │ ├── Decoders.java │ │ │ │ ├── Encodable.java │ │ │ │ ├── Encoders.java │ │ │ │ ├── GetLocalShuffleDataRequest.java │ │ │ │ ├── GetLocalShuffleDataResponse.java │ │ │ │ ├── GetLocalShuffleDataV2Request.java │ │ │ │ ├── GetLocalShuffleIndexRequest.java │ │ │ │ ├── GetLocalShuffleIndexResponse.java │ │ │ │ ├── GetLocalShuffleIndexV2Response.java │ │ │ │ ├── GetMemoryShuffleDataRequest.java │ │ │ │ ├── GetMemoryShuffleDataResponse.java │ │ │ │ ├── GetSortedShuffleDataRequest.java │ │ │ │ ├── GetSortedShuffleDataResponse.java │ │ │ │ ├── Message.java │ │ │ │ ├── MessageWithHeader.java │ │ │ │ ├── RequestMessage.java │ │ │ │ ├── ResponseMessage.java │ │ │ │ ├── RpcResponse.java │ │ │ │ ├── SendShuffleDataRequest.java │ │ │ │ └── SendShuffleDataRequestV1.java │ │ │ ├── records │ │ │ ├── RecordsReader.java │ │ │ └── RecordsWriter.java │ │ │ ├── rpc │ │ │ ├── ClientContextServerInterceptor.java │ │ │ ├── GrpcServer.java │ │ │ ├── MonitoringServerCall.java │ │ │ ├── MonitoringServerCallListener.java │ │ │ ├── MonitoringServerInterceptor.java │ │ │ ├── MonitoringServerTransportFilter.java │ │ │ ├── ServerInterface.java │ │ │ ├── ServerType.java │ │ │ └── StatusCode.java │ │ │ ├── security │ │ │ ├── HadoopSecurityContext.java │ │ │ ├── NoOpSecurityContext.java │ │ │ ├── SecurityConfig.java │ │ │ ├── SecurityContext.java │ │ │ └── SecurityContextFactory.java │ │ │ ├── segment │ │ │ ├── AbstractSegmentSplitter.java │ │ │ ├── FixedSizeSegmentSplitter.java │ │ │ ├── LocalOrderSegmentSplitter.java │ │ │ ├── SegmentSplitter.java │ │ │ └── SegmentSplitterFactory.java │ │ │ ├── serializer │ │ │ ├── BufferSerInputStream.java │ │ │ ├── DeserializationStream.java │ │ │ ├── DynBufferSerOutputStream.java │ │ │ ├── FileSerInputStream.java │ │ │ ├── FileSerOutputStream.java │ │ │ ├── SerInputStream.java │ │ │ ├── SerOutputStream.java │ │ │ ├── SerializationStream.java │ │ │ ├── Serializer.java │ │ │ ├── SerializerFactory.java │ │ │ ├── SerializerInstance.java │ │ │ ├── WrappedByteArrayOutputStream.java │ │ │ ├── kryo │ │ │ │ ├── KryoDeserializationStream.java │ │ │ │ ├── KryoSerializationStream.java │ │ │ │ ├── KryoSerializer.java │ │ │ │ ├── KryoSerializerInstance.java │ │ │ │ └── PoolWrapper.java │ │ │ └── writable │ │ │ │ ├── BufferedRawWritableDeserializationStream.java │ │ │ │ ├── BufferedRawWritableSerializationStream.java │ │ │ │ ├── ComparativeOutputBuffer.java │ │ │ │ ├── RawWritableDeserializationStream.java │ │ │ │ ├── RawWritableSerializationStream.java │ │ │ │ ├── WritableDeserializationStream.java │ │ │ │ ├── WritableSerializationStream.java │ │ │ │ ├── WritableSerializer.java │ │ │ │ └── WritableSerializerInstance.java │ │ │ ├── storage │ │ │ ├── ApplicationStorageInfo.java │ │ │ ├── StorageInfo.java │ │ │ ├── StorageInfoUtils.java │ │ │ ├── StorageMedia.java │ │ │ └── StorageStatus.java │ │ │ ├── util │ │ │ ├── BlockId.java │ │ │ ├── BlockIdLayout.java │ │ │ ├── ByteBufUtils.java │ │ │ ├── ByteBufferUtils.java │ │ │ ├── ByteUnit.java │ │ │ ├── ChecksumUtils.java │ │ │ ├── Constants.java │ │ │ ├── ExitUtils.java │ │ │ ├── ExpiringCloseableSupplier.java │ │ │ ├── GrpcNettyUtils.java │ │ │ ├── IdHelper.java │ │ │ ├── JavaUtils.java │ │ │ ├── JvmPauseMonitor.java │ │ │ ├── NettyUtils.java │ │ │ ├── OutputUtils.java │ │ │ ├── RetryUtils.java │ │ │ ├── RssUtils.java │ │ │ ├── StatefulCloseable.java │ │ │ ├── ThreadUtils.java │ │ │ ├── TripleFunction.java │ │ │ └── UnitConverter.java │ │ │ └── web │ │ │ ├── CoalescedCollectorRegistry.java │ │ │ ├── JerseyAutoDiscoverable.java │ │ │ ├── JettyServer.java │ │ │ ├── JsonConverter.java │ │ │ └── resource │ │ │ ├── Authorization.java │ │ │ ├── AuthorizationRequestFilter.java │ │ │ ├── BaseMetricResource.java │ │ │ ├── BaseResource.java │ │ │ ├── ConfOpsResource.java │ │ │ ├── ConfVO.java │ │ │ ├── MetricResource.java │ │ │ ├── PrometheusMetricResource.java │ │ │ └── Response.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.hbase.thirdparty.org.glassfish.jersey.internal.spi.AutoDiscoverable │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── uniffle │ │ └── common │ │ ├── ArgumentsTest.java │ │ ├── BufferSegmentTest.java │ │ ├── KerberizedHadoop.java │ │ ├── KerberizedHadoopBase.java │ │ ├── PartitionRangeTest.java │ │ ├── ReconfigurableConfManagerTest.java │ │ ├── ReconfigurableRegistryTest.java │ │ ├── RemoteStorageInfoTest.java │ │ ├── ServerStatusTest.java │ │ ├── ShuffleBlockInfoTest.java │ │ ├── ShuffleDataResultTest.java │ │ ├── ShuffleIndexResultTest.java │ │ ├── ShufflePartitionedBlockTest.java │ │ ├── ShufflePartitionedDataTest.java │ │ ├── ShuffleRegisterInfoTest.java │ │ ├── ShuffleServerInfoTest.java │ │ ├── UnionKeyTest.java │ │ ├── compression │ │ └── CompressionTest.java │ │ ├── config │ │ ├── ConfigOptionTest.java │ │ ├── ConfigUtilsTest.java │ │ └── RssConfTest.java │ │ ├── executor │ │ └── ThreadPoolManagerTest.java │ │ ├── filesystem │ │ └── HadoopFilesystemProviderTest.java │ │ ├── future │ │ ├── CompletableFutureExtensionTest.java │ │ └── CompletableFutureUtilsTest.java │ │ ├── log │ │ ├── TestLoggerExtension.java │ │ └── TestLoggerParamResolver.java │ │ ├── merger │ │ └── MergerTest.java │ │ ├── metrics │ │ ├── MetricReporterFactoryTest.java │ │ ├── MetricsManagerTest.java │ │ ├── TestUtils.java │ │ └── prometheus │ │ │ └── PrometheusPushGatewayMetricReporterTest.java │ │ ├── netty │ │ ├── EncoderAndDecoderTest.java │ │ ├── TransportFrameDecoderTest.java │ │ ├── buffer │ │ │ └── FileSegmentManagedBufferTest.java │ │ ├── client │ │ │ ├── TransportClientFactoryTest.java │ │ │ └── TransportClientTestBase.java │ │ └── protocol │ │ │ ├── NettyProtocolTest.java │ │ │ └── NettyProtocolTestUtils.java │ │ ├── records │ │ └── RecordsReaderWriterTest.java │ │ ├── rpc │ │ ├── GrpcServerTest.java │ │ └── StatusCodeTest.java │ │ ├── security │ │ ├── HadoopSecurityContextTest.java │ │ └── SecurityContextFactoryTest.java │ │ ├── segment │ │ ├── FixedSizeSegmentSplitterTest.java │ │ └── LocalOrderSegmentSplitterTest.java │ │ ├── serializer │ │ ├── KryoSerializerTest.java │ │ ├── SerInputOutputStreamTest.java │ │ ├── SerializerFactoryTest.java │ │ ├── SerializerUtils.java │ │ └── WritableSerializerTest.java │ │ ├── storage │ │ └── StorageInfoUtilsTest.java │ │ ├── util │ │ ├── BlockIdLayoutTest.java │ │ ├── BlockIdTest.java │ │ ├── ByteBufUtilsTest.java │ │ ├── ChecksumUtilsTest.java │ │ ├── ExitUtilsTest.java │ │ ├── ExpiringCloseableSupplierTest.java │ │ ├── JavaUtilsTest.java │ │ ├── NettyUtilsTest.java │ │ ├── OutputUtilsTest.java │ │ ├── RetryUtilsTest.java │ │ ├── RssUtilsTest.java │ │ ├── ThreadUtilsTest.java │ │ └── UnitConverterTest.java │ │ └── web │ │ └── JettyServerTest.java │ └── resources │ ├── rss-defaults.conf │ └── server.conf ├── conf ├── coordinator.conf ├── dashboard.conf ├── log4j2.xml ├── rss-env.sh └── server.conf ├── coordinator ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── uniffle │ │ └── coordinator │ │ ├── AccessManager.java │ │ ├── AppInfo.java │ │ ├── ApplicationManager.java │ │ ├── ClusterManager.java │ │ ├── ClusterManagerFactory.java │ │ ├── CoordinatorConf.java │ │ ├── CoordinatorFactory.java │ │ ├── CoordinatorGrpcService.java │ │ ├── CoordinatorServer.java │ │ ├── QuotaManager.java │ │ ├── ServerNode.java │ │ ├── SimpleClusterManager.java │ │ ├── access │ │ ├── AccessCheckResult.java │ │ ├── AccessInfo.java │ │ └── checker │ │ │ ├── AbstractAccessChecker.java │ │ │ ├── AccessCandidatesChecker.java │ │ │ ├── AccessChecker.java │ │ │ ├── AccessClusterLoadChecker.java │ │ │ └── AccessQuotaChecker.java │ │ ├── audit │ │ └── CoordinatorRpcAuditContext.java │ │ ├── conf │ │ ├── AbstractRssClientConfApplyStrategy.java │ │ ├── BypassRssClientConfApplyStrategy.java │ │ ├── ClientConf.java │ │ ├── ClientConfParser.java │ │ ├── DynamicClientConfService.java │ │ ├── LegacyClientConfParser.java │ │ ├── RssClientConfApplyManager.java │ │ ├── RssClientConfFetchInfo.java │ │ └── YamlClientConfParser.java │ │ ├── metric │ │ ├── CoordinatorGrpcMetrics.java │ │ └── CoordinatorMetrics.java │ │ ├── strategy │ │ ├── assignment │ │ │ ├── AbstractAssignmentStrategy.java │ │ │ ├── AssignmentStrategy.java │ │ │ ├── AssignmentStrategyFactory.java │ │ │ ├── BasicAssignmentStrategy.java │ │ │ ├── PartitionBalanceAssignmentStrategy.java │ │ │ └── PartitionRangeAssignment.java │ │ ├── host │ │ │ ├── BasicHostAssignmentStrategy.java │ │ │ ├── HostAssignmentStrategy.java │ │ │ ├── MustDiffHostAssignmentStrategy.java │ │ │ └── PreferDiffHostAssignmentStrategy.java │ │ ├── partition │ │ │ ├── ContinuousSelectPartitionStrategy.java │ │ │ ├── RoundSelectPartitionStrategy.java │ │ │ └── SelectPartitionStrategy.java │ │ └── storage │ │ │ ├── AbstractSelectStorageStrategy.java │ │ │ ├── AppBalanceSelectStorageStrategy.java │ │ │ ├── LowestIOSampleCostSelectStorageStrategy.java │ │ │ ├── RankValue.java │ │ │ └── SelectStorageStrategy.java │ │ ├── util │ │ └── CoordinatorUtils.java │ │ └── web │ │ ├── lombok.config │ │ ├── request │ │ ├── ApplicationRequest.java │ │ ├── CancelDecommissionRequest.java │ │ └── DecommissionRequest.java │ │ ├── resource │ │ ├── APIResource.java │ │ ├── AdminResource.java │ │ ├── ApplicationResource.java │ │ ├── CoordinatorServerResource.java │ │ └── ServerResource.java │ │ └── vo │ │ ├── AppInfoVO.java │ │ ├── CoordinatorConfVO.java │ │ └── UserAppNumVO.java │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── uniffle │ │ └── coordinator │ │ ├── ApplicationManagerTest.java │ │ ├── CoordinatorConfTest.java │ │ ├── CoordinatorServerTest.java │ │ ├── GenericTestUtils.java │ │ ├── QuotaManagerTest.java │ │ ├── ServerNodeTest.java │ │ ├── SimpleClusterManagerTest.java │ │ ├── access │ │ └── AccessManagerTest.java │ │ ├── checker │ │ ├── AccessCandidatesCheckerTest.java │ │ ├── AccessClusterLoadCheckerTest.java │ │ └── AccessQuotaCheckerTest.java │ │ ├── conf │ │ ├── DynamicClientConfServiceTest.java │ │ ├── LegacyClientConfParserTest.java │ │ ├── RssClientConfApplyManagerTest.java │ │ └── YamlClientConfParserTest.java │ │ ├── metric │ │ └── CoordinatorMetricsTest.java │ │ ├── strategy │ │ ├── assignment │ │ │ ├── BasicAssignmentStrategyTest.java │ │ │ ├── PartitionBalanceAssignmentStrategyTest.java │ │ │ ├── PartitionRangeAssignmentTest.java │ │ │ └── PartitionRangeTest.java │ │ ├── partition │ │ │ └── ContinuousSelectPartitionStrategyTest.java │ │ └── storage │ │ │ ├── AppBalanceSelectStorageStrategyTest.java │ │ │ └── LowestIOSampleCostSelectStorageStrategyTest.java │ │ ├── util │ │ └── CoordinatorUtilsTest.java │ │ └── web │ │ ├── CoordinatorTestServer.java │ │ ├── UniffleJavaProcess.java │ │ └── UniffleServicesRESTTest.java │ └── resources │ ├── coordinator.conf │ ├── dynamicClientConf.legacy │ ├── dynamicClientConf.yaml │ ├── empty │ └── quotaFile.properties ├── dashboard ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── uniffle │ │ │ └── dashboard │ │ │ └── web │ │ │ ├── Dashboard.java │ │ │ ├── config │ │ │ └── DashboardConf.java │ │ │ ├── lombok.config │ │ │ ├── proxy │ │ │ └── WebProxyServlet.java │ │ │ ├── resource │ │ │ ├── CoordinatorResource.java │ │ │ ├── DashboardResource.java │ │ │ └── WebResource.java │ │ │ ├── utils │ │ │ └── DashboardUtils.java │ │ │ └── vo │ │ │ └── DashboardConfVO.java │ └── webapp │ │ ├── .eslintignore │ │ ├── .eslintrc.js │ │ ├── .prettierrc.json │ │ ├── babel.config.js │ │ ├── jsconfig.json │ │ ├── package.json │ │ ├── packagescript │ │ ├── cleanfile.js │ │ ├── filecopy.js │ │ └── fileutils.js │ │ ├── public │ │ ├── favicon.ico │ │ └── index.html │ │ ├── src │ │ ├── App.vue │ │ ├── api │ │ │ └── api.js │ │ ├── assets │ │ │ └── uniffle-logo.png │ │ ├── components │ │ │ └── LayoutPage.vue │ │ ├── main.js │ │ ├── mock │ │ │ ├── applicationpage.js │ │ │ ├── coordinatorserverpage.js │ │ │ ├── excludenodelist.js │ │ │ ├── index.js │ │ │ ├── layoutpage.js │ │ │ ├── nodelistpage.js │ │ │ └── shuffleserverpage.js │ │ ├── pages │ │ │ ├── ApplicationPage.vue │ │ │ ├── CoordinatorServerPage.vue │ │ │ ├── DashboardPage.vue │ │ │ ├── ShuffleServerPage.vue │ │ │ └── serverstatus │ │ │ │ └── NodeListPage.vue │ │ ├── router │ │ │ └── index.js │ │ ├── store │ │ │ └── useCurrentServerStore.js │ │ └── utils │ │ │ ├── common.js │ │ │ ├── http.js │ │ │ └── request.js │ │ └── vue.config.js │ └── test │ └── java │ └── org │ └── apache │ └── uniffle │ └── dashboard │ └── web │ └── utils │ └── DashboardUtilsTest.java ├── deploy ├── docker │ ├── README.md │ ├── build.sh │ ├── docker-compose.yml │ ├── spark │ │ └── Dockerfile │ └── uniffle │ │ ├── Dockerfile │ │ └── start.sh └── kubernetes │ ├── build-operator.sh │ ├── docker │ ├── Dockerfile │ ├── base │ │ ├── centos │ │ │ └── Dockerfile │ │ └── debian │ │ │ └── Dockerfile │ ├── build.sh │ ├── hadoopconfig │ │ └── .gitkeep │ ├── rss-env.sh │ └── start.sh │ ├── integration-test │ ├── .gitkeep │ └── e2e │ │ ├── README.md │ │ ├── kind-config │ │ ├── set-up-local-cluster.sh │ │ ├── start-e2e.sh │ │ └── template │ │ ├── metrics-server.yaml │ │ ├── rss-controller-template.yaml │ │ ├── rss-template.yaml │ │ └── rss-webhook-template.yaml │ ├── operator │ ├── Makefile │ ├── PROJECT │ ├── api │ │ └── uniffle │ │ │ └── v1alpha1 │ │ │ ├── doc.go │ │ │ ├── groupversion_info.go │ │ │ ├── remoteshuffleservice_types.go │ │ │ └── zz_generated.deepcopy.go │ ├── cmd │ │ ├── controller │ │ │ └── main.go │ │ └── webhook │ │ │ └── main.go │ ├── config │ │ ├── crd │ │ │ └── bases │ │ │ │ └── uniffle.apache.org_remoteshuffleservices.yaml │ │ ├── manager │ │ │ ├── rss-controller.yaml │ │ │ └── rss-webhook.yaml │ │ └── samples │ │ │ └── uniffle_v1alpha1_remoteshuffleservice.yaml │ ├── examples │ │ ├── configuration.yaml │ │ ├── full-restart │ │ │ ├── README.md │ │ │ └── rss-full-restart.yaml │ │ ├── full-upgrade │ │ │ ├── README.md │ │ │ └── rss-full-upgrade.yaml │ │ ├── partition-upgrade │ │ │ ├── README.md │ │ │ └── rss-partition-upgrade.yaml │ │ ├── pvc-example │ │ │ ├── README.md │ │ │ ├── gce-storage-class.yml │ │ │ └── rss-pvc-on-gce.yaml │ │ └── specific-upgrade │ │ │ ├── README.md │ │ │ └── rss-specific-upgrade.yaml │ ├── go.mod │ ├── go.sum │ ├── hack │ │ ├── Dockerfile │ │ ├── add_header_for_crds.sh │ │ ├── headers │ │ │ ├── header.go.txt │ │ │ └── header.yaml.txt │ │ ├── revive.toml │ │ ├── tools.go │ │ └── update-codegen.sh │ └── pkg │ │ ├── constants │ │ └── constants.go │ │ ├── controller │ │ ├── config │ │ │ └── config.go │ │ ├── constants │ │ │ └── constants.go │ │ ├── controller │ │ │ ├── process_rss_test.go │ │ │ ├── rss.go │ │ │ ├── rss_test.go │ │ │ ├── shuffle_server_test.go │ │ │ └── test.go │ │ ├── sync │ │ │ ├── coordinator │ │ │ │ ├── coordinator.go │ │ │ │ └── coordinator_test.go │ │ │ └── shuffleserver │ │ │ │ ├── shuffleserver.go │ │ │ │ └── shuffleserver_test.go │ │ └── util │ │ │ ├── kubernetes │ │ │ ├── configmap.go │ │ │ ├── deployment.go │ │ │ ├── hpa.go │ │ │ ├── hpa_test.go │ │ │ ├── service.go │ │ │ ├── serviceaccount.go │ │ │ └── statefulset.go │ │ │ ├── properties │ │ │ └── properties.go │ │ │ ├── util.go │ │ │ └── util_test.go │ │ ├── generated │ │ ├── clientset │ │ │ └── versioned │ │ │ │ ├── clientset.go │ │ │ │ ├── doc.go │ │ │ │ ├── fake │ │ │ │ ├── clientset_generated.go │ │ │ │ ├── doc.go │ │ │ │ └── register.go │ │ │ │ ├── scheme │ │ │ │ ├── doc.go │ │ │ │ └── register.go │ │ │ │ └── typed │ │ │ │ └── uniffle │ │ │ │ └── v1alpha1 │ │ │ │ ├── doc.go │ │ │ │ ├── fake │ │ │ │ ├── doc.go │ │ │ │ ├── fake_remoteshuffleservice.go │ │ │ │ └── fake_uniffle_client.go │ │ │ │ ├── generated_expansion.go │ │ │ │ ├── remoteshuffleservice.go │ │ │ │ └── uniffle_client.go │ │ ├── informers │ │ │ └── externalversions │ │ │ │ ├── factory.go │ │ │ │ ├── generic.go │ │ │ │ ├── internalinterfaces │ │ │ │ └── factory_interfaces.go │ │ │ │ └── uniffle │ │ │ │ ├── interface.go │ │ │ │ └── v1alpha1 │ │ │ │ ├── interface.go │ │ │ │ └── remoteshuffleservice.go │ │ └── listers │ │ │ └── uniffle │ │ │ └── v1alpha1 │ │ │ ├── expansion_generated.go │ │ │ └── remoteshuffleservice.go │ │ ├── utils │ │ ├── certs.go │ │ ├── config.go │ │ ├── coordinator.go │ │ ├── rss.go │ │ ├── rss_test.go │ │ ├── shufflerserver.go │ │ └── util.go │ │ └── webhook │ │ ├── config │ │ └── config.go │ │ ├── constants │ │ └── constants.go │ │ ├── inspector │ │ ├── inspector.go │ │ ├── pod.go │ │ ├── pod_test.go │ │ ├── rss.go │ │ └── rss_test.go │ │ ├── manager.go │ │ ├── manager_test.go │ │ ├── syncer │ │ └── syncer.go │ │ └── util │ │ ├── patch.go │ │ ├── util.go │ │ └── util_test.go │ ├── pom.xml │ └── test-operator.sh ├── dev ├── intellij │ ├── install-runconfig.sh │ └── runConfs │ │ ├── CoordinatorServer.run │ │ ├── Dashboard.run │ │ └── ShuffleServer.run ├── local_dev_template │ ├── coordinator.conf │ ├── dashboard.conf │ ├── log4j2.xml │ └── server.conf └── scripts │ └── checkshade.sh ├── docs ├── _config.yml ├── asset │ ├── rss-crd-state-transition.png │ ├── rss_architecture.png │ ├── rss_benchmark1.png │ ├── rss_benchmark2.png │ ├── rss_benchmark3.png │ ├── rss_data_format.png │ ├── rss_remote_merge_architecture.png │ ├── rss_shuffle_write.png │ ├── vanilla_benchmark1.png │ ├── vanilla_benchmark2.png │ └── vanilla_benchmark3.png ├── benchmark.md ├── benchmark_netty_case_report.md ├── client_guide │ ├── client_guide.md │ ├── mr_client_guide.md │ ├── spark_client_guide.md │ └── tez_client_guide.md ├── coordinator_guide.md ├── dashboard_guide.md ├── gen-doc.sh ├── index.md ├── metrics_guide.md ├── operator │ ├── README.md │ ├── design.md │ ├── examples.md │ └── install.md ├── remote_merge_guide.md ├── server_guide.md ├── troubleshooting.md └── uniffle-migration-guide.md ├── integration-test ├── common │ ├── pom.xml │ └── src │ │ └── test │ │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── uniffle │ │ │ └── test │ │ │ ├── AccessCandidatesCheckerHadoopTest.java │ │ │ ├── AccessCandidatesCheckerKerberizedHadoopTest.java │ │ │ ├── AccessClusterTest.java │ │ │ ├── AssignmentWithTagsTest.java │ │ │ ├── CoordinatorAdminServiceTest.java │ │ │ ├── CoordinatorAssignmentTest.java │ │ │ ├── CoordinatorGrpcServerTest.java │ │ │ ├── CoordinatorGrpcTest.java │ │ │ ├── CoordinatorReconfigureNodeMaxTest.java │ │ │ ├── CoordinatorTestBase.java │ │ │ ├── CoordinatorTestUtils.java │ │ │ ├── DiskErrorToleranceTest.java │ │ │ ├── DynamicClientConfServiceHadoopTest.java │ │ │ ├── DynamicClientConfServiceKerberlizedHadoopTest.java │ │ │ ├── FetchClientConfTest.java │ │ │ ├── HealthCheckCoordinatorGrpcTest.java │ │ │ ├── HealthCheckTest.java │ │ │ ├── HybridStorageFaultToleranceBase.java │ │ │ ├── HybridStorageHadoopFallbackTest.java │ │ │ ├── HybridStorageLocalFileFallbackTest.java │ │ │ ├── IntegrationTestBase.java │ │ │ ├── PartitionBalanceCoordinatorGrpcTest.java │ │ │ ├── QuorumTest.java │ │ │ ├── RemoteMergeShuffleWithRssClientTest.java │ │ │ ├── RemoteMergeShuffleWithRssClientTestWhenShuffleFlushed.java │ │ │ ├── RpcClientRetryTest.java │ │ │ ├── ServletTest.java │ │ │ ├── ShuffleReadWriteBase.java │ │ │ ├── ShuffleServerConcurrentWriteOfHadoopTest.java │ │ │ ├── ShuffleServerFaultToleranceTest.java │ │ │ ├── ShuffleServerGrpcTest.java │ │ │ ├── ShuffleServerInternalGrpcTest.java │ │ │ ├── ShuffleServerOnRandomPortTest.java │ │ │ ├── ShuffleServerWithHadoopTest.java │ │ │ ├── ShuffleServerWithKerberizedHadoopTest.java │ │ │ ├── ShuffleServerWithLocalOfExceptionTest.java │ │ │ ├── ShuffleServerWithLocalOfLocalOrderTest.java │ │ │ ├── ShuffleServerWithLocalTest.java │ │ │ ├── ShuffleServerWithMemLocalHadoopTest.java │ │ │ ├── ShuffleServerWithMemoryTest.java │ │ │ └── ShuffleWithRssClientTest.java │ │ └── resources │ │ └── log4j2.xml ├── mr │ ├── pom.xml │ └── src │ │ └── test │ │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── uniffle │ │ │ └── test │ │ │ ├── DynamicConfTest.java │ │ │ ├── FailoverAppMaster.java │ │ │ ├── HadoopConfTest.java │ │ │ ├── LargeSorterTest.java │ │ │ ├── MRIntegrationTestBase.java │ │ │ ├── RMWordCountTest.java │ │ │ ├── SecondarySortTest.java │ │ │ └── WordCountTest.java │ │ └── resources │ │ └── log4j2.xml ├── spark-common │ ├── pom.xml │ └── src │ │ └── test │ │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── uniffle │ │ │ └── test │ │ │ ├── AutoAccessTest.java │ │ │ ├── CombineByKeyTest.java │ │ │ ├── DynamicFetchClientConfTest.java │ │ │ ├── FailingTasksTest.java │ │ │ ├── GroupByKeyTest.java │ │ │ ├── NullOfKeyOrValueTest.java │ │ │ ├── RSSStageDynamicServerReWriteTest.java │ │ │ ├── RSSStageResubmitTest.java │ │ │ ├── RepartitionTest.java │ │ │ ├── RepartitionWithHadoopHybridStorageRssTest.java │ │ │ ├── RepartitionWithLocalFileRssTest.java │ │ │ ├── RepartitionWithMemoryHybridStorageRssTest.java │ │ │ ├── RepartitionWithMemoryRssTest.java │ │ │ ├── RssShuffleManagerTest.java │ │ │ ├── ShuffleServerManagerTestBase.java │ │ │ ├── ShuffleUnregisterWithHadoopTest.java │ │ │ ├── ShuffleUnregisterWithLocalfileTest.java │ │ │ ├── SimpleShuffleServerManagerTest.java │ │ │ ├── SimpleTestBase.java │ │ │ ├── SparkClientWithLocalForMultiPartLocalStorageManagerTest.java │ │ │ ├── SparkClientWithLocalTest.java │ │ │ ├── SparkIntegrationTestBase.java │ │ │ ├── SparkSQLTest.java │ │ │ ├── SparkSQLWithDelegationShuffleManagerFallbackTest.java │ │ │ ├── SparkSQLWithDelegationShuffleManagerTest.java │ │ │ ├── SparkSQLWithMemoryLocalTest.java │ │ │ ├── SparkTaskFailureIntegrationTestBase.java │ │ │ ├── TestUtils.java │ │ │ ├── WriteAndReadMetricsTest.java │ │ │ └── listener │ │ │ └── WriteAndReadMetricsSparkListener.java │ │ └── resources │ │ ├── candidates │ │ └── log4j2.xml ├── spark2 │ ├── pom.xml │ └── src │ │ └── test │ │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── uniffle │ │ │ └── test │ │ │ └── GetReaderTest.java │ │ └── resources │ │ └── log4j2.xml ├── spark3 │ ├── pom.xml │ └── src │ │ └── test │ │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── uniffle │ │ │ └── test │ │ │ ├── AQERepartitionTest.java │ │ │ ├── AQESkewedJoinTest.java │ │ │ ├── AQESkewedJoinWithLocalOrderTest.java │ │ │ ├── ContinuousSelectPartitionStrategyTest.java │ │ │ ├── GetReaderTest.java │ │ │ ├── GetShuffleReportForMultiPartTest.java │ │ │ ├── MapSideCombineTest.java │ │ │ ├── PartitionBlockDataReassignBasicTest.java │ │ │ ├── PartitionBlockDataReassignMultiTimesTest.java │ │ │ ├── PartitionSplitOfLoadBalanceModeTest.java │ │ │ └── ReassignAndStageRetryTest.java │ │ └── resources │ │ └── log4j2.xml └── tez │ ├── pom.xml │ └── src │ └── test │ ├── java │ └── org │ │ └── apache │ │ ├── tez │ │ └── dag │ │ │ └── app │ │ │ └── RssDAGAppMasterForWordCountWithFailures.java │ │ └── uniffle │ │ └── test │ │ ├── RMTezOrderedWordCountTest.java │ │ ├── TezCartesianProductTest.java │ │ ├── TezHashJoinTest.java │ │ ├── TezIntegrationTestBase.java │ │ ├── TezJoinIntegrationTestBase.java │ │ ├── TezOrderedWordCountTest.java │ │ ├── TezSimpleSessionExampleTest.java │ │ ├── TezSortMergeJoinTest.java │ │ ├── TezWordCountTest.java │ │ └── TezWordCountWithFailuresTest.java │ └── resources │ └── log4j2.xml ├── internal-client ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── uniffle │ │ └── client │ │ ├── api │ │ ├── ClientInfo.java │ │ ├── CoordinatorClient.java │ │ ├── ShuffleManagerClient.java │ │ ├── ShuffleServerClient.java │ │ └── ShuffleServerInternalClient.java │ │ ├── common │ │ ├── ShuffleServerPushCost.java │ │ └── ShuffleServerPushCostTracker.java │ │ ├── factory │ │ ├── CoordinatorClientFactory.java │ │ ├── ShuffleManagerClientFactory.java │ │ └── ShuffleServerClientFactory.java │ │ ├── impl │ │ └── grpc │ │ │ ├── CoordinatorGrpcClient.java │ │ │ ├── CoordinatorGrpcRetryableClient.java │ │ │ ├── GrpcClient.java │ │ │ ├── ShuffleManagerGrpcClient.java │ │ │ ├── ShuffleServerGrpcClient.java │ │ │ ├── ShuffleServerGrpcNettyClient.java │ │ │ └── ShuffleServerInternalGrpcClient.java │ │ ├── request │ │ ├── RetryableRequest.java │ │ ├── RssAccessClusterRequest.java │ │ ├── RssAppHeartBeatRequest.java │ │ ├── RssApplicationInfoRequest.java │ │ ├── RssCancelDecommissionRequest.java │ │ ├── RssDecommissionRequest.java │ │ ├── RssFetchClientConfRequest.java │ │ ├── RssFetchRemoteStorageRequest.java │ │ ├── RssFinishShuffleRequest.java │ │ ├── RssGetInMemoryShuffleDataRequest.java │ │ ├── RssGetShuffleAssignmentsRequest.java │ │ ├── RssGetShuffleDataRequest.java │ │ ├── RssGetShuffleIndexRequest.java │ │ ├── RssGetShuffleResultForMultiPartRequest.java │ │ ├── RssGetShuffleResultRequest.java │ │ ├── RssGetSortedShuffleDataRequest.java │ │ ├── RssPartitionToShuffleServerRequest.java │ │ ├── RssReassignOnBlockSendFailureRequest.java │ │ ├── RssRegisterShuffleRequest.java │ │ ├── RssReportShuffleFetchFailureRequest.java │ │ ├── RssReportShuffleReadMetricRequest.java │ │ ├── RssReportShuffleResultRequest.java │ │ ├── RssReportShuffleWriteFailureRequest.java │ │ ├── RssReportShuffleWriteMetricRequest.java │ │ ├── RssSendCommitRequest.java │ │ ├── RssSendHeartBeatRequest.java │ │ ├── RssSendShuffleDataRequest.java │ │ ├── RssStartSortMergeRequest.java │ │ ├── RssUnregisterShuffleByAppIdRequest.java │ │ └── RssUnregisterShuffleRequest.java │ │ └── response │ │ ├── ClientResponse.java │ │ ├── RssAccessClusterResponse.java │ │ ├── RssAppHeartBeatResponse.java │ │ ├── RssApplicationInfoResponse.java │ │ ├── RssCancelDecommissionResponse.java │ │ ├── RssDecommissionResponse.java │ │ ├── RssFetchClientConfResponse.java │ │ ├── RssFetchRemoteStorageResponse.java │ │ ├── RssFinishShuffleResponse.java │ │ ├── RssGetInMemoryShuffleDataResponse.java │ │ ├── RssGetShuffleAssignmentsResponse.java │ │ ├── RssGetShuffleDataResponse.java │ │ ├── RssGetShuffleIndexResponse.java │ │ ├── RssGetShuffleResultResponse.java │ │ ├── RssGetSortedShuffleDataResponse.java │ │ ├── RssReassignOnBlockSendFailureResponse.java │ │ ├── RssReassignOnStageRetryResponse.java │ │ ├── RssRegisterShuffleResponse.java │ │ ├── RssReportShuffleFetchFailureResponse.java │ │ ├── RssReportShuffleReadMetricResponse.java │ │ ├── RssReportShuffleResultResponse.java │ │ ├── RssReportShuffleWriteFailureResponse.java │ │ ├── RssReportShuffleWriteMetricResponse.java │ │ ├── RssSendCommitResponse.java │ │ ├── RssSendHeartBeatResponse.java │ │ ├── RssSendShuffleDataResponse.java │ │ ├── RssStartSortMergeResponse.java │ │ ├── RssUnregisterShuffleByAppIdResponse.java │ │ └── RssUnregisterShuffleResponse.java │ └── test │ └── java │ └── org │ └── apache │ └── uniffle │ └── client │ └── factory │ └── ShuffleManagerClientFactoryTest.java ├── licenses-binary ├── LICENSE-animal-sniffer.txt ├── LICENSE-axios.txt ├── LICENSE-checker-qual.txt ├── LICENSE-core-js.txt ├── LICENSE-element-plus.txt ├── LICENSE-hbase-shaded-jersey.txt ├── LICENSE-jakarta-annotation.txt ├── LICENSE-jakarta-inject.txt ├── LICENSE-javax-activation.txt ├── LICENSE-javax-annotation.txt ├── LICENSE-javax-servlet-api.txt ├── LICENSE-jaxb-api.txt ├── LICENSE-moment.txt ├── LICENSE-protobuf.txt ├── LICENSE-rimraf.txt ├── LICENSE-slf4j.txt ├── LICENSE-vue-resource.txt ├── LICENSE-vue-router.txt ├── LICENSE-vue.txt └── LICENSE-zstd-jni.txt ├── lombok.config ├── mvnw ├── mvnw.cmd ├── patch └── spark │ ├── spark-2.3.4_dynamic_allocation_support.patch │ ├── spark-2.4.6_dynamic_allocation_support.patch │ ├── spark-3.0.1_dynamic_allocation_support.patch │ ├── spark-3.1.2_dynamic_allocation_support.patch │ ├── spark-3.2.1_dynamic_allocation_support.patch │ ├── spark-3.3.1_dynamic_allocation_support.patch │ └── spark-3.4.1_dynamic_allocation_support.patch ├── pom.xml ├── proto ├── pom.xml └── src │ └── main │ └── proto │ └── Rss.proto ├── release ├── asf-settings.xml ├── create-package.sh ├── publish_maven_artifacts.sh └── publish_to_svn.sh ├── security.md ├── server-common └── pom.xml ├── server ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── uniffle │ │ │ └── server │ │ │ ├── Checker.java │ │ │ ├── DefaultFlushEventHandler.java │ │ │ ├── FlushEventHandler.java │ │ │ ├── HealthCheck.java │ │ │ ├── HealthScriptChecker.java │ │ │ ├── HugePartitionUtils.java │ │ │ ├── LocalStorageChecker.java │ │ │ ├── RegisterHeartBeat.java │ │ │ ├── ShuffleDataFlushEvent.java │ │ │ ├── ShuffleDataReadEvent.java │ │ │ ├── ShuffleDetailInfo.java │ │ │ ├── ShuffleFlushManager.java │ │ │ ├── ShuffleServer.java │ │ │ ├── ShuffleServerConf.java │ │ │ ├── ShuffleServerFactory.java │ │ │ ├── ShuffleServerGrpcMetrics.java │ │ │ ├── ShuffleServerGrpcService.java │ │ │ ├── ShuffleServerInternalGrpcService.java │ │ │ ├── ShuffleServerMetrics.java │ │ │ ├── ShuffleServerNettyMetrics.java │ │ │ ├── ShuffleSpecification.java │ │ │ ├── ShuffleTaskInfo.java │ │ │ ├── ShuffleTaskManager.java │ │ │ ├── TopNShuffleDataSizeOfAppCalcTask.java │ │ │ ├── audit │ │ │ └── ServerRpcAuditContext.java │ │ │ ├── block │ │ │ ├── DefaultShuffleBlockIdManager.java │ │ │ ├── PartitionedShuffleBlockIdManager.java │ │ │ ├── ShuffleBlockIdManager.java │ │ │ └── ShuffleBlockIdManagerFactory.java │ │ │ ├── buffer │ │ │ ├── AbstractShuffleBuffer.java │ │ │ ├── PreAllocatedBufferInfo.java │ │ │ ├── RequireBufferStatusCode.java │ │ │ ├── ShuffleBuffer.java │ │ │ ├── ShuffleBufferManager.java │ │ │ ├── ShuffleBufferType.java │ │ │ ├── ShuffleBufferWithLinkedList.java │ │ │ └── ShuffleBufferWithSkipList.java │ │ │ ├── event │ │ │ ├── AppPurgeEvent.java │ │ │ ├── AppUnregisterPurgeEvent.java │ │ │ ├── PurgeEvent.java │ │ │ └── ShufflePurgeEvent.java │ │ │ ├── flush │ │ │ ├── EventDiscardException.java │ │ │ ├── EventInvalidException.java │ │ │ └── EventRetryException.java │ │ │ ├── merge │ │ │ ├── BlockFlushFileReader.java │ │ │ ├── DefaultMergeEventHandler.java │ │ │ ├── MergeEvent.java │ │ │ ├── MergeEventHandler.java │ │ │ ├── MergeStatus.java │ │ │ ├── MergedResult.java │ │ │ ├── Partition.java │ │ │ ├── Shuffle.java │ │ │ └── ShuffleMergeManager.java │ │ │ ├── netty │ │ │ ├── ShuffleServerNettyHandler.java │ │ │ ├── StreamServer.java │ │ │ └── StreamServerMetricHandler.java │ │ │ ├── storage │ │ │ ├── AbstractStorageManagerFallbackStrategy.java │ │ │ ├── HadoopStorageManager.java │ │ │ ├── HadoopStorageManagerFallbackStrategy.java │ │ │ ├── HybridStorageManager.java │ │ │ ├── LocalStorageManager.java │ │ │ ├── LocalStorageManagerFactory.java │ │ │ ├── LocalStorageManagerFallbackStrategy.java │ │ │ ├── MultiPartLocalStorageManager.java │ │ │ ├── RotateStorageManagerFallbackStrategy.java │ │ │ ├── SingleStorageManager.java │ │ │ ├── StorageManager.java │ │ │ ├── StorageManagerFactory.java │ │ │ ├── StorageMediaFromEnvProvider.java │ │ │ └── hybrid │ │ │ │ ├── DefaultStorageManagerSelector.java │ │ │ │ ├── FallbackBasedStorageManagerSelector.java │ │ │ │ ├── HugePartitionSensitiveStorageManagerSelector.java │ │ │ │ └── StorageManagerSelector.java │ │ │ └── web │ │ │ ├── lombok.config │ │ │ ├── resource │ │ │ └── ServerResource.java │ │ │ └── vo │ │ │ └── ServerConfVO.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.uniffle.storage.common.StorageMediaProvider │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── uniffle │ │ └── server │ │ ├── HealthScriptCheckerTest.java │ │ ├── HealthyMockChecker.java │ │ ├── KerberizedShuffleTaskManagerTest.java │ │ ├── LocalSingleStorageTypeFromEnvProviderTest.java │ │ ├── LocalStorageCheckerTest.java │ │ ├── MockedGrpcServer.java │ │ ├── MockedShuffleServer.java │ │ ├── MockedShuffleServerFactory.java │ │ ├── MockedShuffleServerGrpcService.java │ │ ├── ShuffleFlushManagerOnKerberizedHadoopTest.java │ │ ├── ShuffleFlushManagerTest.java │ │ ├── ShuffleServerConfTest.java │ │ ├── ShuffleServerGrpcMetricsTest.java │ │ ├── ShuffleServerMetricsTest.java │ │ ├── ShuffleServerTest.java │ │ ├── ShuffleTaskInfoTest.java │ │ ├── ShuffleTaskManagerTest.java │ │ ├── StorageCheckerTest.java │ │ ├── TopNShuffleDataSizeOfAppCalcTaskTest.java │ │ ├── UnHealthyMockChecker.java │ │ ├── buffer │ │ ├── BufferTestBase.java │ │ ├── ShuffleBufferManagerTest.java │ │ ├── ShuffleBufferWithLinkedListTest.java │ │ └── ShuffleBufferWithSkipListTest.java │ │ ├── merge │ │ ├── BlockFlushFileReaderTest.java │ │ ├── MergedResultTest.java │ │ └── ShuffleMergeManagerTest.java │ │ └── storage │ │ ├── HadoopStorageManagerTest.java │ │ ├── HybridStorageManagerTest.java │ │ ├── LocalStorageManagerTest.java │ │ └── StorageManagerFallbackStrategyTest.java │ └── resources │ ├── confBySizeStringTest.conf │ ├── confTest.conf │ ├── gcTest.conf │ ├── healthy-script1.sh │ ├── healthy-script2.sh │ ├── healthy-script3.sh │ ├── healthy-script4.sh │ ├── log4j2.xml │ └── server.conf ├── spotbugs-exclude.xml ├── storage ├── pom.xml └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── uniffle │ │ └── storage │ │ ├── api │ │ ├── FileReader.java │ │ └── FileWriter.java │ │ ├── common │ │ ├── AbstractStorage.java │ │ ├── CompositeReadingViewStorage.java │ │ ├── DefaultStorageMediaProvider.java │ │ ├── FileBasedShuffleSegment.java │ │ ├── HadoopStorage.java │ │ ├── LocalStorage.java │ │ ├── LocalStorageMeta.java │ │ ├── ShuffleFileInfo.java │ │ ├── ShuffleInfo.java │ │ ├── ShuffleSegment.java │ │ ├── Storage.java │ │ ├── StorageMediaProvider.java │ │ ├── StorageReadMetrics.java │ │ └── StorageWriteMetrics.java │ │ ├── factory │ │ └── ShuffleHandlerFactory.java │ │ ├── handler │ │ ├── AsynDeletionEvent.java │ │ ├── ClientReadHandlerMetric.java │ │ ├── api │ │ │ ├── ClientReadHandler.java │ │ │ ├── ServerReadHandler.java │ │ │ ├── ShuffleDeleteHandler.java │ │ │ ├── ShuffleWriteHandler.java │ │ │ └── ShuffleWriteHandlerWrapper.java │ │ └── impl │ │ │ ├── AbstractClientReadHandler.java │ │ │ ├── AsynDeletionEventManager.java │ │ │ ├── ComposedClientReadHandler.java │ │ │ ├── CompositeLocalFileServerReadHandler.java │ │ │ ├── DataFileSegment.java │ │ │ ├── DataSkippableReadHandler.java │ │ │ ├── FileSegment.java │ │ │ ├── HadoopClientReadHandler.java │ │ │ ├── HadoopFileReader.java │ │ │ ├── HadoopFileWriter.java │ │ │ ├── HadoopShuffleDeleteHandler.java │ │ │ ├── HadoopShuffleReadHandler.java │ │ │ ├── HadoopShuffleWriteHandler.java │ │ │ ├── LocalFileAsyncDeleteHandler.java │ │ │ ├── LocalFileBlackHoleWriter.java │ │ │ ├── LocalFileClientReadHandler.java │ │ │ ├── LocalFileDeleteHandler.java │ │ │ ├── LocalFileNioWriter.java │ │ │ ├── LocalFileNullDeviceWriter.java │ │ │ ├── LocalFileReader.java │ │ │ ├── LocalFileServerReadHandler.java │ │ │ ├── LocalFileWriteHandler.java │ │ │ ├── LocalFileWriter.java │ │ │ ├── LocalFileWriterFactory.java │ │ │ ├── MemoryClientReadHandler.java │ │ │ ├── MultiReplicaClientReadHandler.java │ │ │ ├── PooledHadoopShuffleWriteHandler.java │ │ │ ├── PrefetchableClientReadHandler.java │ │ │ ├── ShuffleServerReadCost.java │ │ │ └── ShuffleServerReadCostTracker.java │ │ ├── request │ │ ├── CreateShuffleDeleteHandlerRequest.java │ │ ├── CreateShuffleReadHandlerRequest.java │ │ └── CreateShuffleWriteHandlerRequest.java │ │ └── util │ │ ├── ShuffleStorageUtils.java │ │ └── StorageType.java │ └── test │ └── java │ └── org │ └── apache │ └── uniffle │ └── storage │ ├── HadoopShuffleHandlerTestBase.java │ ├── HadoopTestBase.java │ ├── common │ ├── DefaultStorageMediaProviderTest.java │ ├── LocalStorageTest.java │ └── ShuffleFileInfoTest.java │ ├── handler │ └── impl │ │ ├── HadoopClientReadHandlerTest.java │ │ ├── HadoopFileReaderTest.java │ │ ├── HadoopFileWriterTest.java │ │ ├── HadoopHandlerTest.java │ │ ├── HadoopShuffleReadHandlerTest.java │ │ ├── KerberizedHadoopClientReadHandlerTest.java │ │ ├── KerberizedHadoopShuffleReadHandlerTest.java │ │ ├── LocalFileHandlerTest.java │ │ ├── LocalFileHandlerTestBase.java │ │ ├── LocalFileServerReadHandlerTest.java │ │ ├── PooledHadoopShuffleWriteHandlerTest.java │ │ └── PrefetchableClientReadHandlerTest.java │ └── util │ ├── ShuffleHadoopStorageUtilsTest.java │ ├── ShuffleKerberizedHadoopStorageUtilsTest.java │ ├── ShuffleStorageUtilsTest.java │ └── StorageTypeTest.java └── tools └── client-simulation-yarn ├── README.md ├── pom.xml └── src └── main └── java └── org └── apache └── uniffle └── client └── simulator ├── Constants.java ├── HadoopConfigApp.java ├── NMCallBackHandler.java ├── UniffleClientSimOnYarnAppMaster.java ├── UniffleClientSimOnYarnClient.java ├── UniffleTask.java └── Utils.java /.baseline/copyright/apache-license-header.txt: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | *.ipr 2 | *.iml 3 | *.iws 4 | *.pyc 5 | *.pyo 6 | .idea/ 7 | .idea_modules/ 8 | .settings 9 | .cache 10 | .DS_Store 11 | .project 12 | .classpath 13 | .scala_dependencies 14 | .mvn/wrapper/maven-wrapper.jar 15 | docs/api/ 16 | lib_managed/ 17 | logs/ 18 | src_managed/ 19 | target/ 20 | reports/ 21 | metastore_db/ 22 | derby.log 23 | dependency-reduced-pom.xml 24 | rss-*.tgz 25 | hadoop-*.tar.gz 26 | deploy/kubernetes/docker/hadoopconfig/* 27 | *.exe 28 | *.exe~ 29 | *.dll 30 | *.so 31 | *.dylib 32 | vendor 33 | VERSION 34 | testbin/* 35 | *.test 36 | *.out 37 | !vendor/**/zz_generated.* 38 | *.swp 39 | *.swo 40 | *~ 41 | deploy/kubernetes/operator/bin 42 | deploy/kubernetes/operator/local 43 | common/build/ 44 | integration-test/common/build/ 45 | storage/build/ 46 | build/ 47 | deploy/kubernetes/integration-test/e2e/rss.yaml 48 | deploy/kubernetes/integration-test/e2e/rss-controller.yaml 49 | deploy/kubernetes/integration-test/e2e/rss-webhook.yaml 50 | dashboard/src/main/webapp/node_modules/ 51 | dashboard/src/main/webapp/node/ 52 | dashboard/src/main/webapp/dist/ 53 | dashboard/src/main/resources/static/ 54 | dashboard/src/main/webapp/package-lock.json 55 | rust/experimental/server/target 56 | rust/experimental/server/.idea 57 | rust/experimental/server/._target 58 | rust/experimental/server/src/proto/uniffle.rs 59 | .flattened-pom.xml 60 | -------------------------------------------------------------------------------- /.mvn/wrapper/maven-wrapper.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.9.6/apache-maven-3.9.6-bin.zip 18 | wrapperUrl=https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.2.0/maven-wrapper-3.2.0.jar 19 | -------------------------------------------------------------------------------- /DISCLAIMER: -------------------------------------------------------------------------------- 1 | Apache Uniffle is an effort undergoing incubation at The Apache 2 | Software Foundation (ASF), sponsored by the Apache Incubator PMC. 3 | 4 | Incubation is required of all newly accepted projects until a further review 5 | indicates that the infrastructure, communications, and decision-making process 6 | have stabilized in a manner consistent with other successful ASF projects. 7 | 8 | While incubation status is not necessarily a reflection of the completeness 9 | or stability of the code, it does indicate that the project has yet to be 10 | fully endorsed by the ASF. 11 | 12 | -------------------------------------------------------------------------------- /NOTICE: -------------------------------------------------------------------------------- 1 | Apache Uniffle 2 | Copyright 2022 and onwards The Apache Software Foundation. 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (https://www.apache.org/). 6 | 7 | The initial codebase was donated to the ASF by Tencent, copyright 2020-2022. 8 | -------------------------------------------------------------------------------- /bin/restart-coordinator.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | set -o pipefail 21 | set -o nounset # exit the script if you try to use an uninitialised variable 22 | set -o errexit # exit the script if any statement returns a non-true return value 23 | 24 | source "$(dirname "$0")/utils.sh" 25 | load_rss_env 26 | 27 | cd "${RSS_HOME}" 28 | 29 | bash ./bin/stop-coordinator.sh 30 | sleep 3 31 | bash ./bin/start-coordinator.sh 32 | 33 | exit 0 34 | -------------------------------------------------------------------------------- /bin/restart-shuffle-server.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | set -o pipefail 21 | set -o nounset # exit the script if you try to use an uninitialised variable 22 | set -o errexit # exit the script if any statement returns a non-true return value 23 | 24 | source "$(dirname "$0")/utils.sh" 25 | load_rss_env 26 | 27 | cd "${RSS_HOME}" 28 | 29 | bash ./bin/stop-shuffle-server.sh 30 | sleep 3 31 | bash ./bin/start-shuffle-server.sh 32 | 33 | exit 0 34 | -------------------------------------------------------------------------------- /bin/stop-coordinator.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | set -o pipefail 21 | set -o nounset # exit the script if you try to use an uninitialised variable 22 | set -o errexit # exit the script if any statement returns a non-true return value 23 | 24 | source "$(dirname "$0")/utils.sh" 25 | load_rss_env 26 | 27 | common_shutdown "coordinator" "${RSS_PID_DIR}" 28 | -------------------------------------------------------------------------------- /bin/stop-dashboard.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | # 20 | 21 | set -o pipefail 22 | set -o nounset # exit the script if you try to use an uninitialised variable 23 | set -o errexit # exit the script if any statement returns a non-true return value 24 | 25 | source "$(dirname "$0")/utils.sh" 26 | load_rss_env 27 | 28 | common_shutdown "dashboard" "${RSS_PID_DIR}" 29 | -------------------------------------------------------------------------------- /bin/stop-shuffle-server.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | set -o pipefail 21 | set -o nounset # exit the script if you try to use an uninitialised variable 22 | set -o errexit # exit the script if any statement returns a non-true return value 23 | 24 | source "$(dirname "$0")/utils.sh" 25 | load_rss_env 26 | 27 | common_shutdown "shuffle-server" "${RSS_PID_DIR}" 28 | -------------------------------------------------------------------------------- /cli/src/main/java/org/apache/uniffle/CustomCommandLine.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle; 19 | 20 | import org.apache.commons.cli.Options; 21 | 22 | public interface CustomCommandLine { 23 | 24 | void addRunOptions(Options baseOptions); 25 | 26 | void addGeneralOptions(Options baseOptions); 27 | } 28 | -------------------------------------------------------------------------------- /cli/src/main/java/org/apache/uniffle/client/RestClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client; 19 | 20 | import java.util.Map; 21 | 22 | /** A underlying http client interface for common rest request. */ 23 | public interface RestClient extends AutoCloseable, Cloneable { 24 | 25 | String get(String path, Map params, String authHeader); 26 | 27 | String post(String path, Map params, String authHeader); 28 | } 29 | -------------------------------------------------------------------------------- /cli/src/main/java/org/apache/uniffle/client/exception/UniffleRestException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.exception; 19 | 20 | public class UniffleRestException extends RuntimeException { 21 | 22 | public UniffleRestException(String message, Throwable cause) { 23 | super(message, cause); 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /client-mr/core/src/main/java/org/apache/hadoop/mapreduce/MRIdHelper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.hadoop.mapreduce; 19 | 20 | import org.apache.uniffle.common.util.IdHelper; 21 | 22 | public class MRIdHelper implements IdHelper { 23 | 24 | @Override 25 | public long getTaskAttemptId(long blockId) { 26 | return RssMRUtils.getTaskAttemptId(blockId); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /client-mr/core/src/test/resources/log4j2.xml: -------------------------------------------------------------------------------- 1 | 2 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | -------------------------------------------------------------------------------- /client-spark/common/src/main/java/org/apache/spark/shuffle/events/UniffleEvent.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.shuffle.events; 19 | 20 | import org.apache.spark.scheduler.SparkListenerEvent; 21 | 22 | public abstract class UniffleEvent implements SparkListenerEvent { 23 | 24 | public boolean logEvent() { 25 | return true; 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BlockFailureCallback.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.shuffle.writer; 19 | 20 | import org.apache.uniffle.common.ShuffleBlockInfo; 21 | 22 | public interface BlockFailureCallback { 23 | void onBlockFailure(ShuffleBlockInfo block); 24 | } 25 | -------------------------------------------------------------------------------- /client-spark/common/src/main/java/org/apache/spark/shuffle/writer/BlockSuccessCallback.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.shuffle.writer; 19 | 20 | import org.apache.uniffle.common.ShuffleBlockInfo; 21 | 22 | public interface BlockSuccessCallback { 23 | void onBlockSuccess(ShuffleBlockInfo block); 24 | } 25 | -------------------------------------------------------------------------------- /client-spark/common/src/main/java/org/apache/spark/shuffle/writer/WrappedByteArrayOutputStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.shuffle.writer; 19 | 20 | import java.io.ByteArrayOutputStream; 21 | 22 | /** Subclass of ByteArrayOutputStream that exposes `buf` directly. */ 23 | public class WrappedByteArrayOutputStream extends ByteArrayOutputStream { 24 | 25 | public WrappedByteArrayOutputStream(int size) { 26 | super(size); 27 | } 28 | 29 | public byte[] getBuf() { 30 | return buf; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /client-spark/extension/src/main/resources/META-INF/services/org.apache.spark.status.AppHistoryServerPlugin: -------------------------------------------------------------------------------- 1 | org.apache.spark.UniffleHistoryServerPlugin -------------------------------------------------------------------------------- /client-spark/spark3/src/main/java/org/apache/spark/shuffle/Spark3VersionUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.spark.shuffle; 19 | 20 | import org.apache.spark.package$; 21 | 22 | public class Spark3VersionUtils extends SparkVersionUtils { 23 | public static final String SPARK_VERSION_SHORT = package$.MODULE$.SPARK_VERSION_SHORT(); 24 | 25 | public static boolean isSpark320() { 26 | return SPARK_VERSION_SHORT.equals("3.2.0"); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /client-tez/src/main/java/org/apache/tez/common/TezIdHelper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.tez.common; 19 | 20 | import org.apache.uniffle.common.util.IdHelper; 21 | 22 | public class TezIdHelper implements IdHelper { 23 | 24 | @Override 25 | public long getTaskAttemptId(long blockId) { 26 | return RssTezUtils.getTaskAttemptId(blockId); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /client-tez/src/test/resources/log4j2.xml: -------------------------------------------------------------------------------- 1 | 2 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | -------------------------------------------------------------------------------- /client/src/main/java/org/apache/uniffle/client/api/ShuffleReadClient.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.api; 19 | 20 | import org.apache.uniffle.client.response.CompressedShuffleBlock; 21 | 22 | public interface ShuffleReadClient { 23 | 24 | CompressedShuffleBlock readShuffleBlockData(); 25 | 26 | void checkProcessedBlockIds(); 27 | 28 | void close(); 29 | 30 | void logStatics(); 31 | } 32 | -------------------------------------------------------------------------------- /client/src/main/java/org/apache/uniffle/client/record/RecordCollection.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.record; 19 | 20 | import java.io.IOException; 21 | 22 | import org.apache.uniffle.common.records.RecordsWriter; 23 | 24 | public interface RecordCollection { 25 | 26 | void addRecord(K key, V value); 27 | 28 | void serialize(RecordsWriter writer) throws IOException; 29 | 30 | int size(); 31 | } 32 | -------------------------------------------------------------------------------- /client/src/main/java/org/apache/uniffle/client/record/metrics/MetricsReporter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.record.metrics; 19 | 20 | public interface MetricsReporter { 21 | void incRecordsRead(long v); 22 | } 23 | -------------------------------------------------------------------------------- /client/src/main/java/org/apache/uniffle/client/record/reader/KeyValueReader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.record.reader; 19 | 20 | import java.io.IOException; 21 | 22 | import org.apache.uniffle.client.record.Record; 23 | 24 | public abstract class KeyValueReader { 25 | 26 | public abstract boolean hasNext() throws IOException; 27 | 28 | public abstract Record next() throws IOException; 29 | } 30 | -------------------------------------------------------------------------------- /client/src/main/java/org/apache/uniffle/client/record/reader/KeyValuesReader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.record.reader; 19 | 20 | import java.io.IOException; 21 | 22 | public abstract class KeyValuesReader { 23 | 24 | public abstract boolean next() throws IOException; 25 | 26 | public abstract K getCurrentKey() throws IOException; 27 | 28 | public abstract Iterable getCurrentValues() throws IOException; 29 | } 30 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/Arguments.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common; 19 | 20 | import picocli.CommandLine.Option; 21 | 22 | public class Arguments { 23 | 24 | @Option( 25 | names = {"-c", "--conf"}, 26 | description = "config file") 27 | private String configFile; 28 | 29 | public String getConfigFile() { 30 | return this.configFile; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/AuditType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common; 19 | 20 | public enum AuditType { 21 | WRITE("w"), 22 | DELETE("d"); 23 | private String value; 24 | 25 | AuditType(String value) { 26 | this.value = value; 27 | } 28 | 29 | public String getValue() { 30 | return value; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/BlockCompletionCallback.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common; 19 | 20 | public interface BlockCompletionCallback { 21 | void onBlockCompletion(ShuffleBlockInfo block, boolean isSuccessful); 22 | } 23 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/ClientType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common; 19 | 20 | public enum ClientType { 21 | GRPC, 22 | GRPC_NETTY 23 | } 24 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/PartitionSplitMode.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common; 19 | 20 | /** The partition split mode for the partition split mechanism */ 21 | public enum PartitionSplitMode { 22 | // Reassign multi servers for one time to write for load balance 23 | LOAD_BALANCE, 24 | // Reassign server one by one once partition split 25 | PIPELINE, 26 | } 27 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/ShuffleDataDistributionType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common; 19 | 20 | /** The type of shuffle data distribution of a single partition. */ 21 | public enum ShuffleDataDistributionType { 22 | NORMAL, 23 | LOCAL_ORDER 24 | } 25 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/StorageType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common; 19 | 20 | public enum StorageType { 21 | MEMORY, 22 | LOCALFILE, 23 | MEMORY_LOCALFILE, 24 | HDFS, 25 | MEMORY_HDFS, 26 | LOCALFILE_HDFS, 27 | MEMORY_LOCALFILE_HDFS; 28 | } 29 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/exception/FileNotFoundException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.exception; 19 | 20 | public class FileNotFoundException extends RssException { 21 | 22 | public FileNotFoundException(String message) { 23 | super(message); 24 | } 25 | 26 | public FileNotFoundException(String message, Throwable e) { 27 | super(message, e); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/exception/InvalidRequestException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.exception; 19 | 20 | public class InvalidRequestException extends RssException { 21 | 22 | public InvalidRequestException(String message) { 23 | super(message); 24 | } 25 | 26 | public InvalidRequestException(String message, Throwable e) { 27 | super(message, e); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/exception/NoBufferException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.exception; 19 | 20 | public class NoBufferException extends RssException { 21 | public NoBufferException(String message) { 22 | super(message); 23 | } 24 | 25 | public NoBufferException(Throwable e) { 26 | super(e); 27 | } 28 | 29 | public NoBufferException(String message, Throwable e) { 30 | super(message, e); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/exception/NoBufferForHugePartitionException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.exception; 19 | 20 | public class NoBufferForHugePartitionException extends RssException { 21 | public NoBufferForHugePartitionException(String message) { 22 | super(message); 23 | } 24 | 25 | public NoBufferForHugePartitionException(Throwable e) { 26 | super(e); 27 | } 28 | 29 | public NoBufferForHugePartitionException(String message, Throwable e) { 30 | super(message, e); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/exception/NoRegisterException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.exception; 19 | 20 | public class NoRegisterException extends NotRetryException { 21 | public NoRegisterException(String message) { 22 | super(message); 23 | } 24 | 25 | public NoRegisterException(Throwable e) { 26 | super(e); 27 | } 28 | 29 | public NoRegisterException(String message, Throwable e) { 30 | super(message, e); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/exception/NotRetryException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.exception; 19 | 20 | public class NotRetryException extends RssException { 21 | 22 | public NotRetryException(String message) { 23 | super(message); 24 | } 25 | 26 | public NotRetryException(Throwable e) { 27 | super(e); 28 | } 29 | 30 | public NotRetryException(String message, Throwable e) { 31 | super(message, e); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/exception/RssException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.exception; 19 | 20 | public class RssException extends RuntimeException { 21 | 22 | public RssException(String message) { 23 | super(message); 24 | } 25 | 26 | public RssException(Throwable e) { 27 | super(e); 28 | } 29 | 30 | public RssException(String message, Throwable e) { 31 | super(message, e); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/exception/RssFetchFailedException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.exception; 19 | 20 | /** Dedicated exception for rss client's shuffle failed related exception. */ 21 | public class RssFetchFailedException extends RssException { 22 | public RssFetchFailedException(String message) { 23 | super(message); 24 | } 25 | 26 | public RssFetchFailedException(String message, Throwable e) { 27 | super(message, e); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/exception/RssSendFailedException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.exception; 19 | 20 | public class RssSendFailedException extends RssException { 21 | public RssSendFailedException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssSendFailedException(Throwable e) { 26 | super(e); 27 | } 28 | 29 | public RssSendFailedException(String message, Throwable e) { 30 | super(message, e); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/exception/RssWaitFailedException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.exception; 19 | 20 | public class RssWaitFailedException extends RssException { 21 | public RssWaitFailedException(String message) { 22 | super(message); 23 | } 24 | 25 | public RssWaitFailedException(Throwable e) { 26 | super(e); 27 | } 28 | 29 | public RssWaitFailedException(String message, Throwable e) { 30 | super(message, e); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/function/ConsumerWithException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.function; 19 | 20 | @FunctionalInterface 21 | public interface ConsumerWithException { 22 | void accept(T t) throws Exception; 23 | } 24 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/function/TupleConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.function; 19 | 20 | @FunctionalInterface 21 | public interface TupleConsumer { 22 | void accept(T t, F f); 23 | } 24 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/merger/KeyValueIterator.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.merger; 19 | 20 | import java.io.IOException; 21 | 22 | public interface KeyValueIterator { 23 | 24 | K getCurrentKey(); 25 | 26 | V getCurrentValue(); 27 | 28 | boolean next() throws IOException; 29 | 30 | void close() throws IOException; 31 | } 32 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/merger/MergeState.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.merger; 19 | 20 | public enum MergeState { 21 | DONE(0), 22 | INITED(1), 23 | MERGING(2), 24 | INTERNAL_ERROR(3); 25 | 26 | private final int code; 27 | 28 | MergeState(int code) { 29 | this.code = code; 30 | } 31 | 32 | public int code() { 33 | return code; 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/metrics/EmptyGRPCMetrics.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.metrics; 19 | 20 | import org.apache.uniffle.common.config.RssConf; 21 | 22 | public class EmptyGRPCMetrics extends GRPCMetrics { 23 | 24 | public EmptyGRPCMetrics(RssConf rssConf, String tags) { 25 | super(rssConf, tags); 26 | } 27 | 28 | @Override 29 | public void registerMetrics() {} 30 | } 31 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/metrics/MetricReporter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.metrics; 19 | 20 | import io.prometheus.client.CollectorRegistry; 21 | 22 | public interface MetricReporter { 23 | 24 | void start(); 25 | 26 | void stop(); 27 | 28 | void addCollectorRegistry(CollectorRegistry registry); 29 | } 30 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/netty/DecodeException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.netty; 19 | 20 | import org.apache.uniffle.common.exception.RssException; 21 | 22 | public class DecodeException extends RssException { 23 | 24 | public DecodeException(String message) { 25 | super(message); 26 | } 27 | 28 | public DecodeException(Throwable e) { 29 | super(e); 30 | } 31 | 32 | public DecodeException(String message, Throwable e) { 33 | super(message, e); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/netty/EncodeException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.netty; 19 | 20 | import org.apache.uniffle.common.exception.RssException; 21 | 22 | public class EncodeException extends RssException { 23 | 24 | public EncodeException(String message) { 25 | super(message); 26 | } 27 | 28 | public EncodeException(Throwable e) { 29 | super(e); 30 | } 31 | 32 | public EncodeException(String message, Throwable e) { 33 | super(message, e); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/netty/FrameDecoder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.netty; 19 | 20 | public interface FrameDecoder { 21 | String HANDLER_NAME = "FrameDecoder"; 22 | int HEADER_SIZE = Integer.BYTES + Byte.BYTES + Integer.BYTES; 23 | } 24 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/netty/IOMode.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.netty; 19 | 20 | public enum IOMode { 21 | NIO, 22 | EPOLL 23 | } 24 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/netty/handle/BaseMessageHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.netty.handle; 19 | 20 | import org.apache.uniffle.common.netty.client.TransportClient; 21 | import org.apache.uniffle.common.netty.protocol.RequestMessage; 22 | 23 | public interface BaseMessageHandler { 24 | 25 | void receive(TransportClient client, RequestMessage msg); 26 | 27 | void exceptionCaught(Throwable cause, TransportClient client); 28 | } 29 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/netty/protocol/Encodable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.netty.protocol; 19 | 20 | import io.netty.buffer.ByteBuf; 21 | 22 | import org.apache.uniffle.common.netty.EncodeException; 23 | 24 | public interface Encodable { 25 | 26 | int encodedLength(); 27 | 28 | void encode(ByteBuf buf) throws EncodeException; 29 | } 30 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/rpc/ServerInterface.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.rpc; 19 | 20 | import java.io.IOException; 21 | 22 | public interface ServerInterface { 23 | 24 | int start() throws IOException; 25 | 26 | void startOnPort(int port) throws Exception; 27 | 28 | void stop() throws InterruptedException; 29 | 30 | void blockUntilShutdown() throws InterruptedException; 31 | } 32 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/rpc/ServerType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.rpc; 19 | 20 | /** This should sync/match with how ClientType changes */ 21 | public enum ServerType { 22 | GRPC, 23 | GRPC_NETTY, 24 | } 25 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/security/SecurityContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.security; 19 | 20 | import java.io.Closeable; 21 | import java.util.concurrent.Callable; 22 | 23 | /** A security context with may be required to run a Callable. */ 24 | public interface SecurityContext extends Closeable { 25 | 26 | T runSecured(String user, Callable securedCallable) throws Exception; 27 | 28 | String getContextLoginUser(); 29 | } 30 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/segment/SegmentSplitter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.segment; 19 | 20 | import java.util.List; 21 | 22 | import org.apache.uniffle.common.ShuffleDataSegment; 23 | import org.apache.uniffle.common.ShuffleIndexResult; 24 | 25 | public interface SegmentSplitter { 26 | 27 | List split(ShuffleIndexResult shuffleIndexResult); 28 | } 29 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/serializer/SerializationStream.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.serializer; 19 | 20 | import java.io.IOException; 21 | 22 | public abstract class SerializationStream { 23 | 24 | public abstract void init(); 25 | 26 | public abstract void writeRecord(Object key, Object value) throws IOException; 27 | 28 | public abstract void flush() throws IOException; 29 | 30 | public abstract void close() throws IOException; 31 | 32 | public abstract long getTotalBytesWritten(); 33 | } 34 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/serializer/Serializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.serializer; 19 | 20 | public abstract class Serializer { 21 | 22 | public abstract SerializerInstance newInstance(); 23 | 24 | public abstract boolean accept(Class c); 25 | } 26 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/util/IdHelper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.util; 19 | 20 | public interface IdHelper { 21 | 22 | long getTaskAttemptId(long blockId); 23 | } 24 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/util/StatefulCloseable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.util; 19 | 20 | import java.io.Closeable; 21 | 22 | /** StatefulCloseable is an interface that utilizes the ExpiringCloseableSupplier delegate. */ 23 | public interface StatefulCloseable extends Closeable { 24 | boolean isClosed(); 25 | } 26 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/util/TripleFunction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.util; 19 | 20 | @FunctionalInterface 21 | public interface TripleFunction { 22 | 23 | R accept(T t, U u, E e); 24 | } 25 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/web/resource/Authorization.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.web.resource; 19 | 20 | import java.lang.annotation.ElementType; 21 | import java.lang.annotation.Retention; 22 | import java.lang.annotation.RetentionPolicy; 23 | import java.lang.annotation.Target; 24 | 25 | import org.apache.hbase.thirdparty.javax.ws.rs.NameBinding; 26 | 27 | @NameBinding 28 | @Target({ElementType.METHOD, ElementType.TYPE}) 29 | @Retention(RetentionPolicy.RUNTIME) 30 | public @interface Authorization {} 31 | -------------------------------------------------------------------------------- /common/src/main/java/org/apache/uniffle/common/web/resource/BaseResource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common.web.resource; 19 | 20 | import java.util.concurrent.Callable; 21 | 22 | public abstract class BaseResource { 23 | protected Response execute(Callable callable) { 24 | try { 25 | return Response.success(callable.call()); 26 | } catch (Throwable e) { 27 | return Response.fail(e.getMessage()); 28 | } 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /common/src/main/resources/META-INF/services/org.apache.hbase.thirdparty.org.glassfish.jersey.internal.spi.AutoDiscoverable: -------------------------------------------------------------------------------- 1 | org.apache.uniffle.common.web.JerseyAutoDiscoverable 2 | -------------------------------------------------------------------------------- /common/src/test/java/org/apache/uniffle/common/ShuffleIndexResultTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.common; 19 | 20 | import org.junit.jupiter.api.Test; 21 | 22 | import static org.junit.jupiter.api.Assertions.assertTrue; 23 | 24 | public class ShuffleIndexResultTest { 25 | 26 | @Test 27 | public void testEmpty() { 28 | assertTrue(new ShuffleIndexResult().isEmpty()); 29 | assertTrue(new ShuffleIndexResult((byte[]) null, -1).isEmpty()); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /common/src/test/resources/rss-defaults.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | # Default system properties included when running rss. 19 | # This is useful for setting default environmental settings. 20 | 21 | # Example: 22 | rss.coordinator.port 12121 23 | rss.server.heartbeat.interval 155 24 | rss.x.y.z true 25 | rss.a.b.c.extraJavaOptions -XX:+PrintGCDetails-Dkey=value-Dnumbers="one two three" 26 | -------------------------------------------------------------------------------- /common/src/test/resources/server.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | rss.jetty.http.port 9527 19 | -------------------------------------------------------------------------------- /conf/coordinator.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | rss.rpc.server.port 19999 19 | rss.jetty.http.port 19998 20 | rss.coordinator.server.heartbeat.timeout 30000 21 | rss.coordinator.app.expired 60000 22 | rss.coordinator.shuffle.nodes.max 13 23 | rss.coordinator.exclude.nodes.file.path file:///xxx 24 | rss.coordinator.select.partition.strategy CONTINUOUS 25 | -------------------------------------------------------------------------------- /conf/dashboard.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | rss.dashboard.http.port 19997 19 | coordinator.web.address http://coordinator.hostname00:19998/,http://coordinator.hostname01:19998/,http://coordinator.hostname02:19998/ 20 | -------------------------------------------------------------------------------- /conf/server.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | rss.rpc.server.port 19999 19 | rss.jetty.http.port 19998 20 | rss.storage.basePath /xxx,/xxx 21 | rss.storage.type MEMORY_LOCALFILE_HDFS 22 | rss.coordinator.quorum xxx:19999,xxx:19999 23 | rss.server.buffer.capacity 40gb 24 | rss.server.read.buffer.capacity 20gb 25 | rss.server.flush.thread.alive 5 26 | rss.server.flush.localfile.threadPool.size 10 27 | rss.server.flush.hadoop.threadPool.size 60 28 | rss.server.disk.capacity 1t 29 | rss.server.single.buffer.flush.enabled true 30 | rss.server.single.buffer.flush.threshold 128m 31 | -------------------------------------------------------------------------------- /coordinator/src/main/java/org/apache/uniffle/coordinator/conf/ClientConfParser.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.coordinator.conf; 19 | 20 | import java.io.InputStream; 21 | 22 | public interface ClientConfParser { 23 | enum Parser { 24 | YAML, 25 | LEGACY, 26 | MIXED 27 | } 28 | 29 | ClientConf tryParse(InputStream fileInputStream) throws Exception; 30 | } 31 | -------------------------------------------------------------------------------- /coordinator/src/main/java/org/apache/uniffle/coordinator/strategy/host/BasicHostAssignmentStrategy.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.coordinator.strategy.host; 19 | 20 | import java.util.List; 21 | 22 | import org.apache.uniffle.coordinator.ServerNode; 23 | 24 | public class BasicHostAssignmentStrategy implements HostAssignmentStrategy { 25 | @Override 26 | public List assign(List allNodes, int expectNum) { 27 | return allNodes.subList(0, expectNum); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /coordinator/src/main/java/org/apache/uniffle/coordinator/strategy/host/HostAssignmentStrategy.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.coordinator.strategy.host; 19 | 20 | import java.util.List; 21 | 22 | import org.apache.uniffle.coordinator.ServerNode; 23 | 24 | public interface HostAssignmentStrategy { 25 | List assign(List allNodes, int expectNum); 26 | } 27 | -------------------------------------------------------------------------------- /coordinator/src/main/java/org/apache/uniffle/coordinator/strategy/storage/SelectStorageStrategy.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.coordinator.strategy.storage; 19 | 20 | import org.apache.uniffle.common.RemoteStorageInfo; 21 | 22 | public interface SelectStorageStrategy { 23 | 24 | void detectStorage(); 25 | 26 | RemoteStorageInfo pickStorage(String appId); 27 | } 28 | -------------------------------------------------------------------------------- /coordinator/src/main/java/org/apache/uniffle/coordinator/web/lombok.config: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | clear lombok.allArgsConstructor.flagUsage 18 | clear lombok.anyConstructor.flagUsage 19 | clear lombok.noArgsConstructor.flagUsage 20 | clear lombok.data.flagUsage 21 | clear lombok.builder.flagUsage 22 | -------------------------------------------------------------------------------- /coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/CancelDecommissionRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.coordinator.web.request; 19 | 20 | import java.util.Set; 21 | 22 | public class CancelDecommissionRequest { 23 | private Set serverIds; 24 | 25 | public Set getServerIds() { 26 | return serverIds; 27 | } 28 | 29 | public void setServerIds(Set serverIds) { 30 | this.serverIds = serverIds; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /coordinator/src/main/java/org/apache/uniffle/coordinator/web/request/DecommissionRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.coordinator.web.request; 19 | 20 | import java.util.Set; 21 | 22 | public class DecommissionRequest { 23 | private Set serverIds; 24 | 25 | public Set getServerIds() { 26 | return serverIds; 27 | } 28 | 29 | public void setServerIds(Set serverIds) { 30 | this.serverIds = serverIds; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /coordinator/src/main/java/org/apache/uniffle/coordinator/web/vo/CoordinatorConfVO.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.coordinator.web.vo; 19 | 20 | import lombok.AllArgsConstructor; 21 | import lombok.Data; 22 | 23 | @Data 24 | @AllArgsConstructor 25 | public class CoordinatorConfVO { 26 | private String argumentKey; 27 | private String argumentValue; 28 | } 29 | -------------------------------------------------------------------------------- /coordinator/src/main/java/org/apache/uniffle/coordinator/web/vo/UserAppNumVO.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.coordinator.web.vo; 19 | 20 | import lombok.AllArgsConstructor; 21 | import lombok.Data; 22 | 23 | @Data 24 | @AllArgsConstructor 25 | public class UserAppNumVO implements Comparable { 26 | 27 | private String userName; 28 | private Integer appNum; 29 | 30 | @Override 31 | public int compareTo(UserAppNumVO userAppNumVO) { 32 | return Integer.compare(appNum, userAppNumVO.getAppNum()); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /coordinator/src/test/resources/dynamicClientConf.legacy: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | k1 v1 19 | k2 v2 20 | 21 | rss.coordinator.remote.storage.path hdfs://x-ns01,hdfs://a-ns01 22 | rss.coordinator.remote.storage.cluster.conf x-ns01,k1=v1,v2,v3,k2=v4,k3=v5,v6;a-ns01,k1=v1,k2=v2 23 | 24 | -------------------------------------------------------------------------------- /coordinator/src/test/resources/empty: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/coordinator/src/test/resources/empty -------------------------------------------------------------------------------- /coordinator/src/test/resources/quotaFile.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | user1 =10 19 | user2= 20 20 | user3 = 30 21 | disable_quota_user1=-1 22 | blank_user1=0 23 | -------------------------------------------------------------------------------- /dashboard/src/main/java/org/apache/uniffle/dashboard/web/lombok.config: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | clear lombok.allArgsConstructor.flagUsage 18 | clear lombok.anyConstructor.flagUsage 19 | clear lombok.noArgsConstructor.flagUsage 20 | clear lombok.data.flagUsage 21 | clear lombok.builder.flagUsage 22 | -------------------------------------------------------------------------------- /dashboard/src/main/java/org/apache/uniffle/dashboard/web/vo/DashboardConfVO.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.dashboard.web.vo; 19 | 20 | import lombok.AllArgsConstructor; 21 | import lombok.Data; 22 | 23 | @Data 24 | @AllArgsConstructor 25 | public class DashboardConfVO { 26 | private String argumentKey; 27 | private String argumentValue; 28 | } 29 | -------------------------------------------------------------------------------- /dashboard/src/main/webapp/.eslintignore: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | /node_modules 19 | /package-lock.json 20 | /dist 21 | .DS_Store 22 | -------------------------------------------------------------------------------- /dashboard/src/main/webapp/.eslintrc.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | module.exports = { 19 | env: { 20 | browser: true, 21 | commonjs: true, 22 | es2021: true, 23 | node: true 24 | }, 25 | extends: ['plugin:vue/essential', 'standard', '@vue/prettier'], 26 | parserOptions: { 27 | ecmaVersion: 13 28 | }, 29 | plugins: ['vue'], 30 | rules: { 31 | quotes: [2, 'single'] 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /dashboard/src/main/webapp/.prettierrc.json: -------------------------------------------------------------------------------- 1 | { 2 | "___asflicense__": [ 3 | "", 4 | "Licensed to the Apache Software Foundation (ASF) under one", 5 | "or more contributor license agreements. See the NOTICE file", 6 | "distributed with this work for additional information", 7 | "regarding copyright ownership. The ASF licenses this file", 8 | "to you under the Apache License, Version 2.0 (the", 9 | "\"License\"); you may not use this file except in compliance", 10 | "with the License. You may obtain a copy of the License at", 11 | "", 12 | " http://www.apache.org/licenses/LICENSE-2.0", 13 | "", 14 | "Unless required by applicable law or agreed to in writing, software", 15 | "distributed under the License is distributed on an \"AS IS\" BASIS,", 16 | "WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.", 17 | "See the License for the specific language governing permissions and", 18 | "limitations under the License." 19 | ], 20 | "semi": false, 21 | "tabWidth": 2, 22 | "singleQuote": true, 23 | "printWidth": 100, 24 | "trailingComma": "none" 25 | } 26 | -------------------------------------------------------------------------------- /dashboard/src/main/webapp/babel.config.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | module.exports = { 19 | presets: ['@vue/cli-plugin-babel/preset'] 20 | } 21 | -------------------------------------------------------------------------------- /dashboard/src/main/webapp/jsconfig.json: -------------------------------------------------------------------------------- 1 | { 2 | "___asflicense__": [ 3 | "", 4 | "Licensed to the Apache Software Foundation (ASF) under one", 5 | "or more contributor license agreements. See the NOTICE file", 6 | "distributed with this work for additional information", 7 | "regarding copyright ownership. The ASF licenses this file", 8 | "to you under the Apache License, Version 2.0 (the", 9 | "\"License\"); you may not use this file except in compliance", 10 | "with the License. You may obtain a copy of the License at", 11 | "", 12 | " http://www.apache.org/licenses/LICENSE-2.0", 13 | "", 14 | "Unless required by applicable law or agreed to in writing, software", 15 | "distributed under the License is distributed on an \"AS IS\" BASIS,", 16 | "WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.", 17 | "See the License for the specific language governing permissions and", 18 | "limitations under the License." 19 | ], 20 | "compilerOptions": { 21 | "target": "es5", 22 | "module": "esnext", 23 | "baseUrl": "./", 24 | "moduleResolution": "node", 25 | "paths": { 26 | "@/*": ["src/*"] 27 | }, 28 | "lib": ["esnext", "dom", "dom.iterable", "scripthost"] 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /dashboard/src/main/webapp/packagescript/filecopy.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | const fileUtil = require('./fileutils') 19 | 20 | // Destination folder 21 | const staticDirectory = '../resources/static' 22 | // Delete 23 | fileUtil.deleteFolder(staticDirectory) 24 | // Copy 25 | fileUtil.copyFolder('./dist', staticDirectory) 26 | console.log('File copy successful!') 27 | -------------------------------------------------------------------------------- /dashboard/src/main/webapp/public/favicon.ico: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/dashboard/src/main/webapp/public/favicon.ico -------------------------------------------------------------------------------- /dashboard/src/main/webapp/src/App.vue: -------------------------------------------------------------------------------- 1 | 17 | 18 | 21 | 22 | 31 | -------------------------------------------------------------------------------- /dashboard/src/main/webapp/src/assets/uniffle-logo.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/dashboard/src/main/webapp/src/assets/uniffle-logo.png -------------------------------------------------------------------------------- /dashboard/src/main/webapp/src/mock/index.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | import '@/mock/coordinatorserverpage' 19 | import '@/mock/layoutpage' 20 | import '@/mock/shuffleserverpage' 21 | import '@/mock/nodelistpage' 22 | import '@/mock/excludenodelist' 23 | import '@/mock/applicationpage' 24 | -------------------------------------------------------------------------------- /dashboard/src/main/webapp/src/mock/layoutpage.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | import Mock from 'mockjs' 19 | 20 | Mock.mock(/\/web\/coordinator\/coordinatorServers/, 'get', function (options) { 21 | return { 22 | code: 0, 23 | data: { 24 | 'sports.sina.com.cn': 'http://sports.sina.com.cn:29998', 25 | 'sports.faceback.com.cn': 'http://sports.faceback.com.cn:29998', 26 | 'sports.twiter.com.cn': 'http://sports.twiter.com.cn:29998' 27 | }, 28 | errMsg: 'success' 29 | } 30 | }) 31 | export default Mock 32 | -------------------------------------------------------------------------------- /dashboard/src/main/webapp/src/mock/shuffleserverpage.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | import Mock from 'mockjs' 19 | 20 | Mock.mock(/\/server\/nodes\/summary/, 'get', function (options) { 21 | return { 22 | code: 0, 23 | data: { 24 | ACTIVE: 10, 25 | DECOMMISSIONED: 1, 26 | DECOMMISSIONING: 1, 27 | EXCLUDED: 19, 28 | LOST: 12, 29 | UNHEALTHY: 10 30 | }, 31 | errMsg: 'success' 32 | } 33 | }) 34 | export default Mock 35 | -------------------------------------------------------------------------------- /dashboard/src/main/webapp/vue.config.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | module.exports = { 19 | // Proxies can be set up by configuring the vue.config.js file to proxy requests to the backend server. 20 | devServer: { 21 | host: 'localhost', 22 | port: 8080, 23 | proxy: { 24 | '/': { 25 | ws: false, 26 | target: 'http://localhost:19997', 27 | changeOrigin: true 28 | } 29 | } 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /deploy/docker/spark/Dockerfile: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | FROM apache/spark:3.5.0 19 | 20 | ENV PATH="${PATH}:/opt/spark/bin" 21 | 22 | USER root 23 | RUN mkdir -p /home/spark /shared; chown spark:spark /home/spark /shared 24 | COPY rss-client-spark3-shaded-*.jar /opt/spark/jars 25 | VOLUME /shared 26 | USER spark 27 | -------------------------------------------------------------------------------- /deploy/docker/uniffle/Dockerfile: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | FROM rss-server:head 19 | 20 | COPY start.sh /data/rssadmin/rss/bin 21 | COPY server.conf /data/rssadmin/rss/conf 22 | COPY dashboard.conf /data/rssadmin/rss/conf 23 | 24 | HEALTHCHECK --interval=3s --timeout=3s --start-period=60s CMD lsof -i:"${RPC_PORT}" -sTCP:LISTEN 25 | -------------------------------------------------------------------------------- /deploy/kubernetes/build-operator.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | echo "Build and Test kubernetes operator..." 21 | 22 | # todo: separate the build process from test process, we need to modify the Makefile 23 | cd operator && make build docker-build 24 | -------------------------------------------------------------------------------- /deploy/kubernetes/docker/base/debian/Dockerfile: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | # this docker file serves the rss server base image for Debian/Ubuntu releated distributions 19 | ARG BASE_IMAGE=eclipse-temurin:11-jdk 20 | FROM ${BASE_IMAGE} 21 | 22 | RUN apt-get update && \ 23 | apt-get install -y zlib1g zlib1g-dev lzop lsof netcat-openbsd dnsutils less procps iputils-ping && \ 24 | apt-get clean && rm -rf /var/lib/apt/lists/* 25 | -------------------------------------------------------------------------------- /deploy/kubernetes/docker/hadoopconfig/.gitkeep: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # -------------------------------------------------------------------------------- /deploy/kubernetes/integration-test/.gitkeep: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # -------------------------------------------------------------------------------- /deploy/kubernetes/integration-test/e2e/kind-config: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | kind: Cluster 19 | apiVersion: kind.x-k8s.io/v1alpha4 20 | nodes: 21 | - role: control-plane 22 | - role: worker 23 | - role: worker 24 | - role: worker 25 | - role: worker 26 | - role: worker 27 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/PROJECT: -------------------------------------------------------------------------------- 1 | domain: apache.org 2 | layout: 3 | - go.kubebuilder.io/v3 4 | projectName: operator 5 | repo: github.com/apache/incubator-uniffle/deploy/kubernetes/operator 6 | resources: 7 | - api: 8 | crdVersion: v1 9 | namespaced: true 10 | domain: apache.org 11 | group: uniffle 12 | kind: Remoteshuffleservice 13 | path: github.com/apache/incubator-uniffle/deploy/kubernetes/operator/api/uniffle/v1alpha1 14 | version: v1alpha1 15 | version: "3" 16 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/api/uniffle/v1alpha1/doc.go: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | // +k8s:deepcopy-gen=package 19 | // +groupName=uniffle.apache.org 20 | 21 | package v1alpha1 22 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/config/samples/uniffle_v1alpha1_remoteshuffleservice.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | --- 19 | apiVersion: uniffle.apache.org/v1alpha1 20 | kind: Remoteshuffleservice 21 | metadata: 22 | name: remoteshuffleservice-sample 23 | spec: 24 | # TODO(user): Add fields here 25 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/examples/pvc-example/gce-storage-class.yml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | --- 19 | apiVersion: storage.k8s.io/v1 20 | kind: StorageClass 21 | metadata: 22 | name: fast 23 | provisioner: kubernetes.io/gce-pd 24 | parameters: 25 | type: pd-ssd 26 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/hack/add_header_for_crds.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | set -e 21 | set -u 22 | 23 | for filename in config/crd/bases/*.yaml; do 24 | header=$(cat hack/headers/header.yaml.txt) 25 | echo "$header" > "$filename.tmp" 26 | cat "$filename" >> "$filename.tmp" 27 | mv "$filename.tmp" "$filename" 28 | done 29 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/hack/headers/header.go.txt: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/hack/headers/header.yaml.txt: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/hack/tools.go: -------------------------------------------------------------------------------- 1 | //go:build tools 2 | // +build tools 3 | 4 | /* 5 | * Licensed to the Apache Software Foundation (ASF) under one or more 6 | * contributor license agreements. See the NOTICE file distributed with 7 | * this work for additional information regarding copyright ownership. 8 | * The ASF licenses this file to You under the Apache License, Version 2.0 9 | * (the "License"); you may not use this file except in compliance with 10 | * the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | package hack 22 | 23 | import ( 24 | // Imports code-generator to force `go mod` to see them as dependencies. 25 | _ "k8s.io/code-generator" 26 | ) 27 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/pkg/generated/clientset/versioned/doc.go: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | // Code generated by client-gen. DO NOT EDIT. 19 | 20 | // This package has the automatically generated clientset. 21 | package versioned 22 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/pkg/generated/clientset/versioned/fake/doc.go: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | // Code generated by client-gen. DO NOT EDIT. 19 | 20 | // This package has the automatically generated fake clientset. 21 | package fake 22 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/pkg/generated/clientset/versioned/scheme/doc.go: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | // Code generated by client-gen. DO NOT EDIT. 19 | 20 | // This package contains the scheme of the automatically generated clientset. 21 | package scheme 22 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/pkg/generated/clientset/versioned/typed/uniffle/v1alpha1/doc.go: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | // Code generated by client-gen. DO NOT EDIT. 19 | 20 | // This package has the automatically generated typed clients. 21 | package v1alpha1 22 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/pkg/generated/clientset/versioned/typed/uniffle/v1alpha1/fake/doc.go: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | // Code generated by client-gen. DO NOT EDIT. 19 | 20 | // Package fake has the automatically generated clients. 21 | package fake 22 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/pkg/generated/clientset/versioned/typed/uniffle/v1alpha1/generated_expansion.go: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | // Code generated by client-gen. DO NOT EDIT. 19 | 20 | package v1alpha1 21 | 22 | type RemoteShuffleServiceExpansion interface{} 23 | -------------------------------------------------------------------------------- /deploy/kubernetes/operator/pkg/webhook/util/patch.go: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package util 19 | 20 | // PatchOperation defines the information of a patch's operation. 21 | type PatchOperation struct { 22 | Op string `json:"op"` 23 | Path string `json:"path"` 24 | Value interface{} `json:"value,omitempty"` 25 | } 26 | -------------------------------------------------------------------------------- /deploy/kubernetes/test-operator.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | echo "Test Kubernetes Operator.." 21 | -------------------------------------------------------------------------------- /dev/local_dev_template/coordinator.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | rss.rpc.server.port 19999 19 | rss.jetty.http.port 19998 20 | rss.coordinator.server.heartbeat.timeout 30000 21 | rss.coordinator.app.expired 60000 22 | rss.coordinator.shuffle.nodes.max 13 23 | rss.coordinator.select.partition.strategy CONTINUOUS 24 | -------------------------------------------------------------------------------- /dev/local_dev_template/dashboard.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | rss.dashboard.http.port 19997 19 | # change to the actual coordinator web addresses 20 | coordinator.web.address http://localhost:19998,http://localhost:19998 21 | -------------------------------------------------------------------------------- /docs/_config.yml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | title: Uniffle homepage 19 | baseurl: "" 20 | url: "" 21 | 22 | # Build settings 23 | theme: minima 24 | 25 | kramdown: 26 | math_engine: mathjax 27 | syntax_highlighter: rouge 28 | -------------------------------------------------------------------------------- /docs/asset/rss-crd-state-transition.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/docs/asset/rss-crd-state-transition.png -------------------------------------------------------------------------------- /docs/asset/rss_architecture.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/docs/asset/rss_architecture.png -------------------------------------------------------------------------------- /docs/asset/rss_benchmark1.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/docs/asset/rss_benchmark1.png -------------------------------------------------------------------------------- /docs/asset/rss_benchmark2.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/docs/asset/rss_benchmark2.png -------------------------------------------------------------------------------- /docs/asset/rss_benchmark3.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/docs/asset/rss_benchmark3.png -------------------------------------------------------------------------------- /docs/asset/rss_data_format.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/docs/asset/rss_data_format.png -------------------------------------------------------------------------------- /docs/asset/rss_remote_merge_architecture.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/docs/asset/rss_remote_merge_architecture.png -------------------------------------------------------------------------------- /docs/asset/rss_shuffle_write.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/docs/asset/rss_shuffle_write.png -------------------------------------------------------------------------------- /docs/asset/vanilla_benchmark1.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/docs/asset/vanilla_benchmark1.png -------------------------------------------------------------------------------- /docs/asset/vanilla_benchmark2.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/docs/asset/vanilla_benchmark2.png -------------------------------------------------------------------------------- /docs/asset/vanilla_benchmark3.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/docs/asset/vanilla_benchmark3.png -------------------------------------------------------------------------------- /docs/gen-doc.sh: -------------------------------------------------------------------------------- 1 | #! /usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | set -e 21 | 22 | BASE_DIR="$(cd "`dirname "$0"`/.."; pwd)" 23 | cd "${BASE_DIR}" 24 | ./mvnw clean javadoc:aggregate 25 | mv target/site/apidocs docs 26 | cd docs 27 | 28 | jekyll build 29 | 30 | -------------------------------------------------------------------------------- /integration-test/spark-common/src/test/resources/candidates: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | test_access_id 19 | -------------------------------------------------------------------------------- /internal-client/src/main/java/org/apache/uniffle/client/request/RetryableRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.request; 19 | 20 | public abstract class RetryableRequest { 21 | protected int retryMax; 22 | protected long retryIntervalMax; 23 | 24 | public int getRetryMax() { 25 | return retryMax; 26 | } 27 | 28 | public long getRetryIntervalMax() { 29 | return retryIntervalMax; 30 | } 31 | 32 | public abstract String operationType(); 33 | } 34 | -------------------------------------------------------------------------------- /internal-client/src/main/java/org/apache/uniffle/client/request/RssCancelDecommissionRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.request; 19 | 20 | public class RssCancelDecommissionRequest {} 21 | -------------------------------------------------------------------------------- /internal-client/src/main/java/org/apache/uniffle/client/request/RssDecommissionRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.request; 19 | 20 | public class RssDecommissionRequest {} 21 | -------------------------------------------------------------------------------- /internal-client/src/main/java/org/apache/uniffle/client/request/RssFetchRemoteStorageRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.request; 19 | 20 | public class RssFetchRemoteStorageRequest { 21 | 22 | private String appId; 23 | 24 | public RssFetchRemoteStorageRequest(String appId) { 25 | this.appId = appId; 26 | } 27 | 28 | public String getAppId() { 29 | return appId; 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /internal-client/src/main/java/org/apache/uniffle/client/response/RssAppHeartBeatResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.response; 19 | 20 | import org.apache.uniffle.common.rpc.StatusCode; 21 | 22 | public class RssAppHeartBeatResponse extends ClientResponse { 23 | 24 | public RssAppHeartBeatResponse(StatusCode statusCode) { 25 | super(statusCode); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /internal-client/src/main/java/org/apache/uniffle/client/response/RssApplicationInfoResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.response; 19 | 20 | import org.apache.uniffle.common.rpc.StatusCode; 21 | 22 | public class RssApplicationInfoResponse extends ClientResponse { 23 | 24 | public RssApplicationInfoResponse(StatusCode statusCode) { 25 | super(statusCode); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /internal-client/src/main/java/org/apache/uniffle/client/response/RssDecommissionResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.response; 19 | 20 | import org.apache.uniffle.common.rpc.StatusCode; 21 | 22 | public class RssDecommissionResponse extends ClientResponse { 23 | private String retMsg; 24 | 25 | public RssDecommissionResponse(StatusCode statusCode, String retMsg) { 26 | super(statusCode); 27 | this.retMsg = retMsg; 28 | } 29 | 30 | public String getRetMsg() { 31 | return retMsg; 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /internal-client/src/main/java/org/apache/uniffle/client/response/RssFinishShuffleResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.response; 19 | 20 | import org.apache.uniffle.common.rpc.StatusCode; 21 | 22 | public class RssFinishShuffleResponse extends ClientResponse { 23 | 24 | public RssFinishShuffleResponse(StatusCode statusCode) { 25 | super(statusCode); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /internal-client/src/main/java/org/apache/uniffle/client/response/RssRegisterShuffleResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.response; 19 | 20 | import org.apache.uniffle.common.rpc.StatusCode; 21 | 22 | public class RssRegisterShuffleResponse extends ClientResponse { 23 | 24 | public RssRegisterShuffleResponse(StatusCode statusCode) { 25 | super(statusCode); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /internal-client/src/main/java/org/apache/uniffle/client/response/RssStartSortMergeResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.response; 19 | 20 | import org.apache.uniffle.common.rpc.StatusCode; 21 | 22 | public class RssStartSortMergeResponse extends ClientResponse { 23 | 24 | public RssStartSortMergeResponse(StatusCode statusCode) { 25 | super(statusCode); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /internal-client/src/main/java/org/apache/uniffle/client/response/RssUnregisterShuffleByAppIdResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.response; 19 | 20 | import org.apache.uniffle.common.rpc.StatusCode; 21 | 22 | public class RssUnregisterShuffleByAppIdResponse extends ClientResponse { 23 | 24 | public RssUnregisterShuffleByAppIdResponse(StatusCode statusCode) { 25 | super(statusCode); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /internal-client/src/main/java/org/apache/uniffle/client/response/RssUnregisterShuffleResponse.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.client.response; 19 | 20 | import org.apache.uniffle.common.rpc.StatusCode; 21 | 22 | public class RssUnregisterShuffleResponse extends ClientResponse { 23 | 24 | public RssUnregisterShuffleResponse(StatusCode statusCode) { 25 | super(statusCode); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /licenses-binary/LICENSE-animal-sniffer.txt: -------------------------------------------------------------------------------- 1 | The MIT License 2 | 3 | Copyright (c) 2009 codehaus.org. 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in 13 | all copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 21 | THE SOFTWARE. 22 | 23 | -------------------------------------------------------------------------------- /licenses-binary/LICENSE-axios.txt: -------------------------------------------------------------------------------- 1 | # Copyright (c) 2014-present Matt Zabriskie & Collaborators 2 | 3 | Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: 4 | 5 | The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. 6 | 7 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. -------------------------------------------------------------------------------- /licenses-binary/LICENSE-checker-qual.txt: -------------------------------------------------------------------------------- 1 | Checker Framework qualifiers 2 | Copyright 2004-present by the Checker Framework developers 3 | 4 | MIT License: 5 | 6 | Permission is hereby granted, free of charge, to any person obtaining a copy 7 | of this software and associated documentation files (the "Software"), to deal 8 | in the Software without restriction, including without limitation the rights 9 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 10 | copies of the Software, and to permit persons to whom the Software is 11 | furnished to do so, subject to the following conditions: 12 | 13 | The above copyright notice and this permission notice shall be included in 14 | all copies or substantial portions of the Software. 15 | 16 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 17 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 18 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 19 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 20 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 21 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 22 | THE SOFTWARE. 23 | -------------------------------------------------------------------------------- /licenses-binary/LICENSE-core-js.txt: -------------------------------------------------------------------------------- 1 | Copyright (c) 2014-2024 Denis Pushkarev 2 | 3 | Permission is hereby granted, free of charge, to any person obtaining a copy 4 | of this software and associated documentation files (the "Software"), to deal 5 | in the Software without restriction, including without limitation the rights 6 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | copies of the Software, and to permit persons to whom the Software is 8 | furnished to do so, subject to the following conditions: 9 | 10 | The above copyright notice and this permission notice shall be included in 11 | all copies or substantial portions of the Software. 12 | 13 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | THE SOFTWARE. -------------------------------------------------------------------------------- /licenses-binary/LICENSE-element-plus.txt: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2020-PRESENT Element Plus 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. -------------------------------------------------------------------------------- /licenses-binary/LICENSE-javax-activation.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/uniffle/b45e986d44712dfdadd123e4c953a78fd088ac35/licenses-binary/LICENSE-javax-activation.txt -------------------------------------------------------------------------------- /licenses-binary/LICENSE-moment.txt: -------------------------------------------------------------------------------- 1 | Copyright (c) JS Foundation and other contributors 2 | 3 | Permission is hereby granted, free of charge, to any person 4 | obtaining a copy of this software and associated documentation 5 | files (the "Software"), to deal in the Software without 6 | restriction, including without limitation the rights to use, 7 | copy, modify, merge, publish, distribute, sublicense, and/or sell 8 | copies of the Software, and to permit persons to whom the 9 | Software is furnished to do so, subject to the following 10 | conditions: 11 | 12 | The above copyright notice and this permission notice shall be 13 | included in all copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, 16 | EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES 17 | OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND 18 | NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT 19 | HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, 20 | WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING 21 | FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR 22 | OTHER DEALINGS IN THE SOFTWARE. -------------------------------------------------------------------------------- /licenses-binary/LICENSE-rimraf.txt: -------------------------------------------------------------------------------- 1 | The ISC License 2 | 3 | Copyright (c) 2011-2023 Isaac Z. Schlueter and Contributors 4 | 5 | Permission to use, copy, modify, and/or distribute this software for any 6 | purpose with or without fee is hereby granted, provided that the above 7 | copyright notice and this permission notice appear in all copies. 8 | 9 | THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES 10 | WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF 11 | MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR 12 | ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES 13 | WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN 14 | ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR 15 | IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. -------------------------------------------------------------------------------- /licenses-binary/LICENSE-slf4j.txt: -------------------------------------------------------------------------------- 1 | Copyright (c) 2004-2022 QOS.ch Sarl (Switzerland) 2 | All rights reserved. 3 | 4 | Permission is hereby granted, free of charge, to any person obtaining 5 | a copy of this software and associated documentation files (the 6 | "Software"), to deal in the Software without restriction, including 7 | without limitation the rights to use, copy, modify, merge, publish, 8 | distribute, sublicense, and/or sell copies of the Software, and to 9 | permit persons to whom the Software is furnished to do so, subject to 10 | the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be 13 | included in all copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, 16 | EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF 17 | MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND 18 | NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE 19 | LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION 20 | OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION 21 | WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. 22 | 23 | 24 | 25 | -------------------------------------------------------------------------------- /licenses-binary/LICENSE-vue-resource.txt: -------------------------------------------------------------------------------- 1 | The MIT License (MIT) 2 | 3 | Copyright (c) 2015-2017 steffans 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in 13 | all copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 21 | THE SOFTWARE. -------------------------------------------------------------------------------- /licenses-binary/LICENSE-vue-router.txt: -------------------------------------------------------------------------------- 1 | MIT License 2 | 3 | Copyright (c) 2013-present Evan You 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in all 13 | copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE 21 | SOFTWARE. -------------------------------------------------------------------------------- /licenses-binary/LICENSE-vue.txt: -------------------------------------------------------------------------------- 1 | The MIT License (MIT) 2 | 3 | Copyright (c) 2013-present, Yuxi (Evan) You 4 | 5 | Permission is hereby granted, free of charge, to any person obtaining a copy 6 | of this software and associated documentation files (the "Software"), to deal 7 | in the Software without restriction, including without limitation the rights 8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 | copies of the Software, and to permit persons to whom the Software is 10 | furnished to do so, subject to the following conditions: 11 | 12 | The above copyright notice and this permission notice shall be included in 13 | all copies or substantial portions of the Software. 14 | 15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 21 | THE SOFTWARE. -------------------------------------------------------------------------------- /server/src/main/java/org/apache/uniffle/server/Checker.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.server; 19 | 20 | import com.google.common.annotations.VisibleForTesting; 21 | 22 | public abstract class Checker { 23 | 24 | Checker(ShuffleServerConf conf) {} 25 | 26 | @VisibleForTesting 27 | public abstract boolean checkIsHealthy(); 28 | } 29 | -------------------------------------------------------------------------------- /server/src/main/java/org/apache/uniffle/server/FlushEventHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.server; 19 | 20 | public interface FlushEventHandler { 21 | void handle(ShuffleDataFlushEvent event); 22 | 23 | int getEventNumInFlush(); 24 | 25 | void stop(); 26 | } 27 | -------------------------------------------------------------------------------- /server/src/main/java/org/apache/uniffle/server/buffer/RequireBufferStatusCode.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.server.buffer; 19 | 20 | public enum RequireBufferStatusCode { 21 | // means no buffer 22 | NO_BUFFER(-1L), 23 | // means app not registered, may be worker is down and should check. 24 | NO_REGISTER(-4L); 25 | 26 | private final long statusCode; 27 | 28 | RequireBufferStatusCode(long code) { 29 | this.statusCode = code; 30 | } 31 | 32 | public long statusCode() { 33 | return statusCode; 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /server/src/main/java/org/apache/uniffle/server/buffer/ShuffleBufferType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.server.buffer; 19 | 20 | public enum ShuffleBufferType { 21 | SKIP_LIST, 22 | LINKED_LIST 23 | } 24 | -------------------------------------------------------------------------------- /server/src/main/java/org/apache/uniffle/server/event/AppUnregisterPurgeEvent.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.server.event; 19 | 20 | public class AppUnregisterPurgeEvent extends PurgeEvent { 21 | public AppUnregisterPurgeEvent(String appId, String user) { 22 | super(appId, user, null); 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /server/src/main/java/org/apache/uniffle/server/merge/MergeEventHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.server.merge; 19 | 20 | public interface MergeEventHandler { 21 | 22 | boolean handle(MergeEvent event); 23 | 24 | int getEventNumInMerge(); 25 | 26 | void stop(); 27 | } 28 | -------------------------------------------------------------------------------- /server/src/main/java/org/apache/uniffle/server/storage/hybrid/StorageManagerSelector.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.server.storage.hybrid; 19 | 20 | import org.apache.uniffle.server.ShuffleDataFlushEvent; 21 | import org.apache.uniffle.server.storage.StorageManager; 22 | 23 | public interface StorageManagerSelector { 24 | 25 | StorageManager select(ShuffleDataFlushEvent flushEvent); 26 | 27 | enum ColdStoragePreferredFactor { 28 | HUGE_EVENT, 29 | OWNED_BY_HUGE_PARTITION, 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /server/src/main/java/org/apache/uniffle/server/web/lombok.config: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | clear lombok.allArgsConstructor.flagUsage 18 | clear lombok.anyConstructor.flagUsage 19 | clear lombok.noArgsConstructor.flagUsage 20 | clear lombok.data.flagUsage 21 | clear lombok.builder.flagUsage 22 | -------------------------------------------------------------------------------- /server/src/main/java/org/apache/uniffle/server/web/vo/ServerConfVO.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.server.web.vo; 19 | 20 | import lombok.AllArgsConstructor; 21 | import lombok.Data; 22 | 23 | @Data 24 | @AllArgsConstructor 25 | public class ServerConfVO { 26 | private String argumentKey; 27 | private String argumentValue; 28 | } 29 | -------------------------------------------------------------------------------- /server/src/main/resources/META-INF/services/org.apache.uniffle.storage.common.StorageMediaProvider: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | org.apache.uniffle.server.storage.StorageMediaFromEnvProvider 19 | org.apache.uniffle.storage.common.DefaultStorageMediaProvider 20 | -------------------------------------------------------------------------------- /server/src/test/java/org/apache/uniffle/server/HealthyMockChecker.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.server; 19 | 20 | public class HealthyMockChecker extends Checker { 21 | 22 | @SuppressWarnings("checkstyle:RedundantModifier") 23 | public HealthyMockChecker(ShuffleServerConf conf) { 24 | super(conf); 25 | } 26 | 27 | @Override 28 | public boolean checkIsHealthy() { 29 | return true; 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /server/src/test/java/org/apache/uniffle/server/MockedShuffleServer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.server; 19 | 20 | public class MockedShuffleServer extends ShuffleServer { 21 | public MockedShuffleServer(ShuffleServerConf shuffleServerConf) throws Exception { 22 | super(shuffleServerConf); 23 | } 24 | 25 | @Override 26 | public void setServer() { 27 | setServer(new MockedShuffleServerFactory(this).getServer()); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /server/src/test/java/org/apache/uniffle/server/UnHealthyMockChecker.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.server; 19 | 20 | public class UnHealthyMockChecker extends Checker { 21 | 22 | @SuppressWarnings("checkstyle:RedundantModifier") 23 | public UnHealthyMockChecker(ShuffleServerConf conf) { 24 | super(conf); 25 | } 26 | 27 | @Override 28 | public boolean checkIsHealthy() { 29 | return false; 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /server/src/test/resources/confBySizeStringTest.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | 19 | rss.server.buffer.capacity 10240 20 | rss.server.buffer.spill.threshold 1gb 21 | rss.server.partition.buffer.size 2MB 22 | rss.server.read.buffer.capacity 32kb 23 | rss.server.write.slow.threshold 10gb 24 | rss.server.event.size.threshold.l1 45mb 25 | rss.server.event.size.threshold.l2 90mb 26 | rss.server.event.size.threshold.l3 120mb 27 | rss.server.disk.capacity 120gb 28 | -------------------------------------------------------------------------------- /server/src/test/resources/confTest.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | rss.rpc.server.port 1234 19 | rss.storage.type LOCALFILE 20 | rss.storage.basePath /var/tmp/test 21 | rss.server.coordinator.ip localhost 22 | rss.server.coordinator.port 9527 23 | rss.server.buffer.capacity 2 24 | rss.server.hadoop.a.b value1 25 | rss.server.had.a.b value2 26 | rss.server.multistorage.enable true 27 | rss.server.cluster.hadoop.clustere1. 28 | plugin.custom.key v1 29 | -------------------------------------------------------------------------------- /server/src/test/resources/gcTest.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | rss.rpc.server.port 1234 19 | rss.server.coordinator.ip localhost 20 | rss.server.coordinator.port 9527 21 | rss.server.gc.threshold 3 22 | rss.server.buffer.capacity 12 23 | rss.server.partition.buffer.size 128 24 | rss.jetty.http.port 12345 25 | rss.jetty.corePool.size 64 26 | rss.storage.basePath /fake/path 27 | -------------------------------------------------------------------------------- /server/src/test/resources/healthy-script1.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | echo "NORMAL: Check Disk is Normal" -------------------------------------------------------------------------------- /server/src/test/resources/healthy-script2.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | echo "ERROR(disk_io): disk with Input/output error" -------------------------------------------------------------------------------- /server/src/test/resources/healthy-script3.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | exit 1 -------------------------------------------------------------------------------- /server/src/test/resources/healthy-script4.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | sleep 5s -------------------------------------------------------------------------------- /server/src/test/resources/log4j2.xml: -------------------------------------------------------------------------------- 1 | 2 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | -------------------------------------------------------------------------------- /server/src/test/resources/server.conf: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | rss.rpc.server.port 1234 19 | rss.storage.type HDFS 20 | rss.storage.basePath /var/tmp/test 21 | rss.coordinator.quorum localhost:9527 22 | rss.server.buffer.capacity 150 23 | rss.server.buffer.spill.threshold 130 24 | rss.server.partition.buffer.size 128 25 | rss.jetty.http.port 12345 26 | rss.jetty.corePool.size 64 27 | rss.server.write.timeout 2000 28 | rss.server.shuffleBufferManager.trigger.flush.interval 500 29 | -------------------------------------------------------------------------------- /spotbugs-exclude.xml: -------------------------------------------------------------------------------- 1 | 2 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | -------------------------------------------------------------------------------- /storage/src/main/java/org/apache/uniffle/storage/common/ShuffleInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.storage.common; 19 | 20 | public class ShuffleInfo { 21 | private String key; 22 | private long size; 23 | 24 | public ShuffleInfo(String key, long size) { 25 | this.key = key; 26 | this.size = size; 27 | } 28 | 29 | public String getKey() { 30 | return key; 31 | } 32 | 33 | public long getSize() { 34 | return size; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /storage/src/main/java/org/apache/uniffle/storage/common/ShuffleSegment.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.storage.common; 19 | 20 | public abstract class ShuffleSegment implements java.io.Serializable {} 21 | -------------------------------------------------------------------------------- /storage/src/main/java/org/apache/uniffle/storage/handler/api/ServerReadHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.storage.handler.api; 19 | 20 | import org.apache.uniffle.common.ShuffleDataResult; 21 | import org.apache.uniffle.common.ShuffleIndexResult; 22 | 23 | public interface ServerReadHandler { 24 | 25 | ShuffleDataResult getShuffleData(long offset, int length); 26 | 27 | ShuffleIndexResult getShuffleIndex(); 28 | 29 | int getStorageId(); 30 | } 31 | -------------------------------------------------------------------------------- /storage/src/main/java/org/apache/uniffle/storage/handler/api/ShuffleDeleteHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.storage.handler.api; 19 | 20 | public interface ShuffleDeleteHandler { 21 | 22 | /** 23 | * Delete shuffle data with appId 24 | * 25 | * @param appId ApplicationId for delete 26 | */ 27 | boolean delete(String[] storageBasePaths, String appId, String user); 28 | } 29 | -------------------------------------------------------------------------------- /storage/src/main/java/org/apache/uniffle/storage/handler/api/ShuffleWriteHandler.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.uniffle.storage.handler.api; 19 | 20 | import java.util.Collection; 21 | 22 | import org.apache.uniffle.common.ShufflePartitionedBlock; 23 | 24 | public interface ShuffleWriteHandler { 25 | 26 | /** 27 | * Write the blocks to storage 28 | * 29 | * @param shuffleBlocks blocks to storage 30 | * @throws Exception 31 | */ 32 | void write(Collection shuffleBlocks) throws Exception; 33 | } 34 | --------------------------------------------------------------------------------