├── .github ├── pull_request_template.md └── workflows │ └── github-actions.yml ├── .gitignore ├── HEADER ├── LICENSE ├── NOTICE ├── README.md ├── ambry-account └── src │ ├── integration-test │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── account │ │ └── MySqlAccountServiceIntegrationTest.java │ ├── main │ ├── java │ │ └── com │ │ │ └── github │ │ │ └── ambry │ │ │ └── account │ │ │ ├── AbstractAccountService.java │ │ │ ├── AccountInfoMap.java │ │ │ ├── AccountMetadataStore.java │ │ │ ├── AccountServiceMetrics.java │ │ │ ├── AccountServiceMetricsWrapper.java │ │ │ ├── AccountUtils.java │ │ │ ├── BackupFileManager.java │ │ │ ├── CachedAccountService.java │ │ │ ├── CompositeAccountService.java │ │ │ ├── CompositeAccountServiceFactory.java │ │ │ ├── HelixAccountService.java │ │ │ ├── HelixAccountServiceFactory.java │ │ │ ├── InMemoryAccountService.java │ │ │ ├── InMemoryAccountServiceFactory.java │ │ │ ├── LegacyMetadataStore.java │ │ │ ├── MySqlAccountService.java │ │ │ ├── MySqlAccountServiceFactory.java │ │ │ ├── RouterStore.java │ │ │ └── mysql │ │ │ ├── AccountDao.java │ │ │ ├── DatasetDao.java │ │ │ ├── MySqlAccountStore.java │ │ │ └── MySqlAccountStoreFactory.java │ └── resources │ │ └── AccountSchema.ddl │ └── test │ ├── accounts.json │ ├── java │ └── com │ │ └── github │ │ └── ambry │ │ └── account │ │ ├── BackupFileManagerTest.java │ │ ├── CompositeAccountServiceTest.java │ │ ├── HelixAccountServiceTest.java │ │ ├── InMemoryAccountServiceTest.java │ │ ├── MockHelixAccountServiceFactory.java │ │ ├── MockNotifier.java │ │ ├── MockRouter.java │ │ ├── MySqlAccountServiceTest.java │ │ ├── RouterStoreTest.java │ │ └── mysql │ │ └── AccountDaoTest.java │ └── resources │ └── mysql.properties ├── ambry-api └── src │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ ├── account │ │ ├── Account.java │ │ ├── AccountBlobsResource.java │ │ ├── AccountBuilder.java │ │ ├── AccountCollectionSerde.java │ │ ├── AccountService.java │ │ ├── AccountServiceCallback.java │ │ ├── AccountServiceErrorCode.java │ │ ├── AccountServiceException.java │ │ ├── AccountServiceFactory.java │ │ ├── AclService.java │ │ ├── Container.java │ │ ├── ContainerBlobsResource.java │ │ ├── ContainerBuilder.java │ │ ├── Dataset.java │ │ ├── DatasetBuilder.java │ │ ├── DatasetRetentionPolicy.java │ │ └── DatasetVersionRecord.java │ │ ├── accountstats │ │ ├── AccountStatsStore.java │ │ └── AccountStatsStoreFactory.java │ │ ├── clustermap │ │ ├── ClusterAgentsFactory.java │ │ ├── ClusterMap.java │ │ ├── ClusterMapChangeListener.java │ │ ├── ClusterMapSnapshotConstants.java │ │ ├── ClusterParticipant.java │ │ ├── DataNodeConfig.java │ │ ├── DataNodeConfigSourceType.java │ │ ├── DataNodeId.java │ │ ├── DiskId.java │ │ ├── DistributedLock.java │ │ ├── DistributedLockLocalImpl.java │ │ ├── HardwareState.java │ │ ├── PartitionId.java │ │ ├── PartitionState.java │ │ ├── PartitionStateChangeListener.java │ │ ├── ReplicaEventType.java │ │ ├── ReplicaId.java │ │ ├── ReplicaSealStatus.java │ │ ├── ReplicaState.java │ │ ├── ReplicaStatusDelegate.java │ │ ├── ReplicaSyncUpManager.java │ │ ├── ReplicaType.java │ │ ├── Resource.java │ │ ├── ResourceStatePolicy.java │ │ ├── ResourceStatePolicyFactory.java │ │ ├── StateModelListenerType.java │ │ └── StateTransitionException.java │ │ ├── commons │ │ ├── Callback.java │ │ ├── CallbackUtils.java │ │ ├── Criteria.java │ │ ├── Notifier.java │ │ ├── PerformanceIndex.java │ │ ├── SSLFactory.java │ │ ├── Thresholds.java │ │ └── TopicListener.java │ │ ├── compression │ │ ├── Compression.java │ │ ├── CompressionException.java │ │ └── CompressionLevel.java │ │ ├── config │ │ ├── AccountServiceConfig.java │ │ ├── AccountStatsMySqlConfig.java │ │ ├── CloudConfig.java │ │ ├── ClusterMapConfig.java │ │ ├── CompositeAccountServiceConfig.java │ │ ├── CompressionConfig.java │ │ ├── Config.java │ │ ├── ConnectionPoolConfig.java │ │ ├── CryptoServiceConfig.java │ │ ├── Default.java │ │ ├── DiskManagerConfig.java │ │ ├── FileCopyBasedReplicationConfig.java │ │ ├── FrontendConfig.java │ │ ├── HelixAccountServiceConfig.java │ │ ├── HelixPropertyStoreConfig.java │ │ ├── HostThrottleConfig.java │ │ ├── Http2ClientConfig.java │ │ ├── InMemoryAccountConfig.java │ │ ├── KMSConfig.java │ │ ├── MySqlAccountServiceConfig.java │ │ ├── MySqlNamedBlobDbConfig.java │ │ ├── MysqlRepairRequestsDbConfig.java │ │ ├── NettyConfig.java │ │ ├── NetworkConfig.java │ │ ├── PerformanceConfig.java │ │ ├── QuotaConfig.java │ │ ├── ReplicaPrioritizationConfig.java │ │ ├── ReplicaPrioritizationStrategy.java │ │ ├── ReplicaSelectionPolicy.java │ │ ├── ReplicationConfig.java │ │ ├── RestServerConfig.java │ │ ├── RouterConfig.java │ │ ├── SSLConfig.java │ │ ├── ServerConfig.java │ │ ├── ServerExecutionMode.java │ │ ├── ServerReplicationMode.java │ │ ├── StatsManagerConfig.java │ │ ├── StorageQuotaConfig.java │ │ ├── StoreConfig.java │ │ └── VerifiableProperties.java │ │ ├── frontend │ │ ├── AccountMetrics.java │ │ ├── AmbryCostModelPolicy.java │ │ ├── BlobMappingExtension.java │ │ ├── ContainerMetrics.java │ │ ├── DatasetVersionPath.java │ │ ├── EntityOperationMetrics.java │ │ ├── IdConverter.java │ │ ├── IdConverterFactory.java │ │ ├── IdSigningService.java │ │ ├── IdSigningServiceFactory.java │ │ ├── NamedBlobListEntry.java │ │ ├── NamedBlobPath.java │ │ ├── Operations.java │ │ ├── Page.java │ │ ├── PutBlobMetaInfo.java │ │ ├── ReservedMetadataIdMetrics.java │ │ ├── SecurityService.java │ │ ├── SecurityServiceFactory.java │ │ ├── StitchRequestSerDe.java │ │ ├── UrlSigningService.java │ │ ├── UrlSigningServiceFactory.java │ │ └── s3 │ │ │ ├── S3MessagePayload.java │ │ │ └── S3MultipartETag.java │ │ ├── messageformat │ │ ├── BlobInfo.java │ │ ├── BlobOutput.java │ │ ├── BlobProperties.java │ │ ├── BlobType.java │ │ └── MessageFormatFlags.java │ │ ├── named │ │ ├── DeleteResult.java │ │ ├── NamedBlobDb.java │ │ ├── NamedBlobDbFactory.java │ │ ├── NamedBlobRecord.java │ │ ├── PutResult.java │ │ ├── StaleNamedBlob.java │ │ └── TransactionIsolationLevel.java │ │ ├── network │ │ ├── BoundedByteBufferSend.java │ │ ├── BoundedNettyByteBufReceive.java │ │ ├── ChannelOutput.java │ │ ├── ConnectedChannel.java │ │ ├── ConnectionPool.java │ │ ├── ConnectionPoolTimeoutException.java │ │ ├── NetworkClient.java │ │ ├── NetworkClientErrorCode.java │ │ ├── NetworkClientFactory.java │ │ ├── NetworkReceive.java │ │ ├── NetworkRequest.java │ │ ├── NetworkRequestQueue.java │ │ ├── NetworkResponse.java │ │ ├── NetworkSend.java │ │ ├── NetworkServer.java │ │ ├── Port.java │ │ ├── PortType.java │ │ ├── Receive.java │ │ ├── RequestInfo.java │ │ ├── RequestQueueType.java │ │ ├── RequestResponseChannel.java │ │ ├── ResponseInfo.java │ │ ├── Selectable.java │ │ ├── Send.java │ │ ├── SendWithCorrelationId.java │ │ └── ServerNetworkResponseMetrics.java │ │ ├── notification │ │ ├── BlobReplicaSourceType.java │ │ ├── NotificationBlobType.java │ │ ├── NotificationSystem.java │ │ └── UpdateType.java │ │ ├── protocol │ │ ├── DatasetVersionState.java │ │ ├── GetOption.java │ │ ├── NamedBlobState.java │ │ └── RequestAPI.java │ │ ├── quota │ │ ├── Chargeable.java │ │ ├── Quota.java │ │ ├── QuotaAction.java │ │ ├── QuotaChargeCallback.java │ │ ├── QuotaEnforcer.java │ │ ├── QuotaEnforcerFactory.java │ │ ├── QuotaException.java │ │ ├── QuotaManager.java │ │ ├── QuotaManagerFactory.java │ │ ├── QuotaMethod.java │ │ ├── QuotaMetrics.java │ │ ├── QuotaMode.java │ │ ├── QuotaName.java │ │ ├── QuotaRecommendation.java │ │ ├── QuotaRecommendationMergePolicy.java │ │ ├── QuotaResource.java │ │ ├── QuotaResourceType.java │ │ ├── QuotaSource.java │ │ ├── QuotaSourceFactory.java │ │ ├── QuotaUsageLevel.java │ │ ├── RequestQuotaCostPolicy.java │ │ ├── ThrottlingRecommendation.java │ │ └── storage │ │ │ └── StorageUsageRefresher.java │ │ ├── repair │ │ ├── RepairRequestRecord.java │ │ ├── RepairRequestsDb.java │ │ └── RepairRequestsDbFactory.java │ │ ├── replication │ │ ├── FindToken.java │ │ ├── FindTokenFactory.java │ │ ├── FindTokenHelper.java │ │ ├── FindTokenType.java │ │ ├── ReplicationAPI.java │ │ ├── ReplicationException.java │ │ ├── ReplicationModelType.java │ │ └── ReplicationSkipPredicate.java │ │ ├── rest │ │ ├── NioServer.java │ │ ├── NioServerFactory.java │ │ ├── RequestPath.java │ │ ├── ResponseStatus.java │ │ ├── RestMethod.java │ │ ├── RestRequest.java │ │ ├── RestRequestHandler.java │ │ ├── RestRequestMetrics.java │ │ ├── RestRequestMetricsTracker.java │ │ ├── RestRequestResponseHandlerFactory.java │ │ ├── RestRequestService.java │ │ ├── RestRequestServiceFactory.java │ │ ├── RestResponseChannel.java │ │ ├── RestResponseHandler.java │ │ ├── RestServiceErrorCode.java │ │ ├── RestServiceException.java │ │ └── RestUtils.java │ │ ├── router │ │ ├── AsyncWritableChannel.java │ │ ├── ByteRange.java │ │ ├── ByteRanges.java │ │ ├── ChunkInfo.java │ │ ├── CryptoService.java │ │ ├── CryptoServiceFactory.java │ │ ├── FutureResult.java │ │ ├── GetBlobOptions.java │ │ ├── GetBlobOptionsBuilder.java │ │ ├── GetBlobResult.java │ │ ├── KeyManagementService.java │ │ ├── KeyManagementServiceFactory.java │ │ ├── OperationTrackerScope.java │ │ ├── PutBlobOptions.java │ │ ├── PutBlobOptionsBuilder.java │ │ ├── ReadableStreamChannel.java │ │ ├── Router.java │ │ ├── RouterErrorCode.java │ │ ├── RouterException.java │ │ └── RouterFactory.java │ │ ├── server │ │ ├── AmbryStatsReport.java │ │ ├── EmptyRequest.java │ │ ├── HostAccountStorageStatsWrapper.java │ │ ├── HostPartitionClassStorageStatsWrapper.java │ │ ├── ServerErrorCode.java │ │ ├── StatsHeader.java │ │ ├── StatsReportType.java │ │ ├── StoreManager.java │ │ └── storagestats │ │ │ ├── AggregatedAccountStorageStats.java │ │ │ ├── AggregatedPartitionClassStorageStats.java │ │ │ ├── ContainerStorageStats.java │ │ │ ├── HostAccountStorageStats.java │ │ │ └── HostPartitionClassStorageStats.java │ │ └── store │ │ ├── AbstractStoreKeyConverter.java │ │ ├── BlobMatchStatus.java │ │ ├── DeleteTombstoneStats.java │ │ ├── FileInfo.java │ │ ├── FileStoreException.java │ │ ├── FindInfo.java │ │ ├── HardDeleteInfo.java │ │ ├── IdUndeletedStoreException.java │ │ ├── IndexMemState.java │ │ ├── LogInfo.java │ │ ├── Message.java │ │ ├── MessageErrorInfo.java │ │ ├── MessageInfo.java │ │ ├── MessageReadSet.java │ │ ├── MessageStoreHardDelete.java │ │ ├── MessageStoreRecovery.java │ │ ├── MessageWriteSet.java │ │ ├── PartitionFileStore.java │ │ ├── Read.java │ │ ├── ReplicationProtocolTransitionType.java │ │ ├── Store.java │ │ ├── StoreBatchDeleteInfo.java │ │ ├── StoreErrorCodes.java │ │ ├── StoreException.java │ │ ├── StoreFactory.java │ │ ├── StoreFileChunk.java │ │ ├── StoreFileInfo.java │ │ ├── StoreGetOptions.java │ │ ├── StoreInfo.java │ │ ├── StoreKey.java │ │ ├── StoreKeyConverter.java │ │ ├── StoreKeyConverterFactory.java │ │ ├── StoreKeyFactory.java │ │ ├── StoreKeyJacksonConfig.java │ │ ├── StoreLogInfo.java │ │ ├── StoreStats.java │ │ ├── TimeRange.java │ │ ├── TransformationOutput.java │ │ ├── Transformer.java │ │ └── Write.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ ├── account │ └── AccountContainerTest.java │ ├── clustermap │ ├── ReplicaSealStatusTest.java │ └── ReplicaStatusDelegateTest.java │ ├── commons │ ├── CallbackUtilsTest.java │ └── ThresholdsTest.java │ ├── compression │ └── CompressionExceptionTest.java │ ├── config │ ├── CompressionConfigTest.java │ └── VerifiablePropertiesTest.java │ ├── frontend │ └── StitchRequestSerDeTest.java │ ├── quota │ ├── QuotaActionSeverityTest.java │ └── ThrottlingRecommendationTest.java │ ├── rest │ ├── MockNioServer.java │ ├── MockNioServerFactory.java │ ├── MockRestRequestResponseHandler.java │ ├── MockRestRequestResponseHandlerFactory.java │ ├── MockRestRequestService.java │ ├── MockRestRequestServiceFactory.java │ ├── RequestPathTest.java │ ├── RestRequestMetricsTest.java │ ├── RestRequestMetricsTrackerTest.java │ └── RestUtilsTest.java │ ├── router │ ├── ByteRangeTest.java │ ├── GetBlobOptionsTest.java │ └── PutBlobOptionsTest.java │ ├── server │ ├── StorageStatsTest.java │ └── StorageStatsUtilTest.java │ └── store │ ├── DeleteTombstoneStatsTest.java │ ├── MessageInfoTest.java │ └── MockWrite.java ├── ambry-cloud ├── README.md └── src │ ├── main │ ├── java │ │ └── com │ │ │ └── github │ │ │ └── ambry │ │ │ └── cloud │ │ │ ├── AmbryVcrSecurityServiceFactory.java │ │ │ ├── BackupIntegrityMonitor.java │ │ │ ├── CloudBlobCryptoAgent.java │ │ │ ├── CloudBlobCryptoAgentFactory.java │ │ │ ├── CloudBlobCryptoAgentFactoryImpl.java │ │ │ ├── CloudBlobCryptoAgentImpl.java │ │ │ ├── CloudBlobMetadata.java │ │ │ ├── CloudBlobStore.java │ │ │ ├── CloudContainerCompactor.java │ │ │ ├── CloudDestination.java │ │ │ ├── CloudDestinationFactory.java │ │ │ ├── CloudMessageFormatWriteSet.java │ │ │ ├── CloudMessageReadSet.java │ │ │ ├── CloudRequestAgent.java │ │ │ ├── CloudStorageCompactor.java │ │ │ ├── CloudStorageException.java │ │ │ ├── CloudStorageManager.java │ │ │ ├── CloudTokenPersistor.java │ │ │ ├── CloudUpdateValidator.java │ │ │ ├── DeprecatedContainerCloudSyncTask.java │ │ │ ├── FindResult.java │ │ │ ├── HelixVcrClusterAgentsFactory.java │ │ │ ├── HelixVcrClusterMetrics.java │ │ │ ├── HelixVcrClusterParticipant.java │ │ │ ├── HelixVcrClusterSpectator.java │ │ │ ├── LeaderStandbyHelixVcrStateModel.java │ │ │ ├── LeaderStandbyHelixVcrStateModelFactory.java │ │ │ ├── OnlineOfflineHelixVcrStateModel.java │ │ │ ├── OnlineOfflineHelixVcrStateModelFactory.java │ │ │ ├── RecoveryManager.java │ │ │ ├── RecoveryMetrics.java │ │ │ ├── RecoveryNetworkClient.java │ │ │ ├── RecoveryNetworkClientCallback.java │ │ │ ├── RecoveryNetworkClientFactory.java │ │ │ ├── RecoveryThread.java │ │ │ ├── RecoveryToken.java │ │ │ ├── RecoveryTokenFactory.java │ │ │ ├── StaticVcrClusterAgentsFactory.java │ │ │ ├── StaticVcrClusterParticipant.java │ │ │ ├── VcrMetrics.java │ │ │ ├── VcrReplicaThread.java │ │ │ ├── VcrReplicationManager.java │ │ │ ├── VcrStateModelFactory.java │ │ │ └── azure │ │ │ ├── ADAuthBasedStorageClient.java │ │ │ ├── AzureBlobDataAccessor.java │ │ │ ├── AzureBlobDeletePolicy.java │ │ │ ├── AzureBlobLayoutStrategy.java │ │ │ ├── AzureCloudConfig.java │ │ │ ├── AzureCloudDestination.java │ │ │ ├── AzureCloudDestinationFactory.java │ │ │ ├── AzureCloudDestinationSync.java │ │ │ ├── AzureCompactionUtil.java │ │ │ ├── AzureContainerCompactor.java │ │ │ ├── AzureMetrics.java │ │ │ ├── AzureMetricsOld.java │ │ │ ├── AzureReplicationFeed.java │ │ │ ├── AzureStorageClient.java │ │ │ ├── AzureStorageCompactor.java │ │ │ ├── AzureStorageContainerMetrics.java │ │ │ ├── AzureStorageContainerMetricsCollector.java │ │ │ ├── AzureUtils.java │ │ │ ├── AzuriteUtils.java │ │ │ ├── ClientSecretCredentialStorageClient.java │ │ │ ├── ConnectionStringBasedStorageClient.java │ │ │ ├── CosmosChangeFeedBasedReplicationFeed.java │ │ │ ├── CosmosChangeFeedFindToken.java │ │ │ ├── CosmosChangeFeedFindTokenFactory.java │ │ │ ├── CosmosContainerDeletionEntry.java │ │ │ ├── CosmosDataAccessor.java │ │ │ ├── CosmosUpdateTimeBasedReplicationFeed.java │ │ │ ├── CosmosUpdateTimeFindToken.java │ │ │ ├── CosmosUpdateTimeFindTokenFactory.java │ │ │ ├── ShardedStorageClient.java │ │ │ ├── StorageClient.java │ │ │ └── VcrInstanceConfig.java │ └── resources │ │ └── bulkDelete.js │ └── test │ ├── java │ └── com │ │ └── github │ │ └── ambry │ │ └── cloud │ │ ├── CloudBlobCryptoAgentImplTest.java │ │ ├── CloudBlobMetadataTest.java │ │ ├── CloudBlobStoreIntegrationTest.java │ │ ├── CloudDataNodeTest.java │ │ ├── CloudStorageCompactorTest.java │ │ ├── CloudTestUtil.java │ │ ├── DummyCloudUpdateValidator.java │ │ ├── RecoveryManagerTest.java │ │ ├── RecoveryNetworkClientTest.java │ │ ├── StaticVcrClusterParticipantTest.java │ │ ├── TestCloudBlobCryptoAgent.java │ │ ├── TestCloudBlobCryptoAgentFactory.java │ │ ├── VcrReplicaThreadTest.java │ │ └── azure │ │ ├── AzureBlobDataAccessorTest.java │ │ ├── AzureBlobLayoutStrategyTest.java │ │ ├── AzureCloudConfigTest.java │ │ ├── AzureCloudDestinationTest.java │ │ ├── AzureContainerCompactorIntegrationTest.java │ │ ├── AzureIntegrationTest.java │ │ ├── AzurePerformanceTest.java │ │ ├── AzureStorageCompactorTest.java │ │ ├── AzureStorageContainerMetricsTest.java │ │ ├── AzureTestUtils.java │ │ ├── CosmosChangeFeedFindTokenFactoryTest.java │ │ ├── CosmosChangeFeedFindTokenTest.java │ │ ├── CosmosDataAccessorTest.java │ │ ├── CosmosUpdateTimeFindTokenFactoryTest.java │ │ ├── CosmosUpdateTimeFindTokenTest.java │ │ └── MockChangeFeedQuery.java │ └── resources │ ├── azure-test.properties │ ├── mockito-extensions │ └── org.mockito.plugins.MockMaker │ └── replicaTokens ├── ambry-clustermap └── src │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── clustermap │ │ ├── AccountStorageStatsIterator.java │ │ ├── AmbryDataNode.java │ │ ├── AmbryDisk.java │ │ ├── AmbryPartition.java │ │ ├── AmbryPartitionStateModel.java │ │ ├── AmbryReplica.java │ │ ├── AmbryReplicaSyncUpManager.java │ │ ├── AmbryServerDataNode.java │ │ ├── AmbryServerReplica.java │ │ ├── AmbryStateModelDefinition.java │ │ ├── AmbryStateModelFactory.java │ │ ├── CloudDataNode.java │ │ ├── CloudReplica.java │ │ ├── CloudServiceDataNode.java │ │ ├── CloudServiceReplica.java │ │ ├── ClusterManagerQueryHelper.java │ │ ├── ClusterMapMetrics.java │ │ ├── ClusterMapUtils.java │ │ ├── CompositeClusterAgentsFactory.java │ │ ├── CompositeClusterManager.java │ │ ├── CompositeDataNodeConfigSource.java │ │ ├── DataNode.java │ │ ├── DataNodeConfigChangeListener.java │ │ ├── DataNodeConfigSource.java │ │ ├── DataNodeConfigSourceMetrics.java │ │ ├── Datacenter.java │ │ ├── DcInfo.java │ │ ├── DefaultLeaderStandbyStateModel.java │ │ ├── Disk.java │ │ ├── FixedBackoffResourceStatePolicy.java │ │ ├── FixedBackoffResourceStatePolicyFactory.java │ │ ├── HardwareLayout.java │ │ ├── HelixAdminFactory.java │ │ ├── HelixAggregatedViewClusterInfo.java │ │ ├── HelixAggregatedViewClusterInitializer.java │ │ ├── HelixBootstrapUpgradeUtil.java │ │ ├── HelixClusterAgentsFactory.java │ │ ├── HelixClusterManager.java │ │ ├── HelixClusterManagerMetrics.java │ │ ├── HelixDatacenterInitializer.java │ │ ├── HelixDcInfo.java │ │ ├── HelixFactory.java │ │ ├── HelixParticipant.java │ │ ├── HelixParticipantMetrics.java │ │ ├── InstanceConfigToDataNodeConfigAdapter.java │ │ ├── MySqlClusterAggregator.java │ │ ├── MySqlReportAggregatorTask.java │ │ ├── NoOpResourceStatePolicy.java │ │ ├── Partition.java │ │ ├── PartitionClassStorageStatsIterator.java │ │ ├── PartitionLayout.java │ │ ├── PropertyStoreCleanUpTask.java │ │ ├── PropertyStoreToDataNodeConfigAdapter.java │ │ ├── RecoveryTestClusterAgentsFactory.java │ │ ├── RecoveryTestClusterManager.java │ │ ├── Replica.java │ │ ├── ResourceInfo.java │ │ ├── StaticClusterAgentsFactory.java │ │ ├── StaticClusterManager.java │ │ └── TaskUtils.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ └── clustermap │ ├── AmbryPartitionTest.java │ ├── AmbryReplicaSyncUpManagerTest.java │ ├── AmbryStateModelFactoryTest.java │ ├── CloudReplicaTest.java │ ├── ClusterChangeHandlerTest.java │ ├── ClusterMapUtilsTest.java │ ├── CompositeDataNodeConfigSourceTest.java │ ├── DataNodeConfigSourceTestBase.java │ ├── DataNodeTest.java │ ├── DatacenterTest.java │ ├── DiskTest.java │ ├── DynamicClusterManagerComponentsTest.java │ ├── FixedBackoffResourceStatePolicyTest.java │ ├── HardwareLayoutTest.java │ ├── HelixClusterManagerTest.java │ ├── HelixParticipantTest.java │ ├── HelixStoreOperator.java │ ├── InstanceConfigToDataNodeConfigAdapterTest.java │ ├── MockHelixAdmin.java │ ├── MockHelixAdminFactory.java │ ├── MockHelixCluster.java │ ├── MockHelixDataAccessor.java │ ├── MockHelixManager.java │ ├── MockHelixManagerFactory.java │ ├── MockHelixParticipant.java │ ├── MockHelixPropertyStore.java │ ├── MySqlClusterAggregatorTest.java │ ├── PartitionLayoutTest.java │ ├── PartitionTest.java │ ├── PropertyStoreCleanUpTaskTest.java │ ├── PropertyStoreToDataNodeConfigAdapterTest.java │ ├── ReplicaTest.java │ ├── StaticClusterManagerTest.java │ ├── TestReplica.java │ └── TestUtils.java ├── ambry-commons └── src │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ ├── clustermap │ │ ├── HelixVcrUtil.java │ │ ├── VcrClusterAgentsFactory.java │ │ ├── VcrClusterParticipant.java │ │ ├── VcrClusterParticipantListener.java │ │ └── VcrClusterSpectator.java │ │ ├── commons │ │ ├── AmbryCache.java │ │ ├── AmbryCacheEntry.java │ │ ├── BlobId.java │ │ ├── BlobIdFactory.java │ │ ├── ByteBufferAsyncWritableChannel.java │ │ ├── ByteBufferReadableStreamChannel.java │ │ ├── CommonUtils.java │ │ ├── ErrorMapping.java │ │ ├── FutureUtils.java │ │ ├── HardwareResource.java │ │ ├── HardwareUsageMeter.java │ │ ├── HelixNotifier.java │ │ ├── HostLevelThrottler.java │ │ ├── InMemNamedBlobDb.java │ │ ├── InMemNamedBlobDbFactory.java │ │ ├── InputStreamReadableStreamChannel.java │ │ ├── JdkSslFactory.java │ │ ├── LoggingNotificationSystem.java │ │ ├── NettyInternalMetrics.java │ │ ├── NettySslFactory.java │ │ ├── NettySslHttp2Factory.java │ │ ├── NettyUtils.java │ │ ├── ReadableStreamChannelInputStream.java │ │ ├── ResponseHandler.java │ │ ├── RetainingAsyncWritableChannel.java │ │ ├── RetryExecutor.java │ │ ├── RetryPolicies.java │ │ ├── RetryPolicy.java │ │ └── ServerMetrics.java │ │ ├── compression │ │ ├── BaseCompression.java │ │ ├── BaseCompressionWithLevel.java │ │ ├── CompressionMap.java │ │ ├── LZ4Compression.java │ │ └── ZstdCompression.java │ │ └── store │ │ ├── StoreKeyConverterFactoryImpl.java │ │ └── StoreKeyConverterImplNoOp.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ ├── commons │ ├── BlobIdTest.java │ ├── ByteBufferAsyncWritableChannelTest.java │ ├── ByteBufferReadableStreamChannelTest.java │ ├── CommonUtilsTest.java │ ├── HelixNotifierTest.java │ ├── HostLevelThrottlerTest.java │ ├── InputStreamReadableStreamChannelTest.java │ ├── JdkSslFactoryTest.java │ ├── NettySslFactoryTest.java │ ├── NettySslHttp2FactoryTest.java │ ├── ReadableStreamChannelInputStreamTest.java │ ├── ResponseHandlerTest.java │ └── RetainingAsyncWritableChannelTest.java │ ├── compression │ ├── BaseCompressionTest.java │ ├── BaseCompressionWithLevelTest.java │ ├── CompressionMapTest.java │ ├── LZ4CompressionTest.java │ ├── ZstdCompressionFailedTests.java │ └── ZstdCompressionTest.java │ └── store │ ├── AbstractStoreKeyConverterTest.java │ ├── StoreKeyConverterFactoryImplTest.java │ └── StoreKeyConverterImplNoOpTest.java ├── ambry-file-transfer └── src │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── filetransfer │ │ ├── DiskAwareFileCopyThreadPoolManager.java │ │ ├── FileChunkInfo.java │ │ ├── FileCopyBasedReplicationManager.java │ │ ├── FileCopyBasedReplicationScheduler.java │ │ ├── FileCopyBasedReplicationSchedulerFactory.java │ │ ├── FileCopyBasedReplicationSchedulerFactoryImpl.java │ │ ├── FileCopyBasedReplicationSchedulerImpl.java │ │ ├── FileCopyBasedReplicationThreadPoolManager.java │ │ ├── FileCopyInfo.java │ │ ├── FileCopyStatusListener.java │ │ ├── FileCopyThread.java │ │ ├── handler │ │ ├── FileCopyHandler.java │ │ ├── FileCopyHandlerException.java │ │ ├── FileCopyHandlerFactory.java │ │ ├── StoreFileCopyHandler.java │ │ └── StoreFileCopyHandlerFactory.java │ │ ├── utils │ │ ├── FileCopyUtils.java │ │ └── OperationRetryHandler.java │ │ └── workflow │ │ ├── BaseWorkFlow.java │ │ ├── GetChunkDataWorkflow.java │ │ └── GetMetadataWorkflow.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ └── filetransfer │ ├── DiskAwareFileCopyThreadPoolManagerTest.java │ ├── FileCopyBasedReplicationSchedulerImplTest.java │ ├── FileCopyThreadTest.java │ ├── FileCopyUtilsTest.java │ ├── handler │ ├── StoreFileCopyHandlerIntegTest.java │ └── StoreFileCopyHandlerTest.java │ └── utils │ └── OperationRetryHandlerTest.java ├── ambry-filesystem └── src │ └── main │ ├── java │ └── com │ │ └── github │ │ └── ambry │ │ └── filesystem │ │ └── package-info.java │ └── resources │ └── AmbryFS.ddl ├── ambry-frontend └── src │ ├── integration-test │ ├── java │ │ └── com │ │ │ └── github │ │ │ └── ambry │ │ │ └── frontend │ │ │ ├── FrontendIntegrationTest.java │ │ │ ├── FrontendIntegrationTestBase.java │ │ │ ├── FrontendQuotaIntegrationTest.java │ │ │ └── S3IntegrationTest.java │ └── resources │ │ └── namedblob_mysql.properties │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── frontend │ │ ├── AccountAndContainerInjector.java │ │ ├── AmbryFrontendMain.java │ │ ├── AmbryIdConverterFactory.java │ │ ├── AmbryIdSigningService.java │ │ ├── AmbryIdSigningServiceFactory.java │ │ ├── AmbrySecurityService.java │ │ ├── AmbrySecurityServiceFactory.java │ │ ├── AmbryUrlSigningService.java │ │ ├── AmbryUrlSigningServiceFactory.java │ │ ├── CopyDatasetVersionHandler.java │ │ ├── DeleteBlobHandler.java │ │ ├── DeleteDatasetHandler.java │ │ ├── FrontendMetrics.java │ │ ├── FrontendRestRequestService.java │ │ ├── FrontendRestRequestServiceFactory.java │ │ ├── FrontendUtils.java │ │ ├── GetAccountsHandler.java │ │ ├── GetBlobHandler.java │ │ ├── GetClusterMapSnapshotHandler.java │ │ ├── GetDatasetsHandler.java │ │ ├── GetPeersHandler.java │ │ ├── GetReplicasHandler.java │ │ ├── GetResourceInfoHandler.java │ │ ├── GetSignedUrlHandler.java │ │ ├── GetStatsReportHandler.java │ │ ├── HeadBlobHandler.java │ │ ├── ListDatasetVersionHandler.java │ │ ├── ListDatasetsHandler.java │ │ ├── NamedBlobListHandler.java │ │ ├── NamedBlobPutHandler.java │ │ ├── NamedBlobsCleanupRunner.java │ │ ├── PostAccountsHandler.java │ │ ├── PostBlobHandler.java │ │ ├── PostDatasetsHandler.java │ │ ├── RestRequestMetricsGroup.java │ │ ├── SignedIdSerDe.java │ │ ├── SimpleAmbryCostModelPolicy.java │ │ ├── TtlUpdateHandler.java │ │ ├── UndeleteHandler.java │ │ └── s3 │ │ ├── S3BaseHandler.java │ │ ├── S3BatchDeleteHandler.java │ │ ├── S3Constants.java │ │ ├── S3DeleteHandler.java │ │ ├── S3GetHandler.java │ │ ├── S3HeadHandler.java │ │ ├── S3ListHandler.java │ │ ├── S3MultipartAbortUploadHandler.java │ │ ├── S3MultipartCompleteUploadHandler.java │ │ ├── S3MultipartCreateUploadHandler.java │ │ ├── S3MultipartListPartsHandler.java │ │ ├── S3MultipartUploadHandler.java │ │ ├── S3MultipartUploadPartHandler.java │ │ ├── S3PostHandler.java │ │ └── S3PutHandler.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ └── frontend │ ├── AmbryIdConverterFactoryTest.java │ ├── AmbryIdSigningServiceTest.java │ ├── AmbrySecurityServiceFactoryTest.java │ ├── AmbrySecurityServiceTest.java │ ├── AmbryUrlSigningServiceTest.java │ ├── FrontendRestRequestServiceFactoryTest.java │ ├── FrontendRestRequestServiceTest.java │ ├── FrontendUtilsTest.java │ ├── GetAccountsHandlerTest.java │ ├── GetClusterMapSnapshotHandlerTest.java │ ├── GetPeersHandlerTest.java │ ├── GetReplicasHandlerTest.java │ ├── GetSignedUrlHandlerTest.java │ ├── GetStatsReportHandlerTest.java │ ├── NamedBlobDeleteHandlerTest.java │ ├── NamedBlobPutHandlerTest.java │ ├── PostAccountContainersHandlerTest.java │ ├── PostAccountsHandlerTest.java │ ├── PostBlobHandlerTest.java │ ├── S3BaseHandlerTest.java │ ├── S3BatchDeleteHandlerTest.java │ ├── S3DeleteHandlerTest.java │ ├── S3HeadHandlerTest.java │ ├── S3ListHandlerTest.java │ ├── S3MultipartUploadTest.java │ ├── S3PutHandlerTest.java │ ├── SignedIdSerDeTest.java │ ├── SimpleAmbryCostModelPolicyTest.java │ ├── TestAmbryFrontendMain.java │ ├── TtlUpdateHandlerTest.java │ └── UndeleteHandlerTest.java ├── ambry-messageformat └── src │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── messageformat │ │ ├── BlobAll.java │ │ ├── BlobData.java │ │ ├── BlobPropertiesSerDe.java │ │ ├── BlobStoreHardDelete.java │ │ ├── BlobStoreRecovery.java │ │ ├── CompositeBlobInfo.java │ │ ├── DeleteMessageFormatInputStream.java │ │ ├── DeleteSubRecord.java │ │ ├── HardDeleteMessageFormatInputStream.java │ │ ├── MessageFormatErrorCodes.java │ │ ├── MessageFormatException.java │ │ ├── MessageFormatInputStream.java │ │ ├── MessageFormatMetrics.java │ │ ├── MessageFormatRecord.java │ │ ├── MessageFormatSend.java │ │ ├── MessageFormatWriteSet.java │ │ ├── MessageMetadata.java │ │ ├── MessageSievingInputStream.java │ │ ├── MetadataContentSerDe.java │ │ ├── PutMessageFormatInputStream.java │ │ ├── SubRecord.java │ │ ├── TtlUpdateMessageFormatInputStream.java │ │ ├── TtlUpdateSubRecord.java │ │ ├── UndeleteMessageFormatInputStream.java │ │ ├── UndeleteSubRecord.java │ │ ├── UpdateRecord.java │ │ └── ValidatingTransformer.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ └── messageformat │ ├── BlobDataTest.java │ ├── BlobPropertiesTest.java │ ├── BlobStoreHardDeleteTest.java │ ├── BlobStoreRecoveryTest.java │ ├── CompositeBlobInfoTest.java │ ├── DeleteMessageFormatV1InputStream.java │ ├── DeleteMessageFormatV2InputStream.java │ ├── HardDeleteMessageFormatInputStreamTest.java │ ├── MessageFormatInputStreamTest.java │ ├── MessageFormatRecordTest.java │ ├── MessageFormatSendTest.java │ ├── MessageFormatTestUtils.java │ ├── MessageFormatWriteSetTest.java │ ├── MessageMetadataTest.java │ ├── MessageSievingInputStreamTest.java │ └── PutMessageFormatBlobV1InputStream.java ├── ambry-mysql └── src │ ├── integration-test │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ ├── accountstats │ │ ├── AccountStatsMySqlStoreIntegrationTest.java │ │ └── PartitionClassReportsDaoTest.java │ │ └── repair │ │ └── MysqlRepairRequestsDbTest.java │ ├── main │ ├── java │ │ └── com │ │ │ └── github │ │ │ └── ambry │ │ │ ├── accountstats │ │ │ ├── AccountReportsDao.java │ │ │ ├── AccountStatsMySqlStore.java │ │ │ ├── AccountStatsMySqlStoreFactory.java │ │ │ ├── AggregatedAccountReportsDao.java │ │ │ ├── AggregatedContainerStorageStatsFunction.java │ │ │ ├── ContainerStorageStatsFunction.java │ │ │ ├── HostnameHelper.java │ │ │ ├── InmemoryAccountStatsStore.java │ │ │ ├── InmemoryAccountStatsStoreFactory.java │ │ │ ├── PartitionClassContainerStorageStatsFunction.java │ │ │ └── PartitionClassReportsDao.java │ │ │ ├── mysql │ │ │ ├── BatchUpdater.java │ │ │ ├── MySqlDataAccessor.java │ │ │ ├── MySqlMetrics.java │ │ │ └── MySqlUtils.java │ │ │ └── repair │ │ │ ├── MysqlRepairRequestsDb.java │ │ │ └── MysqlRepairRequestsDbFactory.java │ └── resources │ │ ├── AmbryContainerStorageStats.ddl │ │ └── AmbryRepairRequests.ddl │ └── test │ ├── java │ └── com │ │ └── github │ │ └── ambry │ │ ├── accountstats │ │ ├── AccountReportsDaoTest.java │ │ ├── AccountStatsMySqlStoreTest.java │ │ ├── AggregatedAccountReportsDaoTest.java │ │ └── HostnameHelperTest.java │ │ └── mysql │ │ ├── BatchUpdaterTest.java │ │ └── MySqlDataAccessorTest.java │ └── resources │ ├── accountstats_mysql.properties │ └── repairRequests_mysql.properties ├── ambry-named-mysql └── src │ ├── integration-test │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── named │ │ ├── MySqlNamedBlobDbIntegrationTest.java │ │ ├── MySqlNamedBlobDbIntergrationBase.java │ │ └── MySqlNamedBlobDbListOperationIntegrationTest.java │ ├── main │ ├── java │ │ └── com │ │ │ └── github │ │ │ └── ambry │ │ │ └── named │ │ │ ├── GetTransactionStateTracker.java │ │ │ ├── MySqlNamedBlobDb.java │ │ │ ├── MySqlNamedBlobDbFactory.java │ │ │ └── TransactionStateTracker.java │ └── resources │ │ └── NamedBlobsSchema.ddl │ └── test │ ├── java │ └── com │ │ └── github │ │ └── ambry │ │ └── named │ │ └── MySqlNamedBlobDbTest.java │ └── resources │ └── mysql.properties ├── ambry-network └── src │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── network │ │ ├── AdaptiveLifoCoDelNetworkRequestQueue.java │ │ ├── BlockingChannel.java │ │ ├── BlockingChannelConnectionPool.java │ │ ├── BlockingChannelInfo.java │ │ ├── CompositeNetworkClient.java │ │ ├── CompositeNetworkClientFactory.java │ │ ├── ConnectionTracker.java │ │ ├── FifoNetworkRequestQueue.java │ │ ├── LocalNetworkClient.java │ │ ├── LocalNetworkClientFactory.java │ │ ├── LocalRequestResponseChannel.java │ │ ├── NettyServerRequest.java │ │ ├── NettyServerRequestResponseChannel.java │ │ ├── NetworkMetrics.java │ │ ├── PlainTextTransmission.java │ │ ├── SSLBlockingChannel.java │ │ ├── SSLTransmission.java │ │ ├── Selector.java │ │ ├── ServerRequestResponseHelper.java │ │ ├── SocketNetworkClient.java │ │ ├── SocketNetworkClientFactory.java │ │ ├── SocketRequestResponseChannel.java │ │ ├── SocketServer.java │ │ ├── Transmission.java │ │ └── http2 │ │ ├── AmbryNetworkRequestHandler.java │ │ ├── AmbrySendToHttp2Adaptor.java │ │ ├── GoAwayException.java │ │ ├── Http2BlockingChannel.java │ │ ├── Http2BlockingChannelPool.java │ │ ├── Http2BlockingChannelResponseHandler.java │ │ ├── Http2BlockingChannelStreamChannelInitializer.java │ │ ├── Http2ChannelPoolHandler.java │ │ ├── Http2ChannelPoolMap.java │ │ ├── Http2ClientMetrics.java │ │ ├── Http2ClientResponseHandler.java │ │ ├── Http2ClientStreamStatsHandler.java │ │ ├── Http2MultiplexedChannelPool.java │ │ ├── Http2NetworkClient.java │ │ ├── Http2NetworkClientFactory.java │ │ ├── Http2PeerCertificateValidator.java │ │ ├── Http2ServerMetrics.java │ │ ├── Http2ServerStreamHandler.java │ │ ├── Http2Utils.java │ │ ├── MultiplexedChannelRecord.java │ │ └── RequestsStatsHandler.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ └── network │ ├── BlockingChannelConnectionPoolTest.java │ ├── BoundedByteBufferSendTest.java │ ├── BoundedNettyByteBufReceiveTest.java │ ├── ByteBufferSend.java │ ├── CompositeNetworkClientTest.java │ ├── ConnectionTrackerTest.java │ ├── EchoServer.java │ ├── FifoNetworkRequestQueueTest.java │ ├── LocalNetworkClientTest.java │ ├── LocalRequestResponseChannelTest.java │ ├── NettyServerRequestResponseChannelTest.java │ ├── SSLBlockingChannelTest.java │ ├── SSLSelectorTest.java │ ├── SelectorTest.java │ ├── ServerRequestResponseHelperTest.java │ ├── SocketNetworkClientTest.java │ ├── SocketRequestResponseChannelTest.java │ ├── SocketServerTest.java │ └── http2 │ ├── AmbrySendToHttp2AdaptorTest.java │ ├── Http2BlockingChannelTest.java │ ├── Http2MultiplexedChannelPoolTest.java │ ├── Http2PeerCertificateValidatorTest.java │ ├── MockSend.java │ ├── MultiplexedChannelRecordTest.java │ └── RequestsStatsHandlerTest.java ├── ambry-prioritization └── src │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── replica │ │ └── prioritization │ │ ├── FCFSPrioritizationManager.java │ │ ├── FileBasedReplicationPrioritizationManagerFactory.java │ │ ├── FileCopyDisruptionBasedPrioritizationManager.java │ │ ├── PrioritizationManager.java │ │ ├── PrioritizationManagerFactory.java │ │ ├── PriorityTier.java │ │ ├── ReplicationPrioritizationManager.java │ │ ├── ReplicationPrioritizationMetrics.java │ │ └── disruption │ │ ├── DefaultDisruptionService.java │ │ ├── DisruptionService.java │ │ ├── Operation.java │ │ └── factory │ │ ├── DefaultDisruptionFactory.java │ │ └── DisruptionServiceFactory.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ └── replica │ └── prioritization │ ├── FSFCPrioritizationManagerTest.java │ ├── FileCopyDisruptionBasedPrioritizationManagerTest.java │ ├── PrioritizationManagerFactoryTest.java │ └── ReplicationPrioritizationManagerTest.java ├── ambry-protocol └── src │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── protocol │ │ ├── AdminRequest.java │ │ ├── AdminRequestOrResponseType.java │ │ ├── AdminResponse.java │ │ ├── AdminResponseWithContent.java │ │ ├── BatchDeletePartitionRequestInfo.java │ │ ├── BatchDeletePartitionResponseInfo.java │ │ ├── BatchDeleteRequest.java │ │ ├── BatchDeleteResponse.java │ │ ├── BlobDeleteStatus.java │ │ ├── BlobIndexAdminRequest.java │ │ ├── BlobStoreControlAction.java │ │ ├── BlobStoreControlAdminRequest.java │ │ ├── CatchupStatusAdminRequest.java │ │ ├── CatchupStatusAdminResponse.java │ │ ├── CompositeSend.java │ │ ├── DeleteRequest.java │ │ ├── DeleteResponse.java │ │ ├── FileCopyGetChunkRequest.java │ │ ├── FileCopyGetChunkResponse.java │ │ ├── FileCopyGetMetaDataRequest.java │ │ ├── FileCopyGetMetaDataResponse.java │ │ ├── ForceDeleteAdminRequest.java │ │ ├── GetRequest.java │ │ ├── GetResponse.java │ │ ├── HealthCheckAdminRequest.java │ │ ├── MessageInfoAndMetadataListSerde.java │ │ ├── PartitionRequestInfo.java │ │ ├── PartitionResponseInfo.java │ │ ├── PurgeRequest.java │ │ ├── PurgeResponse.java │ │ ├── PutRequest.java │ │ ├── PutResponse.java │ │ ├── ReplicaMetadataRequest.java │ │ ├── ReplicaMetadataRequestInfo.java │ │ ├── ReplicaMetadataResponse.java │ │ ├── ReplicaMetadataResponseInfo.java │ │ ├── ReplicateBlobRequest.java │ │ ├── ReplicateBlobResponse.java │ │ ├── ReplicationControlAdminRequest.java │ │ ├── RequestControlAdminRequest.java │ │ ├── RequestHandler.java │ │ ├── RequestHandlerPool.java │ │ ├── RequestOrResponse.java │ │ ├── RequestOrResponseType.java │ │ ├── RequestTracker.java │ │ ├── RequestVisitor.java │ │ ├── Response.java │ │ ├── TtlUpdateRequest.java │ │ ├── TtlUpdateResponse.java │ │ ├── UndeleteRequest.java │ │ └── UndeleteResponse.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ └── protocol │ ├── CompositeSendTest.java │ ├── MessageInfoAndMetadataListSerDeTest.java │ └── RequestResponseTest.java ├── ambry-quota └── src │ ├── integration-test │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── quota │ │ └── storage │ │ └── MySqlStorageUsageRefresherTest.java │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── quota │ │ ├── AmbryQuotaManager.java │ │ ├── AmbryQuotaManagerFactory.java │ │ ├── AtomicAmbryQuotaManager.java │ │ ├── PostProcessQuotaChargeCallback.java │ │ ├── PreProcessQuotaChargeCallback.java │ │ ├── QuotaResourceSynchronizer.java │ │ ├── QuotaUtils.java │ │ ├── SimpleQuotaRecommendationMergePolicy.java │ │ ├── SimpleRequestQuotaCostPolicy.java │ │ ├── capacityunit │ │ ├── AmbryCUQuotaEnforcer.java │ │ ├── AmbryCUQuotaEnforcerFactory.java │ │ ├── AmbryCUQuotaSource.java │ │ ├── AmbryCUQuotaSourceFactory.java │ │ ├── CapacityUnit.java │ │ └── JsonCUQuotaDataProviderUtil.java │ │ └── storage │ │ ├── JSONStringStorageQuotaSource.java │ │ ├── JSONStringStorageQuotaSourceFactory.java │ │ ├── MySqlStorageUsageRefresher.java │ │ ├── StorageQuotaEnforcer.java │ │ ├── StorageQuotaEnforcerFactory.java │ │ └── StorageQuotaServiceMetrics.java │ └── test │ ├── java │ └── com │ │ └── github │ │ └── ambry │ │ └── quota │ │ ├── AmbryQuotaManagerTest.java │ │ ├── AmbryQuotaManagerUpdateNotificationTest.java │ │ ├── PreProcessQuotaChargeCallbackTest.java │ │ ├── QuotaResourceSynchronizerTest.java │ │ ├── SimpleQuotaRecommendationMergePolicyTest.java │ │ ├── SimpleRequestQuotaCostPolicyTest.java │ │ ├── TestCUQuotaEnforcerFactory.java │ │ ├── capacityunit │ │ ├── AmbryCUQuotaEnforcerFactoryTest.java │ │ ├── AmbryCUQuotaEnforcerTest.java │ │ ├── AmbryCUQuotaSourceFactoryTest.java │ │ ├── AmbryCUQuotaSourceTest.java │ │ └── JsonCUQuotaDataProviderUtilTest.java │ │ └── storage │ │ ├── JSONStringStorageQuotaSourceTest.java │ │ └── StorageQuotaEnforcerTest.java │ └── resources │ └── mysql.properties ├── ambry-replication └── src │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── replication │ │ ├── BackupCheckerFileManager.java │ │ ├── BackupCheckerThread.java │ │ ├── BlobIdTransformer.java │ │ ├── DiskTokenPersistor.java │ │ ├── PartitionInfo.java │ │ ├── RemoteReplicaInfo.java │ │ ├── ReplicaThread.java │ │ ├── ReplicaTokenPersistor.java │ │ ├── ReplicationEngine.java │ │ ├── ReplicationManager.java │ │ ├── ReplicationMetrics.java │ │ └── continuous │ │ ├── ActiveGroupTracker.java │ │ ├── DataNodeTracker.java │ │ ├── GroupTracker.java │ │ ├── ReplicaStatus.java │ │ ├── ReplicaTracker.java │ │ └── StandByGroupTracker.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ └── replication │ ├── BackupCheckerThreadTest.java │ ├── BlobIdTransformerTest.java │ ├── DiskTokenPersistorTest.java │ ├── InMemoryStore.java │ ├── LeaderBasedReplicationTest.java │ ├── MockConnectionPool.java │ ├── MockFindToken.java │ ├── MockFindTokenFactory.java │ ├── MockFindTokenHelper.java │ ├── MockHost.java │ ├── MockNetworkClient.java │ ├── MockNetworkClientFactory.java │ ├── MockReplicationManager.java │ ├── RemoteReplicaGroupPollerTest.java │ ├── ReplicationTest.java │ └── ReplicationTestHelper.java ├── ambry-rest └── src │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── rest │ │ ├── AsyncRequestResponseHandler.java │ │ ├── AsyncRequestResponseHandlerFactory.java │ │ ├── ConnectionStatsHandler.java │ │ ├── FrontendNettyChannelInitializer.java │ │ ├── FrontendNettyFactory.java │ │ ├── HealthCheckHandler.java │ │ ├── NettyMessageProcessor.java │ │ ├── NettyMetrics.java │ │ ├── NettyMultipartRequest.java │ │ ├── NettyRequest.java │ │ ├── NettyResponseChannel.java │ │ ├── NettyServer.java │ │ ├── NoOpResponseChannel.java │ │ ├── PublicAccessLogHandler.java │ │ ├── PublicAccessLogger.java │ │ ├── RestServer.java │ │ ├── RestServerMain.java │ │ ├── RestServerState.java │ │ ├── ServerSecurityHandler.java │ │ ├── ServerSecurityService.java │ │ ├── ServerSecurityServiceFactory.java │ │ ├── StorageServerNettyChannelInitializer.java │ │ ├── StorageServerNettyFactory.java │ │ └── WrappedRestRequest.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ └── rest │ ├── AsyncRequestResponseHandlerFactoryTest.java │ ├── AsyncRequestResponseHandlerTest.java │ ├── EchoMethodHandler.java │ ├── FaultyFactory.java │ ├── FrontendNettyFactoryTest.java │ ├── HealthCheckHandlerTest.java │ ├── MockPublicAccessLogger.java │ ├── NettyMessageProcessorTest.java │ ├── NettyMultipartRequestTest.java │ ├── NettyRequestTest.java │ ├── NettyResponseChannelTest.java │ ├── NettyServerTest.java │ ├── PublicAccessLogHandlerTest.java │ ├── PublicAccessLoggerTest.java │ ├── RestServerStateTest.java │ ├── RestServerTest.java │ ├── RestTestUtils.java │ └── ServerSecurityHandlerTest.java ├── ambry-router └── src │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── router │ │ ├── AdaptiveOperationTracker.java │ │ ├── BackgroundDeleteRequest.java │ │ ├── BatchOperationCallbackTracker.java │ │ ├── BlobMetadata.java │ │ ├── CompositeBlobOperationHelper.java │ │ ├── CompressionMetrics.java │ │ ├── CompressionService.java │ │ ├── CryptoJob.java │ │ ├── CryptoJobHandler.java │ │ ├── DecryptJob.java │ │ ├── DeleteManager.java │ │ ├── DeleteOperation.java │ │ ├── EncryptJob.java │ │ ├── GCMCryptoService.java │ │ ├── GCMCryptoServiceFactory.java │ │ ├── GetBlobInfoOperation.java │ │ ├── GetBlobOperation.java │ │ ├── GetManager.java │ │ ├── GetOperation.java │ │ ├── NonBlockingRouter.java │ │ ├── NonBlockingRouterFactory.java │ │ ├── NonBlockingRouterMetrics.java │ │ ├── OperationController.java │ │ ├── OperationQuotaCharger.java │ │ ├── OperationTracker.java │ │ ├── ParanoidDurabilityOperationTracker.java │ │ ├── ProgressTracker.java │ │ ├── PutManager.java │ │ ├── PutOperation.java │ │ ├── QuotaAwareOperationController.java │ │ ├── ReplicateBlobCallback.java │ │ ├── ReplicateBlobManager.java │ │ ├── ReplicateBlobOperation.java │ │ ├── RequestRegistrationCallback.java │ │ ├── RouterCallback.java │ │ ├── RouterOperation.java │ │ ├── RouterUtils.java │ │ ├── SimpleOperationTracker.java │ │ ├── SingleKeyManagementService.java │ │ ├── SingleKeyManagementServiceFactory.java │ │ ├── TrackedRequestFinalState.java │ │ ├── TtlUpdateManager.java │ │ ├── TtlUpdateOperation.java │ │ ├── UndeleteManager.java │ │ ├── UndeleteOperation.java │ │ └── UndeleteOperationTracker.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ └── router │ ├── AdaptiveOperationTrackerTest.java │ ├── AmbryCacheStats.java │ ├── AmbryCacheWithStats.java │ ├── BatchOperationCallbackTrackerTest.java │ ├── ChunkFillTest.java │ ├── CompressionMetricsTest.java │ ├── CompressionServiceTest.java │ ├── CryptoJobHandlerTest.java │ ├── CryptoServiceTest.java │ ├── CryptoTestUtils.java │ ├── DeleteManagerTest.java │ ├── GCMCryptoServiceTest.java │ ├── GetBlobInfoOperationTest.java │ ├── GetBlobOperationTest.java │ ├── GetManagerTest.java │ ├── MockCompositeNetworkClient.java │ ├── MockCryptoService.java │ ├── MockKeyManagementService.java │ ├── MockNetworkClient.java │ ├── MockNetworkClientFactory.java │ ├── MockSelector.java │ ├── MockServer.java │ ├── MockServerLayout.java │ ├── NonBlockingQuotaTest.java │ ├── NonBlockingRouterTest.java │ ├── NonBlockingRouterTestBase.java │ ├── OperationQuotaChargerTest.java │ ├── OperationTrackerTest.java │ ├── ParanoidDurabilityOperationTrackerTest.java │ ├── PutManagerTest.java │ ├── PutOperationTest.java │ ├── QuotaAwareOperationControllerTest.java │ ├── QuotaRejectingOperationController.java │ ├── RejectingQuotaEnforcerFactory.java │ ├── RouterFactoryTest.java │ ├── RouterTestHelpers.java │ ├── RouterUtilsTest.java │ ├── S3NonBlockingRouterTest.java │ ├── SingleKeyManagementServiceTest.java │ ├── TestCUQuotaSourceFactory.java │ ├── TrackedRequestFinalStateTest.java │ ├── TtlUpdateManagerTest.java │ ├── UndeleteManagerTest.java │ └── UndeleteOperationTrackerTest.java ├── ambry-server └── src │ ├── integration-test │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── server │ │ ├── Http2NetworkClientTest.java │ │ ├── RouterServerHttp2Test.java │ │ ├── RouterServerPlaintextTest.java │ │ ├── RouterServerSSLTest.java │ │ ├── RouterServerTestFramework.java │ │ ├── ServerBatchDeleteTest.java │ │ ├── ServerHardDeleteTest.java │ │ ├── ServerHttp2Test.java │ │ ├── ServerPlaintextTest.java │ │ ├── ServerPlaintextTokenTest.java │ │ ├── ServerSSLTest.java │ │ ├── ServerSSLTokenTest.java │ │ └── StatsManagerIntegrationTest.java │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── server │ │ ├── AmbryMain.java │ │ ├── AmbryRequests.java │ │ ├── AmbryServer.java │ │ ├── AmbryServerClusterMapChangeListenerImpl.java │ │ ├── AmbryServerRequests.java │ │ ├── AmbryServerSecurityService.java │ │ ├── AmbryServerSecurityServiceFactory.java │ │ ├── AmbryStatsReportImpl.java │ │ ├── ParticipantsConsistencyChecker.java │ │ ├── RepairRequestsSender.java │ │ ├── ServerHealthStatus.java │ │ ├── StatsManager.java │ │ └── StatsManagerMetrics.java │ └── test │ ├── java │ └── com │ │ └── github │ │ └── ambry │ │ └── server │ │ ├── AmbryServerRequestsTest.java │ │ ├── AmbryServerSecurityServiceFactoryTest.java │ │ ├── AmbryServerSecurityServiceTest.java │ │ ├── AmbryServerTest.java │ │ ├── AmbryStatsReportTest.java │ │ ├── MockStatsManager.java │ │ ├── MockStorageManager.java │ │ ├── ParticipantsConsistencyTest.java │ │ └── StatsManagerTest.java │ └── resources │ ├── accountstats_mysql.properties │ └── repairRequests_mysql.properties ├── ambry-store └── src │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── store │ │ ├── BlobStore.java │ │ ├── BlobStoreCompactor.java │ │ ├── BlobStoreStats.java │ │ ├── BootstrapController.java │ │ ├── BootstrapSession.java │ │ ├── BootstrapSessionManager.java │ │ ├── CompactAllPolicyFactory.java │ │ ├── CompactionDetails.java │ │ ├── CompactionLog.java │ │ ├── CompactionManager.java │ │ ├── CompactionPolicy.java │ │ ├── CompactionPolicyCounter.java │ │ ├── CompactionPolicyFactory.java │ │ ├── CompactionPolicySwitchInfo.java │ │ ├── CostBenefitInfo.java │ │ ├── DiskHealthStatus.java │ │ ├── DiskIOScheduler.java │ │ ├── DiskManager.java │ │ ├── DiskMetrics.java │ │ ├── DiskSpaceAllocator.java │ │ ├── DiskSpaceRequirements.java │ │ ├── FileSpan.java │ │ ├── FileStore.java │ │ ├── FindEntriesCondition.java │ │ ├── HardDeleter.java │ │ ├── HybridCompactionPolicy.java │ │ ├── HybridCompactionPolicyFactory.java │ │ ├── IndexEntry.java │ │ ├── IndexSegment.java │ │ ├── IndexSegmentValidEntryFilter.java │ │ ├── IndexValue.java │ │ ├── Journal.java │ │ ├── Log.java │ │ ├── LogSegment.java │ │ ├── LogSegmentName.java │ │ ├── LogSegmentSizeProvider.java │ │ ├── MessageInfoType.java │ │ ├── Offset.java │ │ ├── PartitionFinder.java │ │ ├── PersistentIndex.java │ │ ├── RemoteTokenTracker.java │ │ ├── ReplicaPlacementValidator.java │ │ ├── ScanResults.java │ │ ├── StatsBasedCompactionPolicy.java │ │ ├── StatsBasedCompactionPolicyFactory.java │ │ ├── StatsUtils.java │ │ ├── StorageManager.java │ │ ├── StorageManagerMetrics.java │ │ ├── StoreDescriptor.java │ │ ├── StoreFindToken.java │ │ ├── StoreFindTokenFactory.java │ │ ├── StoreMessageReadSet.java │ │ └── StoreMetrics.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ └── store │ ├── BlobStoreCompactorTest.java │ ├── BlobStoreStatsTest.java │ ├── BlobStoreTest.java │ ├── BootstrapControllerTest.java │ ├── BootstrapSessionManagerTest.java │ ├── BootstrapSessionTest.java │ ├── CompactAllPolicyTest.java │ ├── CompactionDetailsTest.java │ ├── CompactionLogTest.java │ ├── CompactionManagerTest.java │ ├── CompactionPolicyFactoryTest.java │ ├── CompactionPolicyTest.java │ ├── CostBenefitInfoTest.java │ ├── CuratedLogIndexState.java │ ├── DiskIOSchedulerTest.java │ ├── DiskSpaceAllocatorTest.java │ ├── DiskSpaceRequirementsTest.java │ ├── DummyMessageStoreHardDelete.java │ ├── DummyMessageStoreRecovery.java │ ├── FileSpanTest.java │ ├── FileStoreTest.java │ ├── HardDeleterTest.java │ ├── IndexSegmentTest.java │ ├── IndexTest.java │ ├── IndexValueTest.java │ ├── JournalTest.java │ ├── LogSegmentNameTest.java │ ├── LogSegmentTest.java │ ├── LogTest.java │ ├── MockJournal.java │ ├── MockMessageWriteSet.java │ ├── OffsetTest.java │ ├── StatsBasedCompactionPolicyTest.java │ ├── StorageManagerTest.java │ ├── StoreDescriptorTest.java │ ├── StoreFindTokenTest.java │ ├── StoreMessageReadSetTest.java │ ├── StoreTestUtils.java │ └── TimeRangeTest.java ├── ambry-test-utils └── src │ └── main │ └── java │ └── com │ └── github │ └── ambry │ ├── account │ ├── InMemAccountService.java │ └── InMemAccountServiceFactory.java │ ├── clustermap │ ├── MockClusterAgentsFactory.java │ ├── MockClusterMap.java │ ├── MockDataNodeId.java │ ├── MockDiskId.java │ ├── MockPartitionId.java │ ├── MockReplicaId.java │ ├── MockVcrClusterAgentsFactory.java │ └── MockVcrClusterSpectator.java │ ├── commons │ ├── ByteBufReadableStreamChannel.java │ ├── CommonTestUtils.java │ └── TestSSLUtils.java │ ├── filecopy │ ├── MockFileCopyHandlerFactory.java │ └── MockNoOpFileCopyHandler.java │ ├── quota │ ├── CostPolicyTestUtils.java │ ├── DummyQuotaSource.java │ ├── DummyQuotaSourceFactory.java │ ├── QuotaTestUtils.java │ ├── RejectQuotaEnforcerFactory.java │ └── RejectRequestQuotaEnforcer.java │ ├── rest │ ├── MockRestRequest.java │ ├── MockRestResponseChannel.java │ ├── MockSSLEngine.java │ ├── MockSSLSession.java │ └── NettyClient.java │ ├── router │ ├── ByteBufferAWC.java │ ├── ByteBufferRSC.java │ ├── InMemoryRouter.java │ └── InMemoryRouterFactory.java │ ├── server │ ├── DirectSender.java │ ├── MockCluster.java │ ├── MockNotificationSystem.java │ ├── Payload.java │ ├── ServerTestUtil.java │ └── Verifier.java │ ├── store │ ├── MockId.java │ ├── MockIdFactory.java │ └── MockStoreKeyConverterFactory.java │ └── utils │ ├── AccountTestUtils.java │ ├── HelixControllerManager.java │ ├── MockClock.java │ ├── MockTime.java │ ├── NettyByteBufLeakHelper.java │ └── TestUtils.java ├── ambry-tools ├── scripts │ └── layout-analyzer.py └── src │ ├── integration-test │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ ├── account │ │ └── AccountUpdateToolTest.java │ │ └── clustermap │ │ ├── HelixBootstrapUpgradeToolTest.java │ │ └── HelixVcrPopulateToolTest.java │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ ├── Readme.md │ │ ├── account │ │ ├── AccountTool.java │ │ ├── AccountUpdateTool.java │ │ ├── DatabaseTest.java │ │ ├── MySqlAccountsDBTool.java │ │ └── ServiceIdAccountGenTool.java │ │ ├── cloud │ │ └── azure │ │ │ └── AzureCompactionTool.java │ │ ├── clustermap │ │ ├── HelixBootstrapUpgradeTool.java │ │ ├── HelixTaskWorkflowManagerTool.java │ │ ├── HelixVcrPopulateTool.java │ │ └── PartitionManager.java │ │ ├── store │ │ ├── BlobIndexMetrics.java │ │ ├── CompactionVerifier.java │ │ ├── ConsistencyCheckerTool.java │ │ ├── DiskReformatter.java │ │ ├── DumpCompactionLogTool.java │ │ ├── DumpDataHelper.java │ │ ├── DumpDataTool.java │ │ ├── DumpIndexTool.java │ │ ├── DumpLogTool.java │ │ ├── DumpReplicaTokenTool.java │ │ ├── HardDeleteVerifier.java │ │ ├── IndexReadPerformance.java │ │ ├── IndexWritePerformance.java │ │ ├── StoreCopier.java │ │ └── StoreToolsMetrics.java │ │ └── tools │ │ ├── admin │ │ ├── AzureTokenResetTool.java │ │ ├── BlobValidator.java │ │ ├── ConcurrencyTestTool.java │ │ ├── DirectoryUploader.java │ │ ├── PutGetHelperFactory.java │ │ ├── RouterPutGetHelperFactory.java │ │ ├── SafeServerShutdownTool.java │ │ ├── ServerAdminTool.java │ │ └── TestPartitionAlloc.java │ │ ├── perf │ │ ├── Crc32Benchmark.java │ │ ├── FileSystemWritePerformance.java │ │ ├── NamedBlobMysqlDatabasePerf.java │ │ ├── ServerReadPerformance.java │ │ ├── ServerWritePerformance.java │ │ ├── rest │ │ │ ├── NettyPerfClient.java │ │ │ ├── NoOpAWC.java │ │ │ ├── PerfConfig.java │ │ │ ├── PerfNioServer.java │ │ │ ├── PerfNioServerFactory.java │ │ │ ├── PerfRSC.java │ │ │ ├── PerfRestServerMain.java │ │ │ ├── PerfRouter.java │ │ │ └── PerfRouterFactory.java │ │ └── serverperf │ │ │ ├── GetLoadProducerConsumer.java │ │ │ ├── LoadProducerConsumer.java │ │ │ ├── PutLoadProducerConsumer.java │ │ │ ├── ResponseInfoProcessor.java │ │ │ ├── ServerPerfNetworkQueue.java │ │ │ ├── ServerPerformance.java │ │ │ └── ShutDownException.java │ │ └── util │ │ ├── ToolRequestResponseUtil.java │ │ └── ToolUtils.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ └── store │ └── StoreCopierTest.java ├── ambry-utils └── src │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── utils │ │ ├── AbstractByteBufHolder.java │ │ ├── AsyncOperationTracker.java │ │ ├── BatchBlockingQueue.java │ │ ├── BitUtil.java │ │ ├── BloomCalculations.java │ │ ├── BloomFilter.java │ │ ├── BloomFilterSerializer.java │ │ ├── ByteBufChannel.java │ │ ├── ByteBufferChannel.java │ │ ├── ByteBufferDataInputStream.java │ │ ├── ByteBufferInputStream.java │ │ ├── ByteBufferOutputStream.java │ │ ├── ByteBufferPool.java │ │ ├── CachedHistogram.java │ │ ├── ChannelWriter.java │ │ ├── Crc32.java │ │ ├── CrcInputStream.java │ │ ├── CrcOutputStream.java │ │ ├── ExtendableTimer.java │ │ ├── FileLock.java │ │ ├── FilterFactory.java │ │ ├── GenericThrowableConsumer.java │ │ ├── IBitSet.java │ │ ├── IFilter.java │ │ ├── InvocationOptions.java │ │ ├── Murmur3BloomFilter.java │ │ ├── MurmurHash.java │ │ ├── NettyByteBufDataInputStream.java │ │ ├── OpenBitSet.java │ │ ├── Pair.java │ │ ├── RejectThrottler.java │ │ ├── SimpleByteBufferPool.java │ │ ├── Singleton.java │ │ ├── SystemTime.java │ │ ├── Throttler.java │ │ ├── ThrowingBiConsumer.java │ │ ├── ThrowingConsumer.java │ │ ├── ThrowingFunction.java │ │ ├── Time.java │ │ ├── Utils.java │ │ └── ZeroBytesInputStream.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ └── utils │ ├── BatchBlockingQueueTest.java │ ├── BitUtilTest.java │ ├── BloomFilterTest.java │ ├── ByteBufferChannelTest.java │ ├── ByteBufferInputStreamTest.java │ ├── CachedHistogramTest.java │ ├── ChannelWriterTest.java │ ├── Crc32Test.java │ ├── CrcInputStreamTest.java │ ├── CrcOutputStreamTest.java │ ├── FileLockTest.java │ ├── FilterTestHelper.java │ ├── KeyGenerator.java │ ├── MurmurHashTest.java │ ├── OpenBitSetTest.java │ ├── PairTest.java │ ├── RejectThrottlerTest.java │ ├── ResetableIterator.java │ ├── SimpleByteBufferPoolTest.java │ ├── ThrottlerTest.java │ ├── UtilsTest.java │ └── ZeroBytesInputStreamTest.java ├── ambry-vcr └── src │ ├── main │ └── java │ │ └── com │ │ └── github │ │ └── ambry │ │ └── vcr │ │ ├── AmbryCloudRequests.java │ │ ├── VcrMain.java │ │ ├── VcrRequests.java │ │ └── VcrServer.java │ └── test │ └── java │ └── com │ └── github │ └── ambry │ └── vcr │ ├── CloudAndStoreReplicationTest.java │ ├── CloudBlobStoreTest.java │ ├── CloudStorageManagerTest.java │ ├── CloudTokenPersistorTest.java │ ├── DeprecatedContainerCloudSyncTaskTest.java │ ├── HelixVcrClusterParticipantTest.java │ ├── LatchBasedInMemoryCloudDestination.java │ ├── LatchBasedInMemoryCloudDestinationFactory.java │ ├── VcrAutomationTest.java │ ├── VcrBackupTest.java │ ├── VcrRecoveryTest.java │ ├── VcrRequestsTest.java │ ├── VcrServerTest.java │ └── VcrTestUtil.java ├── build.gradle ├── config ├── DataDumplog4j.props ├── HardwareLayout.json ├── HardwareLayoutHelix.json ├── HardwareLayoutHttp2.json ├── HardwareLayoutLocalThreeNodeOneDisk.json ├── HardwareLayoutLocalThreeNodeOneDiskSsl.json ├── HardwareLayoutMultiPartition.json ├── HardwareLayoutSsl.json ├── HelixTaskWorkflowManagerConfig.properties ├── PartitionLayout.json ├── PartitionLayoutHelix.json ├── PartitionLayoutLocalOnePartitionThreeReplicas.json ├── PartitionLayoutMultiPartition.json ├── WagedRebalancerHelixConfig.json ├── azureconfig │ ├── HardwareLayout.json │ ├── PartitionLayout.json │ ├── server.properties │ └── vcr.properties ├── concurrencyTool.router.properties ├── frontend.http2.properties ├── frontend.properties ├── frontend.ssl.properties ├── frontend.test.properties ├── frontend_helix.properties ├── frontend_json.properties ├── log4j2.xml ├── mysql.properties ├── perf.rest.server.properties ├── server.http2.properties ├── server.properties ├── server.ssl.properties ├── server1.properties ├── server1.ssl.properties ├── server1_helix.properties ├── server2.properties ├── server2.ssl.properties ├── server2_helix.properties ├── server3.properties ├── server3.ssl.properties ├── server3_helix.properties ├── server4_helix.properties ├── server5_helix.properties ├── server6_helix.properties ├── vcrhelixconfig │ ├── CrushEdRebalancerHelixConfig.json │ └── DelayedAutoRebalancerHelixConfig.json └── zkLayout.json ├── gradle.properties ├── gradle ├── buildscript.gradle ├── ci-release.gradle ├── dependency-versions.gradle ├── environment.gradle ├── java-publishing.gradle ├── license.gradle └── wrapper │ ├── gradle-wrapper.jar │ └── gradle-wrapper.properties ├── gradlew ├── gradlew.bat ├── ide └── intellij │ └── codestyles │ └── ambry-style.xml ├── log4j-test-config └── src │ └── main │ └── resources │ └── log4j2.xml ├── remove-dot-dirs.sh ├── settings.gradle └── version.properties /.github/pull_request_template.md: -------------------------------------------------------------------------------- 1 | ## Summary 2 | 3 | 4 | 5 | ## Testing Done 6 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | *.class 2 | *.war 3 | *.ear 4 | target/ 5 | .classpath 6 | .project 7 | .settings/ 8 | .idea/ 9 | .idea_modules/ 10 | *.iml 11 | *.ipr 12 | *.iws 13 | *.jar 14 | */.cache 15 | deploy 16 | *~ 17 | .gradle 18 | */build/ 19 | out/ 20 | jars/ 21 | build/ 22 | logs/ 23 | null/ 24 | .DS_Store 25 | 26 | .remote* 27 | 28 | # files created by azurite for unit testing 29 | __* 30 | AzuriteConfig 31 | -------------------------------------------------------------------------------- /HEADER: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | -------------------------------------------------------------------------------- /ambry-account/src/main/java/com/github/ambry/account/AccountServiceMetricsWrapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2023 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.account; 15 | 16 | import com.codahale.metrics.MetricRegistry; 17 | 18 | 19 | public class AccountServiceMetricsWrapper { 20 | private final AccountServiceMetrics accountServiceMetrics; 21 | 22 | public AccountServiceMetricsWrapper(MetricRegistry metricRegistry) { 23 | this.accountServiceMetrics = new AccountServiceMetrics(metricRegistry); 24 | } 25 | 26 | public AccountServiceMetrics getAccountServiceMetrics() {return this.accountServiceMetrics;} 27 | } 28 | -------------------------------------------------------------------------------- /ambry-account/src/test/accounts.json: -------------------------------------------------------------------------------- 1 | [ 2 | { 3 | "accountId": 1234, 4 | "accountName": "s3tests", 5 | "containers": [ 6 | { 7 | "containerName": "s3testbucket", 8 | "description": "This is my private container", 9 | "isPrivate": "true", 10 | "containerId": 5678, 11 | "version": 2, 12 | "status": "ACTIVE", 13 | "parentAccountId": "1234", 14 | "namedBlobMode": "OPTIONAL" 15 | }, 16 | { 17 | "containerName": "s3_emptybucket", 18 | "description": "This is my empty container", 19 | "isPrivate": "true", 20 | "containerId": 5679, 21 | "version": 2, 22 | "status": "ACTIVE", 23 | "parentAccountId": "1234", 24 | "namedBlobMode": "OPTIONAL" 25 | } 26 | ], 27 | "version": 1, 28 | "status": "ACTIVE", 29 | "quotaResourceType": "CONTAINER" 30 | } 31 | ] -------------------------------------------------------------------------------- /ambry-account/src/test/resources/mysql.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2020 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | mysql.account.service.db.info=[{"url":"jdbc:mysql://localhost/AccountMetadata?serverTimezone=UTC","datacenter":"dc1","isWriteable":"true","username":"travis","password":""}] 13 | mysql.account.service.db.info.new=[{"url":"jdbc:mysql://localhost/AccountMetadata?serverTimezone=UTC","datacenter":"dc1","isWriteable":"true","username":"travis","password":""}] 14 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/account/AccountServiceException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.account; 15 | 16 | /** 17 | * Exceptions thrown by {@link AccountService}. All exceptions are accompanied by a {@link AccountServiceErrorCode}. 18 | */ 19 | public class AccountServiceException extends Exception { 20 | private final AccountServiceErrorCode error; 21 | 22 | public AccountServiceException(String message, AccountServiceErrorCode error) { 23 | super(message); 24 | this.error = error; 25 | } 26 | 27 | public AccountServiceErrorCode getErrorCode() { 28 | return error; 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/accountstats/AccountStatsStoreFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2021 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.accountstats; 15 | 16 | /** 17 | * Factory interface to return a {@link AccountStatsStore}. 18 | */ 19 | public interface AccountStatsStoreFactory { 20 | /** 21 | * Returns an instance of the {@link AccountStatsStore} that the factory generates. 22 | * @return an instance of {@link AccountStatsStore} generated by this factory. 23 | * @throws Exception if the {@link AccountStatsStore} instance cannot be created. 24 | */ 25 | AccountStatsStore getAccountStatsStore() throws Exception; 26 | } 27 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/clustermap/DistributedLockLocalImpl.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2023 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.clustermap; 15 | 16 | import java.util.concurrent.locks.ReentrantLock; 17 | 18 | 19 | /** 20 | * A reentrant lock implementation for {@link DistributedLock}. This is a local lock that only works for one process. 21 | * 22 | */ 23 | public class DistributedLockLocalImpl implements DistributedLock { 24 | private final ReentrantLock lock = new ReentrantLock(); 25 | 26 | @Override 27 | public boolean tryLock() { 28 | return lock.tryLock(); 29 | } 30 | 31 | @Override 32 | public void unlock() { 33 | lock.unlock(); 34 | } 35 | 36 | @Override 37 | public void close() { 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/clustermap/HardwareState.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.clustermap; 15 | 16 | /** 17 | * The states that a hardware component may be in. 18 | */ 19 | public enum HardwareState { 20 | AVAILABLE, UNAVAILABLE 21 | } 22 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/clustermap/PartitionState.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.clustermap; 15 | 16 | /** 17 | * The valid states for a {@link PartitionId}. The state of a partition is resolved via the {@link ReplicaSealStatus}es 18 | * of the partition's replicas. See ClusterMapUtils 19 | * 20 | * The state transition of a partition's state will always follow the following order: 21 | * READ_WRITE <-> PARTIAL_READ_WRITE <-> READ_ONLY 22 | */ 23 | public enum PartitionState { 24 | /** The partition is available for all reads and writes */ 25 | READ_WRITE, 26 | /** The partition is available for all reads but limited writes */ 27 | PARTIAL_READ_WRITE, 28 | /** The partition is available for reads only. */ 29 | READ_ONLY 30 | } 31 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/clustermap/ReplicaType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.clustermap; 16 | 17 | /** 18 | * The type of replica. 19 | */ 20 | public enum ReplicaType { 21 | DISK_BACKED, CLOUD_BACKED 22 | } 23 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/clustermap/Resource.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.clustermap; 15 | 16 | import org.json.JSONObject; 17 | 18 | 19 | /** 20 | * A Resource represents a {@link DiskId}, {@link DataNodeId}, a {@link PartitionId} or a {@link ReplicaId} 21 | */ 22 | 23 | public interface Resource { 24 | /** 25 | * @return a snapshot which includes information that the implementation considers relevant. 26 | */ 27 | JSONObject getSnapshot(); 28 | } 29 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/clustermap/ResourceStatePolicyFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.clustermap; 15 | 16 | /* Factory class interface to get resource state policies */ 17 | 18 | public interface ResourceStatePolicyFactory { 19 | 20 | ResourceStatePolicy getResourceStatePolicy(); 21 | } 22 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/commons/PerformanceIndex.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.commons; 15 | 16 | /** 17 | * The indices used to evaluate performance of single request. 18 | */ 19 | public enum PerformanceIndex { 20 | /** 21 | * Time duration in milliseconds from request arrival to the first byte of response being received. 22 | */ 23 | TimeToFirstByte, 24 | /** 25 | * Time duration in milliseconds from request arrival to its fully completion. 26 | */ 27 | RoundTripTime, 28 | /** 29 | * Average speed of data being transmitted (bytes/sec). 30 | */ 31 | AverageBandwidth 32 | } 33 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/commons/TopicListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.commons; 15 | 16 | /** 17 | * A {@code TopicListener} can subscribe topics for messages through a {@link Notifier}. 18 | * @param The type of message the listener will receive. 19 | */ 20 | public interface TopicListener { 21 | 22 | /** 23 | * After the {@link TopicListener} has subscribed the topic, this method will be called when there is a new message 24 | * for the topic. 25 | * @param topic The topic this {@code TopicListener} subscribes. 26 | * @param message The message for the topic. 27 | */ 28 | public void onMessage(String topic, T message); 29 | } 30 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/compression/CompressionException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.compression; 15 | 16 | /** 17 | * Generic compression failure exception. 18 | */ 19 | public class CompressionException extends Exception { 20 | /** 21 | * Create new instance with a specific message. 22 | * @param message The error message. 23 | */ 24 | public CompressionException(String message) { 25 | super(message); 26 | } 27 | 28 | /** 29 | * Create new instance with specific error message and cause. 30 | * @param message The error message. 31 | * @param cause Cause/inner exception. 32 | */ 33 | public CompressionException(String message, Exception cause) { 34 | super(message, cause); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/config/Config.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.config; 15 | 16 | import java.lang.annotation.Documented; 17 | import java.lang.annotation.ElementType; 18 | import java.lang.annotation.Retention; 19 | import java.lang.annotation.RetentionPolicy; 20 | import java.lang.annotation.Target; 21 | 22 | 23 | @Documented 24 | @Target(ElementType.FIELD) 25 | @Retention(RetentionPolicy.RUNTIME) 26 | public @interface Config { 27 | String[] value(); 28 | } -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/config/Default.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.config; 15 | 16 | import java.lang.annotation.Documented; 17 | import java.lang.annotation.ElementType; 18 | import java.lang.annotation.Retention; 19 | import java.lang.annotation.RetentionPolicy; 20 | import java.lang.annotation.Target; 21 | 22 | 23 | @Documented 24 | @Target(ElementType.FIELD) 25 | @Retention(RetentionPolicy.RUNTIME) 26 | public @interface Default { 27 | String value(); 28 | } -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/config/ReplicaPrioritizationConfig.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2025 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.config; 16 | 17 | public class ReplicaPrioritizationConfig { 18 | public static final String REPLICA_PRIORITIZATION_STRATEGY_FOR_FILE_COPY = "replica.prioritization.strategy.for.file.copy"; 19 | @Config(REPLICA_PRIORITIZATION_STRATEGY_FOR_FILE_COPY) 20 | public final ReplicaPrioritizationStrategy replicaPrioritizationStrategy; 21 | 22 | public ReplicaPrioritizationConfig(VerifiableProperties verifiableProperties) { 23 | replicaPrioritizationStrategy = ReplicaPrioritizationStrategy.valueOf(verifiableProperties.getString(REPLICA_PRIORITIZATION_STRATEGY_FOR_FILE_COPY, 24 | ReplicaPrioritizationStrategy.FirstComeFirstServe.name())); 25 | } 26 | } 27 | 28 | 29 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/config/ReplicaPrioritizationStrategy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2025 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.config; 16 | 17 | public enum ReplicaPrioritizationStrategy { 18 | FirstComeFirstServe, 19 | ACMAdvanceNotificationsBased 20 | } 21 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/config/ReplicaSelectionPolicy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.config; 15 | 16 | /** 17 | * Specifies how the backup-node must select server replicas to back up. 18 | * FIXED: The backup-node will select the same server replica each time for a partition 19 | * ROUND_ROBIN: The backup-node will cycle through all replicas of a partition 20 | */ 21 | public enum ReplicaSelectionPolicy { 22 | FIXED, 23 | ROUND_ROBIN 24 | } 25 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/config/ServerExecutionMode.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.config; 15 | 16 | public enum ServerExecutionMode { 17 | DATA_RECOVERY_MODE, 18 | DATA_VERIFICATION_MODE, 19 | DATA_SERVING_MODE; 20 | } 21 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/config/ServerReplicationMode.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.config; 15 | public enum ServerReplicationMode { 16 | BLOB_BASED, 17 | FILE_BASED; 18 | } 19 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/frontend/IdConverterFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.frontend; 15 | 16 | /** 17 | * IdConverterFactory is a factory to generate all the supporting cast required to instantiate a {@link IdConverter}. 18 | *

19 | * Usually called with the canonical class name and as such might have to support appropriate (multiple) constructors. 20 | */ 21 | public interface IdConverterFactory { 22 | 23 | /** 24 | * Returns an instance of the {@link IdConverter} that the factory generates. 25 | * @return an instance of {@link IdConverter} generated by this factory. 26 | * @throws InstantiationException if the {@link IdConverter} instance cannot be created. 27 | */ 28 | public IdConverter getIdConverter() throws InstantiationException; 29 | } 30 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/frontend/IdSigningServiceFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.frontend; 16 | 17 | /** 18 | * IdSigningServiceFactory is a factory to generate all the supporting cast required to instantiate a 19 | * {@link IdSigningService}. 20 | *

21 | * Usually called with the canonical class name and as such might have to support appropriate (multiple) constructors. 22 | */ 23 | public interface IdSigningServiceFactory { 24 | 25 | /** 26 | * Returns an instance of the {@link IdSigningService} that the factory generates. 27 | * @return an instance of {@link IdSigningService} generated by this factory. 28 | */ 29 | IdSigningService getIdSigningService(); 30 | } 31 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/frontend/UrlSigningServiceFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.frontend; 15 | 16 | /** 17 | * UrlSigningServiceFactory is a factory to generate all the supporting cast required to instantiate a 18 | * {@link UrlSigningService}. 19 | *

20 | * Usually called with the canonical class name and as such might have to support appropriate (multiple) constructors. 21 | */ 22 | public interface UrlSigningServiceFactory { 23 | 24 | /** 25 | * Returns an instance of the {@link UrlSigningService} that the factory generates. 26 | * @return an instance of {@link UrlSigningService} generated by this factory. 27 | */ 28 | UrlSigningService getUrlSigningService(); 29 | } 30 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/messageformat/BlobOutput.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.messageformat; 15 | 16 | import java.io.InputStream; 17 | 18 | 19 | /** 20 | * Contains the blob output 21 | */ 22 | public class BlobOutput { 23 | private long size; 24 | private InputStream stream; 25 | 26 | /** 27 | * The blob output that helps to read a blob 28 | * @param size The size of the blob 29 | * @param stream The stream that contains the blob 30 | */ 31 | public BlobOutput(long size, InputStream stream) { 32 | this.size = size; 33 | this.stream = stream; 34 | } 35 | 36 | public long getSize() { 37 | return size; 38 | } 39 | 40 | public InputStream getStream() { 41 | return stream; 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/messageformat/BlobType.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.messageformat; 15 | 16 | /** 17 | * Set of flags used to identify different types of blob content 18 | */ 19 | public enum BlobType { 20 | DataBlob, MetadataBlob 21 | } 22 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/messageformat/MessageFormatFlags.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.messageformat; 15 | 16 | /** 17 | * Set of flags used to identify different types of messages 18 | */ 19 | public enum MessageFormatFlags { 20 | BlobProperties, BlobUserMetadata, Blob, All, BlobInfo 21 | } 22 | 23 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/named/NamedBlobDbFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * 14 | */ 15 | 16 | package com.github.ambry.named; 17 | 18 | /** 19 | * A factory to build instances of {@link NamedBlobDb}. 20 | */ 21 | public interface NamedBlobDbFactory { 22 | /** 23 | * @return an instance of {@link NamedBlobDb}. 24 | * @throws Exception if there is an error during instantiation. 25 | */ 26 | NamedBlobDb getNamedBlobDb() throws Exception; 27 | } 28 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/named/PutResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * 14 | */ 15 | 16 | package com.github.ambry.named; 17 | 18 | /** 19 | * Class to convey information about a successful put in {@link NamedBlobDb}. 20 | */ 21 | public class PutResult { 22 | private final NamedBlobRecord insertedRecord; 23 | 24 | /** 25 | * @param insertedRecord the new record stored in the DB. 26 | */ 27 | public PutResult(NamedBlobRecord insertedRecord) { 28 | this.insertedRecord = insertedRecord; 29 | } 30 | 31 | /** 32 | * @return the new record stored in the DB. 33 | */ 34 | public NamedBlobRecord getInsertedRecord() { 35 | return insertedRecord; 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/named/TransactionIsolationLevel.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2023 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.named; 15 | 16 | import java.sql.Connection; 17 | 18 | 19 | /** 20 | * Transaction isolation level. Name of the enum must match to constant names in java {@link Connection} class such as 21 | * {@link Connection#TRANSACTION_NONE} or {@link Connection#TRANSACTION_READ_UNCOMMITTED} or 22 | * {@link Connection#TRANSACTION_READ_COMMITTED} or {@link Connection#TRANSACTION_REPEATABLE_READ} or 23 | * {@link Connection#TRANSACTION_SERIALIZABLE} 24 | */ 25 | public enum TransactionIsolationLevel { 26 | TRANSACTION_NONE, 27 | TRANSACTION_READ_UNCOMMITTED, 28 | TRANSACTION_READ_COMMITTED, 29 | TRANSACTION_REPEATABLE_READ, 30 | TRANSACTION_SERIALIZABLE; 31 | } 32 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/network/ChannelOutput.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.network; 15 | 16 | import java.io.DataInputStream; 17 | 18 | 19 | /** 20 | * The receive on the connected channel provides a ChannelOutput that 21 | * can be used to read the output from. 22 | */ 23 | public class ChannelOutput { 24 | 25 | private DataInputStream inputStream; 26 | 27 | private long streamSize; 28 | 29 | public ChannelOutput(DataInputStream inputStream, long streamSize) { 30 | this.inputStream = inputStream; 31 | this.streamSize = streamSize; 32 | } 33 | 34 | public DataInputStream getInputStream() { 35 | return inputStream; 36 | } 37 | 38 | public long getStreamSize() { 39 | return streamSize; 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/network/ConnectionPoolTimeoutException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.network; 15 | 16 | /** 17 | * Exception used by the connection pool to indicate that the operation timedout 18 | */ 19 | public class ConnectionPoolTimeoutException extends Exception { 20 | private static final long serialVersionUID = 1; 21 | 22 | public ConnectionPoolTimeoutException(String message) { 23 | super(message); 24 | } 25 | 26 | public ConnectionPoolTimeoutException(String message, Throwable e) { 27 | super(message, e); 28 | } 29 | 30 | public ConnectionPoolTimeoutException(Throwable e) { 31 | super(e); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/network/NetworkClientErrorCode.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.network; 15 | 16 | /** 17 | * Errors that can be received from the NetworkClient. 18 | */ 19 | public enum NetworkClientErrorCode { 20 | /** 21 | * Request could not be sent because a connection could not be checked out. 22 | */ 23 | ConnectionUnavailable, 24 | 25 | /** 26 | * A network error was encountered. 27 | */ 28 | NetworkError, 29 | 30 | /** 31 | * Request timed out. 32 | */ 33 | TimeoutError, 34 | } 35 | 36 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/network/NetworkClientFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.network; 15 | 16 | import java.io.IOException; 17 | 18 | 19 | /** 20 | * A factory class used to get new instances of a {@link NetworkClient} 21 | */ 22 | public interface NetworkClientFactory { 23 | 24 | /** 25 | * Construct and return a new {@link NetworkClient} 26 | * @return return a new {@link NetworkClient} 27 | * @throws IOException if the {@link NetworkClient} could not be instantiated. 28 | */ 29 | NetworkClient getNetworkClient() throws IOException; 30 | } 31 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/network/NetworkResponse.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.network; 15 | 16 | /** 17 | * Simple response 18 | */ 19 | public interface NetworkResponse { 20 | 21 | /** 22 | * Provides the send object that can be sent over the network 23 | * @return The send object that is part of this response 24 | */ 25 | Send getPayload(); 26 | 27 | /** 28 | * The original request object that this response maps to 29 | * @return The request object that maps to this response 30 | */ 31 | NetworkRequest getRequest(); 32 | } 33 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/network/PortType.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.network; 15 | 16 | /** 17 | * Supported port types 18 | */ 19 | public enum PortType { 20 | PLAINTEXT, SSL, HTTP2 21 | } -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/network/RequestQueueType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2022 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.network; 15 | 16 | /** 17 | * Request queue type. 18 | */ 19 | public enum RequestQueueType { 20 | BASIC_QUEUE_WITH_FIFO, 21 | ADAPTIVE_QUEUE_WITH_LIFO_CO_DEL 22 | } 23 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/network/SendWithCorrelationId.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.network; 15 | 16 | /** 17 | * Most {@link Send}s include a correlation ID that is used to associate a request/response pair. This can be used to 18 | * indicate that a correlation ID is required. 19 | */ 20 | public interface SendWithCorrelationId extends Send { 21 | 22 | /** 23 | * @return the correlation id for the {@link Send} (a process-unique identifier for a request). 24 | */ 25 | int getCorrelationId(); 26 | 27 | /** 28 | * @return The type of the request or response. 29 | */ 30 | String getRequestOrResponseType(); 31 | } 32 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/notification/BlobReplicaSourceType.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.notification; 15 | 16 | /** 17 | * The enumeration of all the sources by which a replica can be created in the system 18 | */ 19 | public enum BlobReplicaSourceType { 20 | /** 21 | * The blob replica was created by a primary write. This means that the blob 22 | * was written directly to the server 23 | */ 24 | PRIMARY, 25 | /** 26 | * The blob replica was created by a repair operation. This could be because the primary 27 | * write failed or because a replica needs to be restored from a repair 28 | */ 29 | REPAIRED 30 | } 31 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/notification/NotificationBlobType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.notification; 16 | 17 | /** 18 | * Denotes the type of blob created. i.e. whether this is a simple blob, a metadata (composite) blob, or a data chunk 19 | */ 20 | public enum NotificationBlobType { 21 | Simple, Composite, DataChunk 22 | } 23 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/notification/UpdateType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.notification; 15 | 16 | public enum UpdateType { 17 | TTL_UPDATE 18 | } 19 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/protocol/DatasetVersionState.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2023 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.protocol; 16 | 17 | public enum DatasetVersionState { 18 | IN_PROGRESS, 19 | READY, 20 | RENAMED 21 | } 22 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/protocol/GetOption.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.protocol; 15 | 16 | /** 17 | * The list of options for the Get request. 18 | */ 19 | public enum GetOption { 20 | /** 21 | * This is the default. This returns all blobs that are not expired and not deleted 22 | */ 23 | None, 24 | /** 25 | * Indicates that the blob should be returned even if it is expired 26 | */ 27 | Include_Expired_Blobs, 28 | /** 29 | * Indicates that the blob should be returned even if it is deleted 30 | */ 31 | Include_Deleted_Blobs, 32 | /** 33 | * Indicates that the blob should be returned regardless of whether it is deleted or expired 34 | */ 35 | Include_All 36 | } 37 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/protocol/NamedBlobState.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.protocol; 16 | 17 | public enum NamedBlobState { 18 | IN_PROGRESS, 19 | READY 20 | } 21 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/quota/QuotaAction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2022 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.quota; 15 | 16 | /** 17 | * Action recommended for handling quota compliance of requests. 18 | * The enum is declared in the order of severity in order to make the severity comparison easy. 19 | * In terms of severity {@link QuotaAction#ALLOW} < {@link QuotaAction#DELAY} < {@link QuotaAction#REJECT}. 20 | */ 21 | public enum QuotaAction { 22 | /** 23 | * Allow the request to go through. 24 | */ 25 | ALLOW, 26 | /** 27 | * Delay the request. 28 | */ 29 | DELAY, 30 | /** 31 | * Reject the request. 32 | */ 33 | REJECT 34 | } 35 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/quota/QuotaEnforcerFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2021 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.quota; 15 | 16 | /** 17 | * Factory to instantiate {@link QuotaEnforcer} class. 18 | */ 19 | public interface QuotaEnforcerFactory { 20 | 21 | /** 22 | * Build and return {@link QuotaEnforcer} class. 23 | * @return QuotaEnforcer object. 24 | */ 25 | QuotaEnforcer getQuotaEnforcer(); 26 | } 27 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/quota/QuotaManagerFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2021 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.quota; 15 | 16 | /** 17 | * Factory to instantiate {@link QuotaManager}. 18 | */ 19 | public interface QuotaManagerFactory { 20 | 21 | /** 22 | * Build and return the {@link QuotaManager} object. 23 | * @return QuotaManager object. 24 | */ 25 | QuotaManager getQuotaManager(); 26 | } 27 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/quota/QuotaMethod.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2021 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.quota; 15 | 16 | /** 17 | * Denotes the type of request for quota tracking purposes. 18 | */ 19 | public enum QuotaMethod { 20 | /** 21 | * This denotes read requests for quota purposes (e.g, Get, GetBlobInfo etc). 22 | */ 23 | READ, 24 | /** 25 | * This denotes write requests for quota purposes (e.g, PUT, POST, DELETE, TTLUPDATE etc). 26 | */ 27 | WRITE 28 | } 29 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/quota/QuotaMode.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.quota; 15 | 16 | /** 17 | * Quota service mode. If mode is tracking, it will not throttle traffics even if the quota is exceeded. 18 | */ 19 | public enum QuotaMode { 20 | TRACKING, THROTTLING 21 | } 22 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/quota/QuotaName.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2021 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.quota; 15 | 16 | /** 17 | * Enum representing various types of quota supported for user requests. It would be used by {@link QuotaSource} to get 18 | * the appropriate quota values. 19 | */ 20 | public enum QuotaName { 21 | READ_CAPACITY_UNIT, WRITE_CAPACITY_UNIT, STORAGE_IN_GB 22 | } 23 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/quota/QuotaResourceType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2021 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.quota; 15 | 16 | /** 17 | * Type of Ambry resource for which quota can be applied. 18 | */ 19 | public enum QuotaResourceType { 20 | ACCOUNT, CONTAINER, SERVICE 21 | } -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/quota/QuotaSourceFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2021 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.quota; 15 | 16 | /** 17 | * Factory to instantiate {@link QuotaSource} object. 18 | */ 19 | public interface QuotaSourceFactory { 20 | /** 21 | * @return the {@link QuotaSource} object. 22 | */ 23 | QuotaSource getQuotaSource(); 24 | } 25 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/quota/QuotaUsageLevel.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2021 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.quota; 15 | 16 | /** 17 | * Quota usage levels. 18 | * The limit for each type of level should be determined by {@link QuotaRecommendationMergePolicy} implementation. 19 | * NOTE that the order of these enums should not be changed since their relative position is used for comparison. 20 | */ 21 | public enum QuotaUsageLevel { 22 | 23 | /** The quota usage is within healthy usage limits. */ 24 | HEALTHY, 25 | 26 | /** The quota usage is approaching limit. */ 27 | WARNING, 28 | 29 | /** Usage at this level is fast approaching the limit at which requests will be throttled. */ 30 | CRITICAL, 31 | 32 | /** The quota usage is at or above limit and will be throttled */ 33 | EXCEEDED 34 | } 35 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/repair/RepairRequestsDbFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2023 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * 14 | */ 15 | 16 | package com.github.ambry.repair; 17 | 18 | /** 19 | * A factory to build instances of {@link RepairRequestsDb}. 20 | */ 21 | public interface RepairRequestsDbFactory { 22 | /** 23 | * @return an instance of {@link RepairRequestsDb}. 24 | * @throws Exception if there is an error during instantiation. 25 | */ 26 | RepairRequestsDb getRepairRequestsDb() throws Exception; 27 | } 28 | 29 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/replication/FindTokenFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.replication; 15 | 16 | import java.io.DataInputStream; 17 | import java.io.IOException; 18 | 19 | 20 | /** 21 | * A factory interface to create the FindToken 22 | */ 23 | public interface FindTokenFactory { 24 | /** 25 | * The find token created using the input stream provided 26 | * @param stream The stream that is used to create the find token 27 | * @return The find token created from the stream 28 | */ 29 | FindToken getFindToken(DataInputStream stream) throws IOException; 30 | 31 | /** 32 | * Provides a new token to bootstrap the find operation 33 | * @return A new find token that helps to bootstrap the find operation 34 | */ 35 | FindToken getNewFindToken(); 36 | } 37 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/replication/FindTokenType.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.replication; 15 | 16 | /** 17 | * The type of replica token 18 | */ 19 | public enum FindTokenType { 20 | Uninitialized, JournalBased, IndexBased, CloudBased 21 | } 22 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/replication/ReplicationModelType.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.replication; 16 | 17 | /** 18 | * The model used for cross-colo replication 19 | * Under ALL_TO_ALL mode, all replicas of local data center replicate from all peer replicas in remote data centers 20 | * Under LEADER_BASED mode, only the leader replica of local data center (as elected by helix) will replicate with other leader replicas in remote data centers 21 | */ 22 | public enum ReplicationModelType { 23 | ALL_TO_ALL, LEADER_BASED 24 | } 25 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/rest/NioServerFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.rest; 15 | 16 | /** 17 | * NioServerFactory is a factory to generate all the supporting cast required to instantiate a {@link NioServer}. 18 | *

19 | * Usually called with the canonical class name and as such might have to support appropriate (multiple) constructors. 20 | */ 21 | public interface NioServerFactory { 22 | 23 | /** 24 | * Returns an instance of the {@link NioServer} that the factory generates. 25 | * @return an instance of {@link NioServer} generated by this factory. 26 | * @throws InstantiationException if the {@link NioServer} instance cannot be created. 27 | */ 28 | public NioServer getNioServer() throws InstantiationException; 29 | } 30 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/rest/RestMethod.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.rest; 15 | 16 | /** 17 | * Generic equivalents of HTTP methods truncated to ones that Ambry supports. 18 | */ 19 | public enum RestMethod { 20 | DELETE, HEAD, GET, POST, PUT, UNKNOWN, OPTIONS 21 | } 22 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/router/CryptoServiceFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.router; 15 | 16 | import java.security.GeneralSecurityException; 17 | 18 | 19 | /** 20 | * CryptoService Factory to assist in fetching an instance of {@link CryptoService} which is capable of encrypting or decrypting 21 | * bytes for a given key 22 | * Ensure that {@link KeyManagementService} implementation is compatible with the same key type. 23 | */ 24 | public interface CryptoServiceFactory { 25 | 26 | /** 27 | * Instantiates and returns the {@link CryptoService} 28 | * @return the {@link CryptoService} instantiated 29 | * @throws GeneralSecurityException on any exception during instantiation 30 | */ 31 | CryptoService getCryptoService() throws GeneralSecurityException; 32 | } 33 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/router/OperationTrackerScope.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.router; 15 | 16 | /** 17 | * The metric scope adopted by operation tracker. For now, the scope defines at which granularity the latency distribution 18 | * should be tracked by operation tracker (i.e. Datacenter means latencies of all requests in local datacenter would be 19 | * kept in a single Histogram) 20 | */ 21 | public enum OperationTrackerScope { 22 | Datacenter, Partition, DataNode, Disk 23 | } 24 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/router/RouterFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.router; 15 | 16 | /** 17 | * RouterFactory is a factory to generate all the supporting cast required to instantiate a {@link Router}. 18 | *

19 | * Usually called with the canonical class name and as such might have to support appropriate (multiple) constructors. 20 | */ 21 | public interface RouterFactory { 22 | /** 23 | * Returns an instance of the {@link Router} that the factory generates. 24 | * @return an instance of {@link Router} generated by this factory. 25 | * @throws InstantiationException if the {@link Router} instance cannot be created. 26 | */ 27 | public Router getRouter() throws InstantiationException; 28 | } 29 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/server/AmbryStatsReport.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2017 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.server; 16 | 17 | /** 18 | * Health report for reporting various stats within a node. 19 | */ 20 | public interface AmbryStatsReport { 21 | /** 22 | * Get the name of the stats report. 23 | * @return the name of the stats report 24 | */ 25 | String getReportName(); 26 | 27 | /** 28 | * Get the type of stats report . The stats report type is defined in {@link StatsReportType} 29 | * @return the type of stats 30 | */ 31 | StatsReportType getStatsReportType(); 32 | 33 | /** 34 | * Get the cluster wide aggregate period for the stats report in minutes. 35 | * @return the cluster wide aggregate period in minutes 36 | */ 37 | long getAggregateIntervalInMinutes(); 38 | } 39 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/server/StatsReportType.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2018 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.server; 16 | 17 | /** 18 | * Type of stats reports which are used by StatsManager and would be transmitted to Helix. 19 | */ 20 | public enum StatsReportType { 21 | PARTITION_CLASS_REPORT, ACCOUNT_REPORT 22 | } 23 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/store/BlobMatchStatus.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.store; 15 | 16 | /** 17 | * Status code blob-state match operation 18 | */ 19 | public enum BlobMatchStatus { 20 | BLOB_INTACT_IN_AZURE, 21 | BLOB_CORRUPT_IN_AZURE, 22 | BLOB_STATE_MATCH, 23 | BLOB_ABSENT, 24 | BLOB_ABSENT_IN_AZURE, 25 | BLOB_ABSENT_IN_SERVER, 26 | BLOB_STATE_CLASS_MISMATCH, 27 | BLOB_STATE_KEY_MISMATCH, 28 | BLOB_STATE_SIZE_MISMATCH, 29 | BLOB_STATE_EXPIRY_MISMATCH, 30 | BLOB_STATE_OBSOLETE_MISMATCH, 31 | BLOB_STATE_CRC_MISMATCH, 32 | BLOB_STATE_VERSION_MISMATCH 33 | } 34 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/store/MessageErrorInfo.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.store; 15 | 16 | public class MessageErrorInfo { 17 | private final MessageInfo messageInfo; 18 | private final StoreErrorCodes error; 19 | 20 | public MessageErrorInfo(MessageInfo messageInfo, StoreErrorCodes error) { 21 | this.messageInfo = messageInfo; 22 | this.error = error; 23 | } 24 | 25 | public MessageInfo getMessageInfo() { 26 | return messageInfo; 27 | } 28 | 29 | public StoreErrorCodes getError() { 30 | return error; 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/store/Read.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.store; 15 | 16 | import java.io.IOException; 17 | import java.nio.ByteBuffer; 18 | 19 | 20 | /** 21 | * Provides a read interface into the underlying storage layer 22 | */ 23 | public interface Read { 24 | 25 | /** 26 | * Read from the underlying store(file) into the buffer starting at the given position in the store. Reads 27 | * exactly {@code buffer.remaining()} amount of data or throws an exception. 28 | * @param buffer The buffer into which the read needs to write to 29 | * @param position The position to start the read from 30 | * @throws IOException 31 | */ 32 | void readInto(ByteBuffer buffer, long position) throws IOException; 33 | } 34 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/store/StoreFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.store; 15 | 16 | import java.util.Properties; 17 | 18 | 19 | /** 20 | * The storage engine factory that is used to create a given type of a store 21 | */ 22 | public interface StoreFactory { 23 | 24 | /** 25 | * Returns the store after creating the store instance with the given properties 26 | * @param config The properties config that is used by the store 27 | * @return The store represented by this factory 28 | */ 29 | public Store getStore(Properties config); 30 | } -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/store/StoreGetOptions.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.store; 15 | 16 | /** 17 | * The list of options that can be used while reading messages 18 | * from the store 19 | */ 20 | public enum StoreGetOptions { 21 | /** 22 | * This option indicates that the store needs to return the message even if it is expired 23 | * as long as the message has not been physically deleted from the store. 24 | */ 25 | Store_Include_Expired, 26 | /** 27 | * This option indicates that the store needs to return the message even if it has been 28 | * marked for deletion as long as the message has not been physically deleted from the 29 | * store. 30 | */ 31 | Store_Include_Deleted 32 | } 33 | -------------------------------------------------------------------------------- /ambry-api/src/main/java/com/github/ambry/store/StoreInfo.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.store; 15 | 16 | import java.util.List; 17 | 18 | 19 | /** 20 | * The info returned by the store on a get call 21 | */ 22 | public class StoreInfo { 23 | private final MessageReadSet readSet; 24 | private final List messageSetInfos; 25 | 26 | public StoreInfo(MessageReadSet readSet, List messageSetInfos) { 27 | this.readSet = readSet; 28 | this.messageSetInfos = messageSetInfos; 29 | } 30 | 31 | public MessageReadSet getMessageReadSet() { 32 | return readSet; 33 | } 34 | 35 | public List getMessageReadSetInfo() { 36 | return messageSetInfos; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /ambry-api/src/test/java/com/github/ambry/compression/CompressionExceptionTest.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.compression; 15 | 16 | import org.junit.Assert; 17 | import org.junit.Test; 18 | 19 | 20 | public class CompressionExceptionTest { 21 | 22 | @Test 23 | public void testConstructor() { 24 | CompressionException ex = new CompressionException("error"); 25 | Assert.assertNotNull(ex.getMessage()); 26 | 27 | ex = new CompressionException("failed", new Exception()); 28 | Assert.assertNotNull(ex.getMessage()); 29 | Assert.assertNotNull(ex.getCause()); 30 | } 31 | } -------------------------------------------------------------------------------- /ambry-cloud/README.md: -------------------------------------------------------------------------------- 1 | # ambry-cloud 2 | 3 | This package contains classes that enable a Virtual Cloud Replica (VCR) 4 | to reflect partition-level replication changes to a remote cloud backup. 5 | Currently Azure is supported; other vendors will be added later. 6 | 7 | Components 8 | 9 | * CloudBlobStore: a Store implementation that works with a CloudDestination 10 | to reflect blob operations to a cloud store. 11 | * CloudDestination: interface for transmitting blob data and metadata between the 12 | local cluster and a remote cloud storage engine. 13 | * AzureCloudDestination: a CloudDestination implementation that stores blob data 14 | in Azure Blob Storage, and metadata in Azure Cosmos DB. 15 | -------------------------------------------------------------------------------- /ambry-cloud/src/main/java/com/github/ambry/cloud/CloudBlobCryptoAgentFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.cloud; 15 | 16 | /** 17 | * Factory for constructing {@link CloudBlobCryptoAgent} instances. 18 | */ 19 | public interface CloudBlobCryptoAgentFactory { 20 | 21 | CloudBlobCryptoAgent getCloudBlobCryptoAgent(); 22 | } 23 | -------------------------------------------------------------------------------- /ambry-cloud/src/main/java/com/github/ambry/cloud/CloudContainerCompactor.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.cloud; 15 | 16 | import com.github.ambry.clustermap.PartitionId; 17 | import java.util.Collection; 18 | import java.util.List; 19 | 20 | 21 | /** 22 | * Interface that provides methods for compacting deprecated container from assigned partitions. 23 | */ 24 | public interface CloudContainerCompactor { 25 | /** 26 | * Compact deprecated containers from partitions assigned to this vcr node. 27 | * @param assignedPartitions {@link Collection} of {@link PartitionId}s assigned to this node. 28 | */ 29 | void compactAssignedDeprecatedContainers(Collection assignedPartitions); 30 | 31 | /** 32 | * Shutdown the compactor. 33 | */ 34 | void shutdown(); 35 | } 36 | -------------------------------------------------------------------------------- /ambry-cloud/src/main/java/com/github/ambry/cloud/CloudDestinationFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.cloud; 15 | 16 | /** 17 | * A factory for creating instances of {@link CloudDestination}. 18 | */ 19 | public interface CloudDestinationFactory { 20 | 21 | /** 22 | * @return an instance of {@link CloudDestination} generated by this factory. 23 | * @throws IllegalStateException if the {@link CloudDestination} instance cannot be created. 24 | */ 25 | CloudDestination getCloudDestination() throws IllegalStateException, ReflectiveOperationException; 26 | } 27 | -------------------------------------------------------------------------------- /ambry-cloud/src/main/java/com/github/ambry/cloud/RecoveryNetworkClientCallback.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.cloud; 15 | 16 | import com.github.ambry.protocol.ReplicaMetadataRequestInfo; 17 | 18 | public interface RecoveryNetworkClientCallback { 19 | default void onListBlobs(ReplicaMetadataRequestInfo request) {} 20 | } 21 | -------------------------------------------------------------------------------- /ambry-cloud/src/main/java/com/github/ambry/cloud/RecoveryTokenFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.cloud; 15 | 16 | import com.github.ambry.replication.FindToken; 17 | import com.github.ambry.replication.FindTokenFactory; 18 | import java.io.DataInputStream; 19 | import java.io.IOException; 20 | 21 | 22 | public class RecoveryTokenFactory implements FindTokenFactory { 23 | @Override 24 | public FindToken getFindToken(DataInputStream stream) throws IOException { 25 | return RecoveryToken.fromBytes(stream); 26 | } 27 | 28 | @Override 29 | public FindToken getNewFindToken() { 30 | // Called when a RemoteReplica object is created 31 | return new RecoveryToken(); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /ambry-cloud/src/main/java/com/github/ambry/cloud/VcrStateModelFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.cloud; 15 | 16 | import org.apache.helix.participant.statemachine.StateModel; 17 | import org.apache.helix.participant.statemachine.StateModelFactory; 18 | 19 | 20 | /** 21 | * An abstract class to create {@link StateModelFactory} for vcr. 22 | */ 23 | public abstract class VcrStateModelFactory extends StateModelFactory { 24 | protected HelixVcrClusterParticipant helixVcrClusterParticipant; 25 | 26 | /** 27 | * @return Helix state model name of this state model. 28 | */ 29 | abstract String getStateModelName(); 30 | } 31 | -------------------------------------------------------------------------------- /ambry-cloud/src/main/java/com/github/ambry/cloud/azure/AzureBlobDeletePolicy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.cloud.azure; 15 | 16 | public enum AzureBlobDeletePolicy { 17 | /** 18 | * Attach a delete-timestamp to azure-blob, but do not delete it. 19 | * Let compaction job eventually delete the blob. 20 | */ 21 | EVENTUAL, 22 | /** 23 | * Just delete the blob when a DELETE message is received from server. 24 | */ 25 | IMMEDIATE 26 | } 27 | -------------------------------------------------------------------------------- /ambry-cloud/src/main/java/com/github/ambry/cloud/azure/CosmosChangeFeedFindTokenFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.cloud.azure; 15 | 16 | import com.github.ambry.replication.FindToken; 17 | import com.github.ambry.replication.FindTokenFactory; 18 | import java.io.DataInputStream; 19 | import java.io.IOException; 20 | 21 | 22 | /** 23 | * Factory that creates {@link CosmosChangeFeedFindToken} objects. 24 | */ 25 | public class CosmosChangeFeedFindTokenFactory implements FindTokenFactory { 26 | @Override 27 | public FindToken getFindToken(DataInputStream stream) throws IOException { 28 | return CosmosChangeFeedFindToken.fromBytes(stream); 29 | } 30 | 31 | @Override 32 | public FindToken getNewFindToken() { 33 | return new CosmosChangeFeedFindToken(); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /ambry-cloud/src/main/java/com/github/ambry/cloud/azure/CosmosUpdateTimeFindTokenFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.cloud.azure; 15 | 16 | import com.github.ambry.replication.FindToken; 17 | import com.github.ambry.replication.FindTokenFactory; 18 | import java.io.DataInputStream; 19 | import java.io.IOException; 20 | 21 | 22 | /** 23 | * Factory that creates {@link CosmosUpdateTimeFindTokenFactory} objects. 24 | */ 25 | public class CosmosUpdateTimeFindTokenFactory implements FindTokenFactory { 26 | 27 | @Override 28 | public FindToken getFindToken(DataInputStream stream) throws IOException { 29 | return CosmosUpdateTimeFindToken.fromBytes(stream); 30 | } 31 | 32 | @Override 33 | public FindToken getNewFindToken() { 34 | return new CosmosUpdateTimeFindToken(); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /ambry-cloud/src/test/java/com/github/ambry/cloud/DummyCloudUpdateValidator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.cloud; 15 | 16 | import com.github.ambry.store.StoreKey; 17 | import java.util.Map; 18 | 19 | 20 | /** 21 | * A no op implementation for {@link CloudUpdateValidator} 22 | */ 23 | public class DummyCloudUpdateValidator implements CloudUpdateValidator { 24 | @Override 25 | public boolean validateUpdate(CloudBlobMetadata metadata, StoreKey key, Map updateFields) { 26 | // no op 27 | return true; 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /ambry-cloud/src/test/java/com/github/ambry/cloud/TestCloudBlobCryptoAgent.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.cloud; 15 | 16 | import java.nio.ByteBuffer; 17 | import java.security.GeneralSecurityException; 18 | 19 | 20 | /** 21 | * Dummy implementation for testing. 22 | */ 23 | class TestCloudBlobCryptoAgent implements CloudBlobCryptoAgent { 24 | @Override 25 | public ByteBuffer encrypt(ByteBuffer buffer) throws GeneralSecurityException { 26 | return buffer; 27 | } 28 | 29 | @Override 30 | public ByteBuffer decrypt(ByteBuffer buffer) throws GeneralSecurityException { 31 | return buffer; 32 | } 33 | 34 | @Override 35 | public String getEncryptionContext() { 36 | return null; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /ambry-cloud/src/test/java/com/github/ambry/cloud/TestCloudBlobCryptoAgentFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.cloud; 15 | 16 | import com.codahale.metrics.MetricRegistry; 17 | import com.github.ambry.config.VerifiableProperties; 18 | 19 | 20 | /** 21 | * Factory to create TestCloudBlobCryptoAgent 22 | */ 23 | public class TestCloudBlobCryptoAgentFactory implements CloudBlobCryptoAgentFactory { 24 | private final CloudBlobCryptoAgent cryptoAgent; 25 | public TestCloudBlobCryptoAgentFactory(VerifiableProperties verifiableProperties, String clusterName, 26 | MetricRegistry metricRegistry) { 27 | this.cryptoAgent = new TestCloudBlobCryptoAgent(); 28 | } 29 | @Override 30 | public CloudBlobCryptoAgent getCloudBlobCryptoAgent() { 31 | return cryptoAgent; 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /ambry-cloud/src/test/resources/azure-test.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2019 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | azure.storage.connection.string= 13 | cosmos.endpoint= 14 | cosmos.database=ambry-metadata 15 | cosmos.collection=blob-metadata 16 | cosmos.deleted.container.collection=deleted-container-test 17 | cosmos.key= 18 | azure.storage.authority= 19 | azure.storage.clientId= 20 | azure.storage.secret= 21 | azure.storage.scope= 22 | azure.storage.endpoint= 23 | azure.identity.tenant.id 24 | azure.identity.client.id= 25 | azure.identity.secret= 26 | -------------------------------------------------------------------------------- /ambry-cloud/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker: -------------------------------------------------------------------------------- 1 | mock-maker-inline 2 | -------------------------------------------------------------------------------- /ambry-cloud/src/test/resources/replicaTokens: -------------------------------------------------------------------------------- 1 | Version: 2 2 | PartitionId: 666 3 | Hostname: omaha-23 4 | Port: 12345 5 | Replica path: /var/ambry/666/5h67xc07 6 | totalBytesRead: 45900320 7 | tokenType: JOURNAL 8 | offset: 889402 9 | PartitionId: 666 10 | Hostname: omaha-27 11 | Port: 12345 12 | Replica path: /var/ambry/666/f7334bd90 13 | totalBytesRead: 49428479 14 | tokenType: JOURNAL 15 | offset: 85300 16 | crc: 59226038 17 | -------------------------------------------------------------------------------- /ambry-clustermap/src/main/java/com/github/ambry/clustermap/DataNodeConfigChangeListener.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * 14 | */ 15 | 16 | package com.github.ambry.clustermap; 17 | 18 | /** 19 | * Interface to implement a listener that will be called on updates to {@link DataNodeConfig}s. 20 | */ 21 | public interface DataNodeConfigChangeListener { 22 | /** 23 | * Called when there is a relevant update to one or more {@link DataNodeConfig}s. 24 | * @param configs the new or updated configs. 25 | */ 26 | void onDataNodeConfigChange(Iterable configs); 27 | 28 | /** 29 | * Called when a node is deleted. The node is identified by its instanceName. 30 | * @param instanceName the deleted instance. 31 | */ 32 | void onDataNodeDelete(String instanceName); 33 | } 34 | -------------------------------------------------------------------------------- /ambry-clustermap/src/main/java/com/github/ambry/clustermap/HelixAdminFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.clustermap; 15 | 16 | import org.apache.helix.HelixAdmin; 17 | import org.apache.helix.manager.zk.ZKHelixAdmin; 18 | 19 | 20 | /** 21 | * A factory class to construct and get a reference to a {@link HelixAdmin} 22 | */ 23 | public class HelixAdminFactory { 24 | /** 25 | * Get a reference to a {@link HelixAdmin} 26 | * @param zkAddr the address identifying the zk service to which this request is to be made. 27 | * @return the reference to the {@link HelixAdmin}. 28 | */ 29 | public HelixAdmin getHelixAdmin(String zkAddr) { 30 | return new ZKHelixAdmin(zkAddr); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /ambry-clustermap/src/main/java/com/github/ambry/clustermap/NoOpResourceStatePolicy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.clustermap; 15 | 16 | public class NoOpResourceStatePolicy implements ResourceStatePolicy { 17 | @Override 18 | public boolean isHardDown() { 19 | return false; 20 | } 21 | 22 | @Override 23 | public boolean isDown() { 24 | return false; 25 | } 26 | 27 | @Override 28 | public void onError() { 29 | 30 | } 31 | 32 | @Override 33 | public void onSuccess() { 34 | 35 | } 36 | 37 | @Override 38 | public void onHardDown() { 39 | 40 | } 41 | 42 | @Override 43 | public void onHardUp() { 44 | 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /ambry-clustermap/src/test/java/com/github/ambry/clustermap/TestReplica.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.clustermap; 16 | 17 | import org.json.JSONException; 18 | import org.json.JSONObject; 19 | 20 | 21 | // Permits Replica to be constructed with a null Partition 22 | public class TestReplica extends Replica { 23 | public TestReplica(HardwareLayout hardwareLayout, JSONObject jsonObject) throws JSONException { 24 | super(hardwareLayout, null, jsonObject); 25 | } 26 | 27 | public TestReplica(TestUtils.TestHardwareLayout hardwareLayout, Disk disk) throws JSONException { 28 | super(null, disk, hardwareLayout.clusterMapConfig); 29 | } 30 | 31 | @Override 32 | public void validatePartition() { 33 | // Null OK 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /ambry-commons/src/main/java/com/github/ambry/clustermap/VcrClusterAgentsFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.clustermap; 15 | 16 | import com.github.ambry.config.CloudConfig; 17 | import com.github.ambry.config.ClusterMapConfig; 18 | 19 | 20 | /** 21 | * A factory interface to get {@link VcrClusterParticipant} and {@link VcrClusterSpectator}. 22 | */ 23 | public interface VcrClusterAgentsFactory { 24 | 25 | /** 26 | * @return an instance of {@link VcrClusterParticipant} generated by the factory. 27 | */ 28 | VcrClusterParticipant getVcrClusterParticipant() throws Exception; 29 | 30 | /** 31 | * @return an instance of {@link VcrClusterSpectator} generated by the factory. 32 | */ 33 | VcrClusterSpectator getVcrClusterSpectator(CloudConfig cloudConfig, ClusterMapConfig clusterMapConfig); 34 | } 35 | -------------------------------------------------------------------------------- /ambry-commons/src/main/java/com/github/ambry/clustermap/VcrClusterParticipantListener.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | * Copyright 2019 LinkedIn Corp. All rights reserved. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * 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 | */ 15 | package com.github.ambry.clustermap; 16 | 17 | /** 18 | * {@link VcrClusterParticipantListener} takes actions on {@link VcrClusterParticipant} partition add or removal. 19 | */ 20 | public interface VcrClusterParticipantListener { 21 | 22 | /** 23 | * Action to take when new Partition is added. 24 | * @param partitionId on add. 25 | */ 26 | void onPartitionAdded(PartitionId partitionId); 27 | 28 | /** 29 | * Action to take when new Partition is removed. 30 | * @param partitionId on remove. 31 | */ 32 | void onPartitionRemoved(PartitionId partitionId); 33 | } 34 | -------------------------------------------------------------------------------- /ambry-commons/src/main/java/com/github/ambry/commons/AmbryCacheEntry.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.commons; 16 | 17 | /** 18 | * Implements a generic cache entry for the AmbryCache 19 | */ 20 | public interface AmbryCacheEntry { 21 | 22 | /** 23 | * @return size of AmbryCacheEntry in bytes 24 | */ 25 | default int sizeBytes() { return 0; } 26 | } 27 | -------------------------------------------------------------------------------- /ambry-commons/src/main/java/com/github/ambry/commons/HardwareResource.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.commons; 15 | 16 | /** 17 | * Hardware Resource that Ambry needs to monitor for throttling 18 | */ 19 | public enum HardwareResource { 20 | CPU, HEAP_MEMORY, DIRECT_MEMORY 21 | } 22 | -------------------------------------------------------------------------------- /ambry-commons/src/main/java/com/github/ambry/store/StoreKeyConverterFactoryImpl.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2018 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.store; 16 | 17 | import com.codahale.metrics.MetricRegistry; 18 | import com.github.ambry.config.VerifiableProperties; 19 | 20 | 21 | /** 22 | * Default StoreKeyConverterFactoryImpl. Creates StoreKeyConverterImplNoOp 23 | */ 24 | public class StoreKeyConverterFactoryImpl implements StoreKeyConverterFactory { 25 | 26 | public StoreKeyConverterFactoryImpl(VerifiableProperties verifiableProperties, MetricRegistry metricRegistry) { 27 | 28 | } 29 | 30 | @Override 31 | public StoreKeyConverter getStoreKeyConverter() throws InstantiationException { 32 | return new StoreKeyConverterImplNoOp(); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /ambry-commons/src/test/java/com/github/ambry/commons/NettySslFactoryTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.commons; 16 | 17 | import com.github.ambry.commons.NettySslFactory; 18 | import com.github.ambry.commons.TestSSLUtils; 19 | import org.junit.Test; 20 | 21 | 22 | /** 23 | * Test {@link NettySslFactory} 24 | */ 25 | public class NettySslFactoryTest { 26 | 27 | /** 28 | * Run sanity checks for {@link NettySslFactory}. 29 | * @throws Exception 30 | */ 31 | @Test 32 | public void testSSLFactory() throws Exception { 33 | TestSSLUtils.testSSLFactoryImpl(NettySslFactory.class.getName()); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /ambry-file-transfer/src/main/java/com/github/ambry/filetransfer/FileCopyBasedReplicationScheduler.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2025 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.filetransfer; 16 | 17 | /** 18 | * Interface for FileCopy Based Replication Scheduler.This scheduler 19 | * schedules the replication of files from one node to another 20 | * based on the Priority of the Partition. 21 | */ 22 | public interface FileCopyBasedReplicationScheduler extends Runnable{ 23 | 24 | /** 25 | * Shutdown the scheduler. 26 | */ 27 | void shutdown() throws InterruptedException; 28 | 29 | /** 30 | * Schedule the replication of Partitions. 31 | */ 32 | void scheduleFileCopy() throws InterruptedException; 33 | 34 | /** 35 | * @return the number thread pool size. 36 | */ 37 | int getThreadPoolSize(); 38 | 39 | } -------------------------------------------------------------------------------- /ambry-file-transfer/src/main/java/com/github/ambry/filetransfer/FileCopyBasedReplicationSchedulerFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2025 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.filetransfer; 16 | 17 | /** 18 | * Interface for Factory class which returns the {@link FileCopyBasedReplicationScheduler} depending on the implementation 19 | */ 20 | public interface FileCopyBasedReplicationSchedulerFactory { 21 | /** 22 | * @return returns the {@link FileCopyBasedReplicationScheduler} 23 | */ 24 | FileCopyBasedReplicationScheduler getFileCopyBasedReplicationScheduler(); 25 | } -------------------------------------------------------------------------------- /ambry-file-transfer/src/main/java/com/github/ambry/filetransfer/handler/FileCopyHandlerFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2025 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.filetransfer.handler; 15 | 16 | import com.github.ambry.filetransfer.handler.FileCopyHandler; 17 | 18 | 19 | /** 20 | * Interface for Factory class which returns the {@link FileCopyHandler} depending on the implementation 21 | */ 22 | public interface FileCopyHandlerFactory { 23 | 24 | /** 25 | * @return returns the {@link FileCopyHandler} 26 | */ 27 | FileCopyHandler getFileCopyHandler(); 28 | } 29 | -------------------------------------------------------------------------------- /ambry-filesystem/src/main/java/com/github/ambry/filesystem/package-info.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2023 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.filesystem; 15 | -------------------------------------------------------------------------------- /ambry-frontend/src/integration-test/resources/namedblob_mysql.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2022 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | mysql.named.blob.db.info=[{"url":"jdbc:mysql://localhost/AmbryNamedBlobs?serverTimezone=UTC","datacenter":"localDc","isWriteable":"true","username":"travis","password":""}] -------------------------------------------------------------------------------- /ambry-frontend/src/main/java/com/github/ambry/frontend/AmbryFrontendMain.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.frontend; 15 | 16 | import com.github.ambry.rest.RestServerMain; 17 | 18 | 19 | /** 20 | * Used for starting/stopping an instance of {@link com.github.ambry.rest.RestServer} that acts as an Ambry frontend. 21 | */ 22 | public class AmbryFrontendMain { 23 | 24 | public static void main(String[] args) { 25 | RestServerMain.main(args); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /ambry-frontend/src/test/java/com/github/ambry/frontend/TestAmbryFrontendMain.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.frontend; 15 | 16 | /** 17 | * Used for starting/stopping an instance of {@link com.github.ambry.rest.RestServer} that acts as an Ambry frontend. 18 | * This can use InMemoryRouter and other testing classes if required. 19 | */ 20 | public class TestAmbryFrontendMain { 21 | 22 | public static void main(String[] args) { 23 | AmbryFrontendMain.main(args); 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /ambry-messageformat/src/main/java/com/github/ambry/messageformat/DeleteSubRecord.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2017 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.messageformat; 15 | 16 | /** 17 | * Contains the delete sub-record info 18 | */ 19 | public class DeleteSubRecord implements SubRecord { 20 | 21 | @Override 22 | public Type getType() { 23 | return SubRecord.Type.DELETE; 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /ambry-messageformat/src/main/java/com/github/ambry/messageformat/MessageFormatErrorCodes.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.messageformat; 15 | 16 | /** 17 | * The error codes that the message format package returns 18 | */ 19 | //@formatter:off 20 | public enum MessageFormatErrorCodes { 21 | DataCorrupt, 22 | HeaderConstraintError, 23 | UnknownFormatVersion, 24 | StoreKeyIdMisMatch, 25 | IOError 26 | } 27 | //@formatter:on 28 | -------------------------------------------------------------------------------- /ambry-messageformat/src/main/java/com/github/ambry/messageformat/MessageFormatException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.messageformat; 15 | 16 | public class MessageFormatException extends Exception { 17 | private final MessageFormatErrorCodes error; 18 | 19 | public MessageFormatException(String message, MessageFormatErrorCodes error) { 20 | super(message); 21 | this.error = error; 22 | } 23 | 24 | public MessageFormatException(String message, Throwable e, MessageFormatErrorCodes error) { 25 | super(message, e); 26 | this.error = error; 27 | } 28 | 29 | public MessageFormatException(Throwable e, MessageFormatErrorCodes error) { 30 | super(e); 31 | this.error = error; 32 | } 33 | 34 | public MessageFormatErrorCodes getErrorCode() { 35 | return this.error; 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /ambry-messageformat/src/main/java/com/github/ambry/messageformat/SubRecord.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.messageformat; 15 | 16 | /** 17 | * Abstract class for {@link UpdateRecord} SubRecords 18 | */ 19 | public interface SubRecord { 20 | 21 | /** 22 | * @return the type of SubRecord 23 | */ 24 | Type getType(); 25 | 26 | enum Type { 27 | DELETE, TTL_UPDATE, UNDELETE 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /ambry-messageformat/src/main/java/com/github/ambry/messageformat/TtlUpdateSubRecord.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.messageformat; 15 | 16 | /** 17 | * In mem representation of a TTL update record 18 | */ 19 | public class TtlUpdateSubRecord implements SubRecord { 20 | private final long updatedExpiryTimeMs; 21 | 22 | /** 23 | * @param updatedExpiryTimeMs the new expiry time in ms 24 | */ 25 | TtlUpdateSubRecord(long updatedExpiryTimeMs) { 26 | this.updatedExpiryTimeMs = updatedExpiryTimeMs; 27 | } 28 | 29 | /** 30 | * @return the expiry time in ms 31 | */ 32 | public long getUpdatedExpiryTimeMs() { 33 | return updatedExpiryTimeMs; 34 | } 35 | 36 | @Override 37 | public Type getType() { 38 | return SubRecord.Type.TTL_UPDATE; 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /ambry-messageformat/src/main/java/com/github/ambry/messageformat/UndeleteSubRecord.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.messageformat; 15 | 16 | public class UndeleteSubRecord implements SubRecord { 17 | 18 | @Override 19 | public Type getType() { 20 | return SubRecord.Type.UNDELETE; 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /ambry-mysql/src/main/java/com/github/ambry/accountstats/AggregatedContainerStorageStatsFunction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.accountstats; 15 | 16 | import com.github.ambry.server.storagestats.ContainerStorageStats; 17 | 18 | 19 | /** 20 | * A callback function to call when processing aggregated container storage stats. 21 | */ 22 | @FunctionalInterface 23 | public interface AggregatedContainerStorageStatsFunction { 24 | /** 25 | * Process container storage usage. 26 | * @param accountId The account id. 27 | * @param containerStats The {@link ContainerStorageStats} 28 | */ 29 | void apply(short accountId, ContainerStorageStats containerStats); 30 | } 31 | -------------------------------------------------------------------------------- /ambry-mysql/src/main/java/com/github/ambry/accountstats/ContainerStorageStatsFunction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.accountstats; 15 | 16 | import com.github.ambry.server.storagestats.ContainerStorageStats; 17 | 18 | 19 | /** 20 | * A callback function to call when processing container storage stats. 21 | */ 22 | @FunctionalInterface 23 | public interface ContainerStorageStatsFunction { 24 | 25 | /** 26 | * Process container storage stats. 27 | * @param partitionId The partition id. 28 | * @param accountId The account id. 29 | * @param containerStats The {@link ContainerStorageStats} 30 | * @param updatedAtMs The timestamp in milliseconds when this data is updated. 31 | */ 32 | void apply(int partitionId, short accountId, ContainerStorageStats containerStats, long updatedAtMs); 33 | } 34 | -------------------------------------------------------------------------------- /ambry-mysql/src/test/resources/accountstats_mysql.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2020 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | account.stats.mysql.db.info=[{"url":"jdbc:mysql://localhost/ambry_container_storage_stats?serverTimezone=UTC","datacenter":"dc1","isWriteable":"true","username":"travis","password":""}] 13 | -------------------------------------------------------------------------------- /ambry-mysql/src/test/resources/repairRequests_mysql.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2023 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | mysql.repair.requests.db.info=[{"url":"jdbc:mysql://localhost/AmbryRepairRequests?serverTimezone=UTC","datacenter":"DC1","isWriteable":"true","username":"travis","password":""}] 13 | -------------------------------------------------------------------------------- /ambry-named-mysql/src/main/resources/NamedBlobsSchema.ddl: -------------------------------------------------------------------------------- 1 | /* New Table */ 2 | CREATE TABLE IF NOT EXISTS named_blobs_v2 ( 3 | account_id int NOT NULL, 4 | container_id int NOT NULL, 5 | blob_name varchar(350) NOT NULL, 6 | version bigint NOT NULL, 7 | blob_id varbinary(50) NOT NULL, 8 | blob_state smallint NOT NULL, 9 | deleted_ts datetime(6) DEFAULT NULL, 10 | blob_size bigint unsigned DEFAULT 0, 11 | modified_ts datetime(6) DEFAULT CURRENT_TIMESTAMP(6), 12 | PRIMARY KEY (account_id, container_id, blob_name, version) 13 | ) 14 | 15 | ENGINE=InnoDB 16 | DEFAULT CHARSET=utf8mb4 17 | COLLATE=utf8mb4_bin 18 | COMMENT='Holds mappings between blob names and blob IDs'; 19 | 20 | /* Soft Delete Index */ 21 | CREATE INDEX named_blobs_dt_v2 ON named_blobs_v2(deleted_ts); 22 | 23 | /* Reverse Lookup Index */ 24 | CREATE INDEX named_blobs_id_v2 ON named_blobs_v2(blob_id); 25 | 26 | /* Blob Name Index */ 27 | CREATE INDEX named_blobs_name_v2 ON named_blobs_v2(blob_name); 28 | -------------------------------------------------------------------------------- /ambry-named-mysql/src/test/resources/mysql.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2020 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | mysql.named.blob.db.info=[{"url":"jdbc:mysql://localhost/AmbryNamedBlobs?serverTimezone=UTC","datacenter":"dc1","isWriteable":"true","username":"travis","password":""}] 13 | -------------------------------------------------------------------------------- /ambry-network/src/test/java/com/github/ambry/network/BoundedByteBufferSendTest.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.network; 15 | 16 | // TODO 17 | public class BoundedByteBufferSendTest { 18 | } 19 | -------------------------------------------------------------------------------- /ambry-prioritization/src/main/java/com/github/ambry/replica/prioritization/FileBasedReplicationPrioritizationManagerFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2025 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.replica.prioritization; 16 | 17 | import com.github.ambry.config.ReplicaPrioritizationStrategy; 18 | 19 | 20 | public class FileBasedReplicationPrioritizationManagerFactory implements PrioritizationManagerFactory{ 21 | 22 | @Override 23 | public PrioritizationManager getPrioritizationManager(ReplicaPrioritizationStrategy replicaPrioritizationStrategy) { 24 | if(replicaPrioritizationStrategy == ReplicaPrioritizationStrategy.FirstComeFirstServe) { 25 | return new FCFSPrioritizationManager(); 26 | } 27 | else 28 | return null; 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /ambry-prioritization/src/main/java/com/github/ambry/replica/prioritization/PrioritizationManagerFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2025 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.replica.prioritization; 16 | 17 | import com.github.ambry.config.ReplicaPrioritizationStrategy; 18 | 19 | /** 20 | * Interface for Factory class which returns the {@link PrioritizationManager} depending on the implementation 21 | */ 22 | public interface PrioritizationManagerFactory { 23 | /** 24 | * @return returns the {@link PrioritizationManager} 25 | */ 26 | PrioritizationManager getPrioritizationManager(ReplicaPrioritizationStrategy replicaPrioritizationStrategy); 27 | } 28 | -------------------------------------------------------------------------------- /ambry-prioritization/src/main/java/com/github/ambry/replica/prioritization/PriorityTier.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2025 LinkedIn Corp. All rights reserved. 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * Unless required by applicable law or agreed to in writing, software 8 | * distributed under the License is distributed on an "AS IS" BASIS, 9 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 10 | */ 11 | package com.github.ambry.replica.prioritization; 12 | 13 | /** 14 | * Enum representing the priority tiers for replica prioritization. 15 | * The order of the enum values indicates their priority level, with the first value being the highest priority. 16 | */ 17 | public enum PriorityTier { 18 | BELOW_MIN_REPLICA_WITH_DISRUPTION, // Highest priority 19 | BELOW_MIN_REPLICA_NO_DISRUPTION, // Second priority 20 | MIN_REPLICA_WITH_DISRUPTION, // Third priority 21 | MIN_REPLICA_NO_DISRUPTION, // Fourth priority 22 | NORMAL // Lowest priority (filler) 23 | } 24 | -------------------------------------------------------------------------------- /ambry-prioritization/src/main/java/com/github/ambry/replica/prioritization/disruption/Operation.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2025 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.replica.prioritization.disruption; 16 | 17 | 18 | /** 19 | * The Operation interface defines the contract for operations in the system. 20 | * Implementing classes should provide the start time of the operation. 21 | */ 22 | public interface Operation extends Comparable { 23 | /** 24 | * Gets the start time of the operation. 25 | * 26 | * @return the start time of the operation in milliseconds. 27 | */ 28 | long getStartTime(); 29 | } 30 | -------------------------------------------------------------------------------- /ambry-prioritization/src/main/java/com/github/ambry/replica/prioritization/disruption/factory/DisruptionServiceFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2025 LinkedIn Corp. All rights reserved. 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * http://www.apache.org/licenses/LICENSE-2.0 7 | * Unless required by applicable law or agreed to in writing, software 8 | * distributed under the License is distributed on an "AS IS" BASIS, 9 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 10 | */ 11 | package com.github.ambry.replica.prioritization.disruption.factory; 12 | 13 | import com.github.ambry.replica.prioritization.disruption.DisruptionService; 14 | 15 | 16 | /** 17 | * Factory interface for creating instances of {@link DisruptionService}. 18 | * This interface allows for the creation of different implementations of 19 | * {@link DisruptionService} based on the provided configuration. 20 | */ 21 | public interface DisruptionServiceFactory { 22 | 23 | /** 24 | * @return an instance of {@link DisruptionService}. 25 | */ 26 | DisruptionService getDisruptionService(); 27 | } 28 | -------------------------------------------------------------------------------- /ambry-protocol/src/main/java/com/github/ambry/protocol/AdminRequestOrResponseType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.protocol; 15 | 16 | /** 17 | * Enum of types of administration requests/responses. 18 | *

19 | * The order of these enums should not be changed since their relative position goes into the serialized form of 20 | * requests/responses 21 | */ 22 | public enum AdminRequestOrResponseType { 23 | TriggerCompaction, RequestControl, ReplicationControl, CatchupStatus, BlobStoreControl, HealthCheck, BlobIndex, ForceDelete 24 | } 25 | -------------------------------------------------------------------------------- /ambry-protocol/src/main/java/com/github/ambry/protocol/BlobStoreControlAction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.protocol; 15 | 16 | /** 17 | * Enum of actions of BlobStore control. 18 | * The order of these enums should not be changed since their relative position goes into the serialized form of 19 | * requests. 20 | */ 21 | public enum BlobStoreControlAction { 22 | StopStore, StartStore, AddStore, RemoveStore 23 | } 24 | -------------------------------------------------------------------------------- /ambry-protocol/src/main/java/com/github/ambry/protocol/RequestTracker.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2022 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.protocol; 15 | 16 | /** 17 | * Used for tracking any operations that needs to be done on the request. 18 | */ 19 | public interface RequestTracker { 20 | /** 21 | * @param visitor object that visits the implementation of this object. 22 | */ 23 | void accept(RequestVisitor visitor); 24 | } 25 | -------------------------------------------------------------------------------- /ambry-quota/src/test/resources/mysql.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2020 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | account.stats.mysql.db.info=[{"url":"jdbc:mysql://localhost/ambry_container_storage_stats?serverTimezone=UTC","datacenter":"dc1","isWriteable":"true","username":"travis","password":""}] 13 | -------------------------------------------------------------------------------- /ambry-replication/src/main/java/com/github/ambry/replication/continuous/StandByGroupTracker.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.replication.continuous; 15 | 16 | /** 17 | * This class tracks for a current state for StandBy groups for a continuous replication cycle. 18 | * Standby group has replicas with STANDBY_TIMED_OUT state only 19 | */ 20 | public class StandByGroupTracker extends GroupTracker { 21 | 22 | public StandByGroupTracker(int groupId) { 23 | super(groupId); 24 | } 25 | @Override 26 | public String toString() { 27 | return "StandByGroupTracker :[" + super.toString() + "]"; 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /ambry-replication/src/test/java/com/github/ambry/replication/MockFindTokenFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.replication; 16 | 17 | import java.io.DataInputStream; 18 | import java.io.IOException; 19 | 20 | 21 | public class MockFindTokenFactory implements FindTokenFactory { 22 | @Override 23 | public FindToken getFindToken(DataInputStream stream) throws IOException { 24 | return new MockFindToken(stream); 25 | } 26 | 27 | @Override 28 | public FindToken getNewFindToken() { 29 | return new MockFindToken(0, 0); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /ambry-replication/src/test/java/com/github/ambry/replication/MockFindTokenHelper.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.replication; 15 | 16 | import com.github.ambry.clustermap.ReplicaType; 17 | import com.github.ambry.config.ReplicationConfig; 18 | import com.github.ambry.store.StoreKeyFactory; 19 | 20 | 21 | public class MockFindTokenHelper extends FindTokenHelper { 22 | 23 | public MockFindTokenHelper(StoreKeyFactory storeKeyFactory, ReplicationConfig replicationConfig) 24 | throws ReflectiveOperationException { 25 | } 26 | 27 | public MockFindTokenHelper() { 28 | } 29 | 30 | @Override 31 | public FindTokenFactory getFindTokenFactoryFromReplicaType(ReplicaType replicaType) { 32 | return new MockFindTokenFactory(); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /ambry-rest/src/main/java/com/github/ambry/rest/RestServerState.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.rest; 15 | 16 | /** 17 | * Service that maintains the health of the server 18 | */ 19 | public class RestServerState { 20 | private volatile boolean serviceUp = false; 21 | private final String healthCheckUri; 22 | 23 | public RestServerState(String healthCheckUri) { 24 | this.healthCheckUri = healthCheckUri; 25 | } 26 | 27 | public String getHealthCheckUri() { 28 | return healthCheckUri; 29 | } 30 | 31 | public void markServiceUp() { 32 | serviceUp = true; 33 | } 34 | 35 | public boolean isServiceUp() { 36 | return serviceUp; 37 | } 38 | 39 | public void markServiceDown() { 40 | serviceUp = false; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /ambry-rest/src/main/java/com/github/ambry/rest/ServerSecurityServiceFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.rest; 16 | 17 | public interface ServerSecurityServiceFactory { 18 | /** 19 | * Returns an instance of the {@link ServerSecurityService} that the factory generates. 20 | * @return an instance of {@link ServerSecurityService} generated by this factory. 21 | * @throws InstantiationException if the {@link ServerSecurityService} instance cannot be created. 22 | */ 23 | public ServerSecurityService getServerSecurityService() throws InstantiationException; 24 | } 25 | -------------------------------------------------------------------------------- /ambry-router/src/main/java/com/github/ambry/router/CryptoJob.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.router; 15 | 16 | import java.security.GeneralSecurityException; 17 | 18 | 19 | /** 20 | * CryptoJob representing the job that needs processing by {@link CryptoJobHandler} 21 | */ 22 | public interface CryptoJob extends Runnable { 23 | /** 24 | * Close the job with the given {@code gse} 25 | * @param gse the {@link GeneralSecurityException} that needs to be set while invoking callback for the job 26 | */ 27 | void closeJob(GeneralSecurityException gse); 28 | } 29 | -------------------------------------------------------------------------------- /ambry-router/src/main/java/com/github/ambry/router/RouterOperation.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.router; 15 | 16 | public enum RouterOperation { 17 | GetBlobOperation, 18 | GetBlobInfoOperation, 19 | PutOperation, 20 | DeleteOperation, 21 | TtlUpdateOperation, 22 | UndeleteOperation, 23 | ReplicateBlobOperation 24 | } 25 | -------------------------------------------------------------------------------- /ambry-router/src/test/java/com/github/ambry/router/CompressionMetricsTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2022 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.router; 15 | 16 | import com.codahale.metrics.MetricRegistry; 17 | import org.junit.Assert; 18 | import org.junit.Test; 19 | 20 | public class CompressionMetricsTest { 21 | 22 | @Test 23 | public void testConstructor() { 24 | CompressionMetrics metrics = new CompressionMetrics(new MetricRegistry()); 25 | metrics.decompressSuccessRate.mark(); 26 | Assert.assertEquals(1, metrics.decompressSuccessRate.getCount()); 27 | 28 | CompressionMetrics.AlgorithmMetrics algorithmMetrics = metrics.getAlgorithmMetrics("ABC"); 29 | algorithmMetrics.compressRate.mark(); 30 | Assert.assertEquals(1, algorithmMetrics.compressRate.getCount()); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /ambry-server/src/main/java/com/github/ambry/server/ServerHealthStatus.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2022 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.server; 15 | 16 | /** 17 | * Server Health Status Levels 18 | * Each level determines the health of the server. This can be extended to include bootstrapping 19 | */ 20 | public enum ServerHealthStatus { 21 | /** 22 | * The server is healthy based on the replica state's and disk state 23 | */ 24 | GOOD, 25 | /** 26 | * There was at least one issue with either the replica state or disk state on this server 27 | */ 28 | BAD, 29 | } 30 | 31 | -------------------------------------------------------------------------------- /ambry-server/src/test/resources/accountstats_mysql.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2021 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | account.stats.mysql.db.info=[{"url":"jdbc:mysql://localhost/ambry_container_storage_stats?serverTimezone=UTC","datacenter":"DC1","isWriteable":"true","username":"travis","password":""}] 13 | -------------------------------------------------------------------------------- /ambry-server/src/test/resources/repairRequests_mysql.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2023 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | mysql.repair.requests.db.info=[{"url":"jdbc:mysql://localhost/AmbryRepairRequests?serverTimezone=UTC","datacenter":"DC1","isWriteable":"true","username":"travis","password":""}] 13 | -------------------------------------------------------------------------------- /ambry-store/src/main/java/com/github/ambry/store/CompactionPolicyFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2017 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.store; 15 | 16 | /** 17 | * Factory class interface to get compaction policy 18 | */ 19 | interface CompactionPolicyFactory { 20 | 21 | /** 22 | * @return the {@link CompactionPolicy} to use 23 | */ 24 | CompactionPolicy getCompactionPolicy(); 25 | } 26 | -------------------------------------------------------------------------------- /ambry-store/src/main/java/com/github/ambry/store/HybridCompactionPolicyFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2020 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.store; 15 | 16 | import com.github.ambry.config.StoreConfig; 17 | import com.github.ambry.utils.Time; 18 | 19 | public class HybridCompactionPolicyFactory implements CompactionPolicyFactory { 20 | private final StoreConfig storeConfig; 21 | private final Time time; 22 | 23 | public HybridCompactionPolicyFactory(StoreConfig storeConfig, Time time) { 24 | this.storeConfig = storeConfig; 25 | this.time = time; 26 | } 27 | 28 | public CompactionPolicy getCompactionPolicy() { 29 | return new HybridCompactionPolicy(storeConfig, time); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /ambry-store/src/main/java/com/github/ambry/store/LogSegmentSizeProvider.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.store; 15 | 16 | /** 17 | * Provide data size for log segments. 18 | */ 19 | public interface LogSegmentSizeProvider { 20 | /** 21 | * Return the data size for the given {@link LogSegmentName}. 22 | * @param name The {@link LogSegmentName}. 23 | * @return The data size. 24 | */ 25 | long getLogSegmentSize(LogSegmentName name); 26 | } 27 | -------------------------------------------------------------------------------- /ambry-store/src/main/java/com/github/ambry/store/MessageInfoType.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2023 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.store; 15 | 16 | public enum MessageInfoType { 17 | PUT, DELETE, TTL_UPDATE, UNDELETE, EXPIRED 18 | } 19 | -------------------------------------------------------------------------------- /ambry-store/src/main/java/com/github/ambry/store/StatsBasedCompactionPolicyFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2017 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.store; 15 | 16 | import com.github.ambry.config.StoreConfig; 17 | import com.github.ambry.utils.Time; 18 | 19 | 20 | /** 21 | * Factory class interface to get {@link StatsBasedCompactionPolicy} 22 | */ 23 | public class StatsBasedCompactionPolicyFactory implements CompactionPolicyFactory { 24 | 25 | private final StoreConfig storeConfig; 26 | private final Time time; 27 | 28 | public StatsBasedCompactionPolicyFactory(StoreConfig storeConfig, Time time) { 29 | this.storeConfig = storeConfig; 30 | this.time = time; 31 | } 32 | 33 | public CompactionPolicy getCompactionPolicy() { 34 | return new StatsBasedCompactionPolicy(storeConfig, time); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /ambry-store/src/test/java/com/github/ambry/store/DummyMessageStoreRecovery.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.store; 15 | 16 | import java.util.ArrayList; 17 | 18 | 19 | public class DummyMessageStoreRecovery implements MessageStoreRecovery { 20 | 21 | @Override 22 | public RecoveryResult recover(Read read, long startOffset, long endOffset, StoreKeyFactory factory) { 23 | return new RecoveryResult(new ArrayList(), null, endOffset); 24 | } 25 | } 26 | -------------------------------------------------------------------------------- /ambry-test-utils/src/main/java/com/github/ambry/filecopy/MockFileCopyHandlerFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2025 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.filecopy; 15 | 16 | import com.github.ambry.filetransfer.handler.FileCopyHandler; 17 | import com.github.ambry.filetransfer.handler.FileCopyHandlerFactory; 18 | 19 | 20 | public class MockFileCopyHandlerFactory implements FileCopyHandlerFactory { 21 | public MockFileCopyHandlerFactory() { 22 | 23 | } 24 | 25 | @Override 26 | public FileCopyHandler getFileCopyHandler() { 27 | return new MockNoOpFileCopyHandler(); 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /ambry-test-utils/src/main/java/com/github/ambry/server/Payload.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.server; 15 | 16 | import com.github.ambry.messageformat.BlobProperties; 17 | 18 | 19 | /** 20 | * All of the content of a blob 21 | */ 22 | class Payload { 23 | public byte[] blob; 24 | public byte[] metadata; 25 | public BlobProperties blobProperties; 26 | public String blobId; 27 | 28 | public Payload(BlobProperties blobProperties, byte[] metadata, byte[] blob, String blobId) { 29 | this.blobProperties = blobProperties; 30 | this.metadata = metadata; 31 | this.blob = blob; 32 | this.blobId = blobId; 33 | } 34 | } 35 | 36 | -------------------------------------------------------------------------------- /ambry-test-utils/src/main/java/com/github/ambry/store/MockIdFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.store; 15 | 16 | import java.io.DataInputStream; 17 | import java.io.IOException; 18 | 19 | 20 | public class MockIdFactory implements StoreKeyFactory { 21 | 22 | @Override 23 | public StoreKey getStoreKey(DataInputStream value) throws IOException { 24 | return new MockId(value); 25 | } 26 | 27 | @Override 28 | public StoreKey getStoreKey(String input) throws IOException { 29 | return new MockId(input); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /ambry-test-utils/src/main/java/com/github/ambry/utils/MockClock.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.utils; 16 | 17 | import com.codahale.metrics.Clock; 18 | import java.util.concurrent.TimeUnit; 19 | 20 | 21 | public class MockClock extends Clock { 22 | private long tick = 0; 23 | 24 | @Override 25 | public long getTick() { 26 | return tick; 27 | } 28 | 29 | public void tick(int seconds) { 30 | // Meter's TICK_INTERVAL = TimeUnit.SECONDS.toNanos(5) 31 | tick += TimeUnit.SECONDS.toNanos(seconds) + 1; 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /ambry-tools/src/main/java/com/github/ambry/tools/admin/PutGetHelperFactory.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.tools.admin; 15 | 16 | /** 17 | * Factory that assists in creating the {@link ConcurrencyTestTool.PutGetHelper} 18 | */ 19 | interface PutGetHelperFactory { 20 | 21 | /** 22 | * Returns a new instance of {@link ConcurrencyTestTool.PutGetHelper} 23 | * @returns a new instance of {@link ConcurrencyTestTool.PutGetHelper} 24 | * @throws Exception 25 | */ 26 | ConcurrencyTestTool.PutGetHelper getPutGetHelper() throws Exception; 27 | } 28 | -------------------------------------------------------------------------------- /ambry-tools/src/main/java/com/github/ambry/tools/perf/rest/PerfRestServerMain.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.tools.perf.rest; 15 | 16 | import com.github.ambry.rest.RestServerMain; 17 | 18 | 19 | /** 20 | * Used for starting/stopping an instance of {@link com.github.ambry.rest.RestServer} that acts as a perf REST server. 21 | */ 22 | public class PerfRestServerMain { 23 | 24 | public static void main(String[] args) { 25 | RestServerMain.main(args); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /ambry-tools/src/main/java/com/github/ambry/tools/perf/serverperf/LoadProducerConsumer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.tools.perf.serverperf; 15 | 16 | /** 17 | * Interface which needs to implemented for Load producer and consumer for 18 | * Server performance test 19 | */ 20 | public interface LoadProducerConsumer { 21 | /** 22 | * This will be called continuously until {@link ShutDownException} is thrown. 23 | * @throws ShutDownException shutdown exception 24 | * @throws Exception exception 25 | */ 26 | void produce() throws Exception; 27 | 28 | /** 29 | * This will be called continuously until {@link ShutDownException} is thrown. 30 | * @throws ShutDownException shutdown exception 31 | * @throws Exception exception 32 | */ 33 | void consume() throws Exception; 34 | } 35 | -------------------------------------------------------------------------------- /ambry-tools/src/main/java/com/github/ambry/tools/perf/serverperf/ResponseInfoProcessor.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.tools.perf.serverperf; 15 | 16 | import com.github.ambry.network.ResponseInfo; 17 | 18 | 19 | /** 20 | * Interface to implement a class which can process {@link ResponseInfo} 21 | * and can be passed to {@link ServerPerfNetworkQueue#poll(ResponseInfoProcessor)} 22 | */ 23 | interface ResponseInfoProcessor { 24 | void process(ResponseInfo responseInfo) throws Exception; 25 | } 26 | -------------------------------------------------------------------------------- /ambry-tools/src/main/java/com/github/ambry/tools/perf/serverperf/ShutDownException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2024 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.tools.perf.serverperf; 15 | 16 | public class ShutDownException extends Exception { 17 | 18 | public ShutDownException() { 19 | 20 | } 21 | } 22 | -------------------------------------------------------------------------------- /ambry-utils/src/main/java/com/github/ambry/utils/BloomFilterSerializer.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.utils; 15 | 16 | import java.io.DataInput; 17 | import java.io.DataOutput; 18 | import java.io.IOException; 19 | 20 | 21 | abstract class BloomFilterSerializer { 22 | public void serialize(BloomFilter bf, DataOutput out) throws IOException { 23 | out.writeInt(bf.hashCount); 24 | bf.bitset.serialize(out); 25 | } 26 | 27 | public BloomFilter deserialize(DataInput in, int maxPageCount) throws IOException { 28 | int hashes = in.readInt(); 29 | IBitSet bs = OpenBitSet.deserialize(in, maxPageCount); 30 | return createFilter(hashes, bs); 31 | } 32 | 33 | protected abstract BloomFilter createFilter(int hashes, IBitSet bs); 34 | } 35 | -------------------------------------------------------------------------------- /ambry-utils/src/main/java/com/github/ambry/utils/ByteBufferOutputStream.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.utils; 15 | 16 | import java.io.IOException; 17 | import java.io.OutputStream; 18 | import java.nio.ByteBuffer; 19 | 20 | 21 | public class ByteBufferOutputStream extends OutputStream { 22 | private ByteBuffer buffer; 23 | 24 | public ByteBufferOutputStream(ByteBuffer buffer) { 25 | this.buffer = buffer; 26 | } 27 | 28 | public void write(int b) throws IOException { 29 | buffer.put((byte) b); 30 | } 31 | 32 | public void write(byte[] bytes, int off, int len) throws IOException { 33 | buffer.put(bytes, off, len); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /ambry-utils/src/main/java/com/github/ambry/utils/GenericThrowableConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2021 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.utils; 15 | 16 | import java.util.function.Consumer; 17 | 18 | 19 | /** 20 | * Similar to {@link Consumer}, but able to throw checked exceptions. 21 | * @param the type of the input to the operation 22 | * @param the {@link Throwable} to throw 23 | */ 24 | @FunctionalInterface 25 | public interface GenericThrowableConsumer { 26 | 27 | /** 28 | * Performs this operation on the given argument. 29 | * 30 | * @param t the input argument 31 | */ 32 | void accept(T t) throws E; 33 | } 34 | -------------------------------------------------------------------------------- /ambry-utils/src/main/java/com/github/ambry/utils/ThrowingBiConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | 15 | package com.github.ambry.utils; 16 | 17 | /** 18 | * Similar to {@link java.util.function.BiConsumer}, but able to throw checked exceptions. 19 | * @param the type of the first argument to the operation 20 | * @param the type of the second argument to the operation 21 | */ 22 | public interface ThrowingBiConsumer { 23 | 24 | /** 25 | * Performs this operation on the given arguments. 26 | * 27 | * @param t the first input argument 28 | * @param u the second input argument 29 | */ 30 | void accept(T t, U u) throws Exception; 31 | } 32 | -------------------------------------------------------------------------------- /ambry-utils/src/main/java/com/github/ambry/utils/ThrowingConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.utils; 15 | 16 | import java.util.function.Consumer; 17 | 18 | 19 | /** 20 | * Similar to {@link Consumer}, but able to throw checked exceptions. 21 | * @param the type of the input to the operation 22 | */ 23 | public interface ThrowingConsumer { 24 | 25 | /** 26 | * Performs this operation on the given argument. 27 | * 28 | * @param t the input argument 29 | */ 30 | void accept(T t) throws Exception; 31 | } 32 | -------------------------------------------------------------------------------- /ambry-utils/src/main/java/com/github/ambry/utils/ThrowingFunction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2018 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.utils; 15 | 16 | import java.util.function.Function; 17 | 18 | /** 19 | * Similar to {@link Function}, but able to throw checked exceptions. 20 | * @param the type of the input to the operation 21 | * @param the type of the return value of the operation 22 | */ 23 | public interface ThrowingFunction { 24 | /** 25 | * Applies this function to the given argument. 26 | * @param t the function argument 27 | * @return the function result 28 | * @throws Exception if the function fails 29 | */ 30 | R apply(T t) throws Exception; 31 | } 32 | -------------------------------------------------------------------------------- /ambry-utils/src/test/java/com/github/ambry/utils/FileLockTest.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.utils; 15 | 16 | import java.io.File; 17 | import java.io.IOException; 18 | import org.junit.Assert; 19 | import org.junit.Test; 20 | 21 | 22 | /** 23 | * Tests for file lock 24 | */ 25 | public class FileLockTest { 26 | @Test 27 | public void testFileLock() throws IOException { 28 | File file = File.createTempFile("temp", "1"); 29 | file.deleteOnExit(); 30 | FileLock lock = new FileLock(file); 31 | lock.lock(); 32 | Assert.assertFalse(lock.tryLock()); 33 | lock.unlock(); 34 | Assert.assertTrue(lock.tryLock()); 35 | lock.unlock(); 36 | lock.destroy(); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /ambry-utils/src/test/java/com/github/ambry/utils/ResetableIterator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2016 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.utils; 15 | 16 | import java.util.Iterator; 17 | 18 | 19 | public interface ResetableIterator extends Iterator { 20 | public void reset(); 21 | 22 | int size(); 23 | } 24 | -------------------------------------------------------------------------------- /ambry-utils/src/test/java/com/github/ambry/utils/ZeroBytesInputStreamTest.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2018 LinkedIn Corp. All rights reserved. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | */ 14 | package com.github.ambry.utils; 15 | 16 | import org.junit.Test; 17 | 18 | 19 | /** 20 | * Test for ZeroBytesInputStream 21 | */ 22 | public class ZeroBytesInputStreamTest { 23 | 24 | @Test 25 | public void testZeroBytesInputStream() throws Exception { 26 | int size = 16; 27 | ZeroBytesInputStream zbis = new ZeroBytesInputStream(size); 28 | TestUtils.validateInputStreamContract(zbis); 29 | TestUtils.readInputStreamAndValidateSize(zbis, size); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /config/DataDumplog4j.props: -------------------------------------------------------------------------------- 1 | # Root logger option 2 | log4j.rootLogger=INFO, file 3 | #log4jrootLogger=INFO, stdout 4 | 5 | log4j.appender.file=org.apache.log4j.RollingFileAppender 6 | log4j.appender.file.File=/tmp/DumpData.out 7 | log4j.appender.file.MaxFileSize=100MB 8 | log4j.appender.file.MaxBackupIndex=10 9 | log4j.appender.file.layout=org.apache.log4j.PatternLayout 10 | log4j.appender.file.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n 11 | 12 | # Direct log messages to stdout 13 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 14 | log4j.appender.stdout.Target=System.out 15 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 16 | log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n 17 | -------------------------------------------------------------------------------- /config/HardwareLayout.json: -------------------------------------------------------------------------------- 1 | { 2 | "clusterName": "OneDiskOneReplica", 3 | "version": 99, 4 | "datacenters": [ 5 | { 6 | "dataNodes": [ 7 | { 8 | "disks": [ 9 | { 10 | "capacityInBytes": 21474836480, 11 | "hardwareState": "AVAILABLE", 12 | "mountPath": "/tmp" 13 | } 14 | ], 15 | "hardwareState": "AVAILABLE", 16 | "hostname": "localhost", 17 | "port": 6667, 18 | "http2port": 26667 19 | } 20 | ], 21 | "name": "Datacenter", 22 | "id" : "1" 23 | } 24 | ] 25 | } 26 | -------------------------------------------------------------------------------- /config/HardwareLayoutHttp2.json: -------------------------------------------------------------------------------- 1 | { 2 | "clusterName": "OneDiskOneReplica", 3 | "version": 99, 4 | "datacenters": [ 5 | { 6 | "dataNodes": [ 7 | { 8 | "disks": [ 9 | { 10 | "capacityInBytes": 21474836480, 11 | "hardwareState": "AVAILABLE", 12 | "mountPath": "/tmp" 13 | } 14 | ], 15 | "hardwareState": "AVAILABLE", 16 | "hostname": "localhost", 17 | "port": 6667, 18 | "http2port" : 8667 19 | } 20 | ], 21 | "name": "Datacenter", 22 | "id": "1" 23 | } 24 | ] 25 | } 26 | -------------------------------------------------------------------------------- /config/HardwareLayoutSsl.json: -------------------------------------------------------------------------------- 1 | { 2 | "clusterName": "OneDiskOneReplica", 3 | "version": 99, 4 | "datacenters": [ 5 | { 6 | "dataNodes": [ 7 | { 8 | "disks": [ 9 | { 10 | "capacityInBytes": 21474836480, 11 | "hardwareState": "AVAILABLE", 12 | "mountPath": "/tmp" 13 | } 14 | ], 15 | "hardwareState": "AVAILABLE", 16 | "hostname": "localhost", 17 | "port": 6667, 18 | "sslPort" : 7667, 19 | "http2port" : 8667 20 | } 21 | ], 22 | "name": "Datacenter", 23 | "id": "1" 24 | } 25 | ] 26 | } 27 | -------------------------------------------------------------------------------- /config/HelixTaskWorkflowManagerConfig.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2022 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | zk.layout.file.path=../config/zkLayout.json 14 | cluster.name= 15 | workflow.name= 16 | stats.reports.to.aggregate=PARTITION_CLASS_REPORT,ACCOUNT_REPORT 17 | delete.specified.workflow=false 18 | recurrent.interval.in.minutes=10 19 | -------------------------------------------------------------------------------- /config/PartitionLayout.json: -------------------------------------------------------------------------------- 1 | { 2 | "clusterName": "OneDiskOneReplica", 3 | "version": 88, 4 | "partitions": [ 5 | { 6 | "id": 0, 7 | "partitionClass": "max-replicas-all-datacenters", 8 | "partitionState": "READ_WRITE", 9 | "replicaCapacityInBytes": 10737418240, 10 | "replicas": [ 11 | { 12 | "hostname": "localhost", 13 | "mountPath": "/tmp", 14 | "port": 6667 15 | } 16 | ] 17 | } 18 | ] 19 | } 20 | -------------------------------------------------------------------------------- /config/PartitionLayoutLocalOnePartitionThreeReplicas.json: -------------------------------------------------------------------------------- 1 | { 2 | "clusterName": "LocalSetUp_ThreeNodeOneDisk_OnePartitionThreeReplicas", 3 | "version": 1, 4 | "partitions": [ 5 | { 6 | "id": 0, 7 | "partitionClass": "max-replicas-all-datacenters", 8 | "partitionState": "READ_WRITE", 9 | "replicaCapacityInBytes": 10737418240, 10 | "replicas": [ 11 | { 12 | "hostname": "localhost", 13 | "mountPath": "/tmp/1", 14 | "port": 6670 15 | }, 16 | { 17 | "hostname": "localhost", 18 | "mountPath": "/tmp/2", 19 | "port": 6671 20 | }, 21 | { 22 | "hostname": "localhost", 23 | "mountPath": "/tmp/3", 24 | "port": 6672 25 | } 26 | ] 27 | } 28 | ] 29 | } 30 | -------------------------------------------------------------------------------- /config/WagedRebalancerHelixConfig.json: -------------------------------------------------------------------------------- 1 | { 2 | "idealStateConfigFields": { 3 | "numReplicas": 3, 4 | "minActiveReplicas": 2 5 | }, 6 | "clusterConfigFields": { 7 | "evenness": 3, 8 | "lessMovement": 4, 9 | "partitionDiskWeightInGB": 386, 10 | "maxPartitionInTransition": 1500, 11 | "delayRebalanceTimeInMs": 21600000, 12 | "errorThresholdForLoadBalance": 2000, 13 | "maxThreadsForStateTransition": 300 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /config/azureconfig/HardwareLayout.json: -------------------------------------------------------------------------------- 1 | { 2 | "clusterName": "OneDiskOneReplica", 3 | "version": 99, 4 | "datacenters": [ 5 | { 6 | "dataNodes": [ 7 | { 8 | "disks": [ 9 | { 10 | "capacityInBytes": 21474836480, 11 | "hardwareState": "AVAILABLE", 12 | "mountPath": "/tmp" 13 | } 14 | ], 15 | "hardwareState": "AVAILABLE", 16 | "hostname": "localhost", 17 | "port": 6673 18 | } 19 | ], 20 | "name": "Datacenter", 21 | "id" : "1" 22 | } 23 | ] 24 | } 25 | -------------------------------------------------------------------------------- /config/azureconfig/PartitionLayout.json: -------------------------------------------------------------------------------- 1 | { 2 | "clusterName": "OneDiskOneReplica", 3 | "version": 88, 4 | "partitions": [ 5 | { 6 | "id": 0, 7 | "partitionClass": "max-replicas-all-datacenters", 8 | "partitionState": "READ_WRITE", 9 | "replicaCapacityInBytes": 10737418240, 10 | "replicas": [ 11 | { 12 | "hostname": "localhost", 13 | "mountPath": "/tmp", 14 | "port": 6673 15 | } 16 | ] 17 | } 18 | ] 19 | } 20 | -------------------------------------------------------------------------------- /config/azureconfig/server.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2019 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | host.name=localhost 14 | port=6673 15 | 16 | # cluster map 17 | clustermap.cluster.name=AmbryDev 18 | clustermap.datacenter.name=Datacenter 19 | clustermap.host.name=localhost 20 | -------------------------------------------------------------------------------- /config/concurrencyTool.router.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | # rest server 14 | rest.server.rest.request.service.factory=com.github.ambry.frontend.FrontendRestRequestServiceFactory 15 | 16 | rest.server.router.factory=com.github.ambry.router.NonBlockingRouterFactory 17 | 18 | # router 19 | router.hostname=localhost 20 | router.datacenter.name=Datacenter 21 | router.put.success.target=1 22 | router.delete.success.target=1 23 | router.scaling.unit.max.connections.per.port.plain.text=20 24 | 25 | # cluster map 26 | clustermap.cluster.name=Ambry_Dev 27 | clustermap.datacenter.name=Datacenter 28 | clustermap.host.name=localhost 29 | 30 | #kms 31 | kms.default.container.key=B374A26A71490437AA024E4FADD5B497FDFF1A8EA6FF12F6FB65AF2720B59CCF 32 | -------------------------------------------------------------------------------- /config/frontend.test.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | # rest server 14 | rest.server.rest.request.service.factory=com.github.ambry.frontend.FrontendRestRequestServiceFactory 15 | rest.server.router.factory=com.github.ambry.router.InMemoryRouterFactory 16 | 17 | # router 18 | router.hostname=localhost 19 | router.datacenter.name=Datacenter 20 | router.put.success.target=1 21 | 22 | # cluster map 23 | clustermap.cluster.name=Ambry_Dev 24 | clustermap.datacenter.name=Datacenter 25 | clustermap.host.name=localhost 26 | 27 | #kms 28 | kms.default.container.key=B374A26A71490437AA024E4FADD5B497FDFF1A8EA6FF12F6FB65AF2720B59CCF 29 | -------------------------------------------------------------------------------- /config/frontend_helix.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | # rest server 13 | rest.server.rest.request.service.factory=com.github.ambry.frontend.FrontendRestRequestServiceFactory 14 | # router 15 | router.hostname=localhost 16 | router.datacenter.name=dc1 17 | router.put.success.target=1 18 | router.delete.success.target=1 19 | 20 | #kms 21 | kms.default.container.key=B374A26A71490437AA024E4FADD5B497FDFF1A8EA6FF12F6FB65AF2720B59CCF 22 | 23 | #clustermap 24 | clustermap.host.name=localhost 25 | clustermap.cluster.name=Ambry-Proto 26 | clustermap.datacenter.name=dc1 27 | clustermap.clusteragents.factory=com.github.ambry.clustermap.HelixClusterAgentsFactory 28 | clustermap.dcs.zk.connect.strings={"zkInfo" : [ { "datacenter":"dc1", "id":"1", "zkConnectStr":"localhost:2199"}, { "datacenter":"dc2", "id":"2", "zkConnectStr":"localhost:2300"} ] } 29 | -------------------------------------------------------------------------------- /config/log4j2.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | %m%n 10 | 11 | 12 | 13 | 14 | 15 | 16 | 17 | 18 | 19 | 20 | 21 | 22 | 23 | -------------------------------------------------------------------------------- /config/mysql.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | mysql.account.service.db.info=[{"url":"jdbc:mysql://localhost/AccountMetadata?serverTimezone=UTC","datacenter":"dc1","isWriteable":"true","username":"travis","password":""}] 13 | mysql.account.service.db.info.new=[{"url":"jdbc:mysql://localhost/AccountMetadataNew?serverTimezone=UTC","datacenter":"dc1","isWriteable":"true","username":"travis","password":""}] 14 | -------------------------------------------------------------------------------- /config/perf.rest.server.properties: -------------------------------------------------------------------------------- 1 | # rest server 2 | rest.server.rest.request.service.factory=com.github.ambry.frontend.FrontendRestRequestServiceFactory 3 | rest.server.nio.server.factory=com.github.ambry.tools.perf.rest.PerfNioServerFactory 4 | rest.server.router.factory=com.github.ambry.tools.perf.rest.PerfRouterFactory 5 | 6 | #router 7 | router.hostname=localhost 8 | router.datacenter.name=Perf 9 | 10 | -------------------------------------------------------------------------------- /config/server.http2.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | host.name=localhost 14 | 15 | # cluster map 16 | clustermap.cluster.name=Ambry_Dev 17 | clustermap.datacenter.name=Datacenter 18 | clustermap.host.name=localhost 19 | clustermap.port=6667 20 | 21 | # server http2 22 | rest.server.rest.request.service.factory=com.github.ambry.server.StorageRestRequestService 23 | rest.server.nio.server.factory=com.github.ambry.rest.StorageServerNettyFactory 24 | ssl.client.authentication=none 25 | server.enable.store.data.prefetch=true 26 | -------------------------------------------------------------------------------- /config/server.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | host.name=localhost 14 | 15 | # cluster map 16 | clustermap.cluster.name=Ambry_Dev 17 | clustermap.datacenter.name=Datacenter 18 | clustermap.host.name=localhost 19 | clustermap.port=6667 20 | 21 | # server http2 22 | rest.server.rest.request.service.factory=com.github.ambry.server.StorageRestRequestService 23 | rest.server.nio.server.factory=com.github.ambry.rest.StorageServerNettyFactory 24 | ssl.client.authentication=none 25 | -------------------------------------------------------------------------------- /config/server1.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | host.name=localhost 14 | port=6670 15 | 16 | # cluster map 17 | clustermap.cluster.name=Ambry_Dev 18 | clustermap.datacenter.name=Datacenter 19 | clustermap.host.name=localhost 20 | clustermap.port=6670 21 | clustermap.enable.http2.replication = true 22 | 23 | # server http2 24 | rest.server.rest.request.service.factory=com.github.ambry.server.StorageRestRequestService 25 | rest.server.nio.server.factory=com.github.ambry.rest.StorageServerNettyFactory 26 | ssl.client.authentication=none -------------------------------------------------------------------------------- /config/server1_helix.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | host.name=localhost 14 | port=15088 15 | clustermap.host.name=localhost 16 | clustermap.port=15088 17 | clustermap.cluster.name=Ambry-Proto 18 | clustermap.datacenter.name=dc1 19 | clustermap.clusteragents.factory=com.github.ambry.clustermap.HelixClusterAgentsFactory 20 | clustermap.dcs.zk.connect.strings={"zkInfo" : [ { "datacenter":"dc1", "id":"1", "zkConnectStr":"localhost:2199"}, { "datacenter":"dc2", "id":"2", "zkConnectStr":"localhost:2300"} ] } 21 | -------------------------------------------------------------------------------- /config/server2.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | host.name=localhost 14 | port=6671 15 | 16 | # cluster map 17 | clustermap.cluster.name=Ambry_Dev 18 | clustermap.datacenter.name=Datacenter 19 | clustermap.host.name=localhost 20 | clustermap.port=6671 21 | 22 | # server http2 23 | rest.server.rest.request.service.factory=com.github.ambry.server.StorageRestRequestService 24 | rest.server.nio.server.factory=com.github.ambry.rest.StorageServerNettyFactory 25 | ssl.client.authentication=none -------------------------------------------------------------------------------- /config/server2_helix.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | host.name=localhost 14 | port=16088 15 | clustermap.host.name=localhost 16 | clustermap.port=16088 17 | clustermap.cluster.name=Ambry-Proto 18 | clustermap.datacenter.name=dc1 19 | clustermap.clusteragents.factory=com.github.ambry.clustermap.HelixClusterAgentsFactory 20 | clustermap.dcs.zk.connect.strings={"zkInfo" : [ { "datacenter":"dc1", "id":"1", "zkConnectStr":"localhost:2199"}, { "datacenter":"dc2", "id":"2", "zkConnectStr":"localhost:2300"} ] } 21 | -------------------------------------------------------------------------------- /config/server3.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | host.name=localhost 14 | port=6672 15 | 16 | # cluster map 17 | clustermap.cluster.name=Ambry_Dev 18 | clustermap.datacenter.name=Datacenter 19 | clustermap.host.name=localhost 20 | clustermap.port=6672 21 | 22 | # server http2 23 | rest.server.rest.request.service.factory=com.github.ambry.server.StorageRestRequestService 24 | rest.server.nio.server.factory=com.github.ambry.rest.StorageServerNettyFactory 25 | ssl.client.authentication=none -------------------------------------------------------------------------------- /config/server3_helix.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | host.name=localhost 14 | port=17088 15 | clustermap.host.name=localhost 16 | clustermap.port=17088 17 | clustermap.cluster.name=Ambry-Proto 18 | clustermap.datacenter.name=dc1 19 | clustermap.clusteragents.factory=com.github.ambry.clustermap.HelixClusterAgentsFactory 20 | clustermap.dcs.zk.connect.strings={"zkInfo" : [ { "datacenter":"dc1", "id":"1", "zkConnectStr":"localhost:2199"}, { "datacenter":"dc2", "id":"2", "zkConnectStr":"localhost:2300"} ] } 21 | -------------------------------------------------------------------------------- /config/server4_helix.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | host.name=localhost 14 | port=18088 15 | clustermap.host.name=localhost 16 | clustermap.port=18088 17 | clustermap.cluster.name=Ambry-Proto 18 | clustermap.datacenter.name=dc2 19 | clustermap.clusteragents.factory=com.github.ambry.clustermap.HelixClusterAgentsFactory 20 | clustermap.dcs.zk.connect.strings={"zkInfo" : [ { "datacenter":"dc1", "id":"1", "zkConnectStr":"localhost:2199"}, { "datacenter":"dc2", "id":"2", "zkConnectStr":"localhost:2300"} ] } 21 | -------------------------------------------------------------------------------- /config/server5_helix.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | host.name=localhost 14 | port=19088 15 | clustermap.host.name=localhost 16 | clustermap.port=19088 17 | clustermap.cluster.name=Ambry-Proto 18 | clustermap.datacenter.name=dc2 19 | clustermap.clusteragents.factory=com.github.ambry.clustermap.HelixClusterAgentsFactory 20 | clustermap.dcs.zk.connect.strings={"zkInfo" : [ { "datacenter":"dc1", "id":"1", "zkConnectStr":"localhost:2199"}, { "datacenter":"dc2", "id":"2", "zkConnectStr":"localhost:2300"} ] } 21 | -------------------------------------------------------------------------------- /config/server6_helix.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | 13 | host.name=localhost 14 | port=20088 15 | clustermap.host.name=localhost 16 | clustermap.port=20088 17 | clustermap.cluster.name=Ambry-Proto 18 | clustermap.datacenter.name=dc2 19 | clustermap.clusteragents.factory=com.github.ambry.clustermap.HelixClusterAgentsFactory 20 | clustermap.dcs.zk.connect.strings={"zkInfo" : [ { "datacenter":"dc1", "id":"1", "zkConnectStr":"localhost:2199"}, { "datacenter":"dc2", "id":"2", "zkConnectStr":"localhost:2300"} ] } 21 | -------------------------------------------------------------------------------- /config/vcrhelixconfig/CrushEdRebalancerHelixConfig.json: -------------------------------------------------------------------------------- 1 | { 2 | "clusterConfigFields": { 3 | "maxOfflineInstancesAllowed": 4, 4 | "numOfflineInstancesForAutoExit": 2, 5 | "allowAutoJoin": true 6 | }, 7 | "idealStateConfigFields": { 8 | "numReplicas": 2, 9 | "stateModelDefRef": "OnlineOffline", 10 | "rebalanceStrategy": "org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy" 11 | } 12 | } 13 | -------------------------------------------------------------------------------- /config/vcrhelixconfig/DelayedAutoRebalancerHelixConfig.json: -------------------------------------------------------------------------------- 1 | { 2 | "clusterConfigFields": { 3 | "maxOfflineInstancesAllowed": 100, 4 | "numOfflineInstancesForAutoExit": 2, 5 | "allowAutoJoin": true 6 | }, 7 | "idealStateConfigFields": { 8 | "numReplicas": 1, 9 | "stateModelDefRef": "OnlineOffline", 10 | "rebalanceStrategy": "org.apache.helix.controller.rebalancer.strategy.CrushEdRebalanceStrategy", 11 | "minActiveReplicas": 0, 12 | "rebalancerClassName": "org.apache.helix.controller.rebalancer.DelayedAutoRebalancer", 13 | "rebalanceDelayInMins": 7200000 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /config/zkLayout.json: -------------------------------------------------------------------------------- 1 | { 2 | "zkInfo" : [ 3 | { 4 | "datacenter":"dc1", 5 | "id" : "1", 6 | "zkConnectStr":"localhost:2199" 7 | }, 8 | { 9 | "datacenter":"dc2", 10 | "id" : "2", 11 | "zkConnectStr":"localhost:2300" 12 | } 13 | ] 14 | } 15 | -------------------------------------------------------------------------------- /gradle.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 3 | # 4 | # Licensed under the Apache License, Version 2.0 (the "License"); you may not use 5 | # this file except in compliance with the License. You may obtain a copy of the 6 | # License at http://www.apache.org/licenses/LICENSE-2.0 7 | # 8 | # Unless required by applicable law or agreed to in writing, software distributed 9 | # under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 10 | # CONDITIONS OF ANY KIND, either express or implied. 11 | # 12 | org.gradle.daemon=true 13 | org.gradle.configureondemand=true 14 | org.gradle.caching=true 15 | -------------------------------------------------------------------------------- /gradle/buildscript.gradle: -------------------------------------------------------------------------------- 1 | // Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); you may not use 4 | // this file except in compliance with the License. You may obtain a copy of the 5 | // License at http://www.apache.org/licenses/LICENSE-2.0 6 | // 7 | // Unless required by applicable law or agreed to in writing, software distributed 8 | // under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 9 | // CONDITIONS OF ANY KIND, either express or implied. 10 | repositories { 11 | repositories { 12 | // For various gradle plugins. 13 | maven { 14 | url "https://plugins.gradle.org/m2/" 15 | } 16 | } 17 | } 18 | 19 | dependencies { 20 | classpath "gradle.plugin.com.hierynomus.gradle.plugins:license-gradle-plugin:0.15.0" 21 | classpath "org.shipkit:shipkit-auto-version:0.0.30" 22 | // this is actually the artifactory plugin 23 | classpath "org.jfrog.buildinfo:build-info-extractor-gradle:4.32.0" 24 | classpath "org.gradle:test-retry-gradle-plugin:1.3.1" 25 | } 26 | -------------------------------------------------------------------------------- /gradle/environment.gradle: -------------------------------------------------------------------------------- 1 | // Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); you may not use 4 | // this file except in compliance with the License. You may obtain a copy of the 5 | // License at http://www.apache.org/licenses/LICENSE-2.0 6 | // 7 | // Unless required by applicable law or agreed to in writing, software distributed 8 | // under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 9 | // CONDITIONS OF ANY KIND, either express or implied. 10 | /* 11 | * This script allows the Gradle environment to be overridden with custom 12 | * settings. This is useful in environments (such as a private company) where a 13 | * third party wishes to use custom repositories, or inject certain 14 | * functionality into the default Gradle build lifecycle. 15 | */ 16 | if (project.hasProperty('overrideBuildEnvironment')) { 17 | // Otherwise, assume overrideBuildEnvironment defines a path to a file, and apply it. 18 | apply from: project.overrideBuildEnvironment 19 | } 20 | -------------------------------------------------------------------------------- /gradle/license.gradle: -------------------------------------------------------------------------------- 1 | // Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); you may not use 4 | // this file except in compliance with the License. You may obtain a copy of the 5 | // License at http://www.apache.org/licenses/LICENSE-2.0 6 | // 7 | // Unless required by applicable law or agreed to in writing, software distributed 8 | // under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 9 | // CONDITIONS OF ANY KIND, either express or implied. 10 | subprojects { 11 | apply plugin: 'com.github.hierynomus.license' 12 | 13 | license { 14 | header rootProject.file('HEADER') 15 | // Skip Twitter bootstrap JS and CSS. 16 | exclude '**/bulkDelete.js' 17 | exclude '**/log4j2.xml' 18 | skipExistingHeaders = true 19 | } 20 | } 21 | -------------------------------------------------------------------------------- /gradle/wrapper/gradle-wrapper.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/linkedin/ambry/f4003842863a1d24d3f238a0c317e3b9915a1bd2/gradle/wrapper/gradle-wrapper.jar -------------------------------------------------------------------------------- /gradle/wrapper/gradle-wrapper.properties: -------------------------------------------------------------------------------- 1 | #Mon Feb 10 10:27:18 PST 2020 2 | distributionUrl=https\://services.gradle.org/distributions/gradle-5.2.1-all.zip 3 | distributionBase=GRADLE_USER_HOME 4 | distributionPath=wrapper/dists 5 | zipStorePath=wrapper/dists 6 | zipStoreBase=GRADLE_USER_HOME 7 | -------------------------------------------------------------------------------- /remove-dot-dirs.sh: -------------------------------------------------------------------------------- 1 | #! /bin/bash 2 | 3 | function mvdotdir() { 4 | OLDDIR=$(dirname $1) 5 | NEWDIR=${OLDDIR//./\/} 6 | if [ "$OLDDIR" != "$NEWDIR" ] 7 | then 8 | echo moving $1 to $NEWDIR 9 | mkdir -p $NEWDIR 10 | git mv $1 $NEWDIR 11 | fi 12 | } 13 | 14 | find -f **/src/*/java -type f | while read file; do mvdotdir "$file"; done 15 | -------------------------------------------------------------------------------- /settings.gradle: -------------------------------------------------------------------------------- 1 | // Copyright (C) 2014-2016 LinkedIn Corp. All rights reserved. 2 | // 3 | // Licensed under the Apache License, Version 2.0 (the "License"); you may not use 4 | // this file except in compliance with the License. You may obtain a copy of the 5 | // License at http://www.apache.org/licenses/LICENSE-2.0 6 | // 7 | // Unless required by applicable law or agreed to in writing, software distributed 8 | // under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR 9 | // CONDITIONS OF ANY KIND, either express or implied. 10 | include 'ambry-api', 11 | 'ambry-account', 12 | 'ambry-server', 13 | 'ambry-network', 14 | 'ambry-utils', 15 | 'ambry-test-utils', 16 | 'ambry-store', 17 | 'ambry-commons', 18 | 'ambry-messageformat', 19 | 'ambry-replication', 20 | 'ambry-clustermap', 21 | 'ambry-tools', 22 | 'ambry-protocol', 23 | 'ambry-rest', 24 | 'ambry-router', 25 | 'ambry-frontend', 26 | 'ambry-cloud', 27 | 'ambry-named-mysql', 28 | 'log4j-test-config', 29 | 'ambry-all', 30 | 'ambry-quota', 31 | 'ambry-mysql', 32 | 'ambry-filesystem', 33 | 'ambry-vcr', 34 | 'ambry-file-transfer', 35 | 'ambry-prioritization' 36 | 37 | -------------------------------------------------------------------------------- /version.properties: -------------------------------------------------------------------------------- 1 | # shipkit-auto-version Gradle plugin uses this version spec to deduct the version to build 2 | # it increments patch version based on most recent tag. 3 | # You can put explicit version here if needed, e.g. "1.0.0" 4 | # More information: https://github.com/shipkit/shipkit-auto-version/blob/master/README.md 5 | version=0.5.* 6 | --------------------------------------------------------------------------------