├── .asf.yaml ├── .circleci └── config.yml ├── .gitignore ├── CHANGES.txt ├── DEV-README.md ├── LICENSE.txt ├── NOTICE.txt ├── README.md ├── analytics-sidecar-client-common ├── build.gradle └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── cassandra │ │ └── sidecar │ │ ├── client │ │ ├── SidecarInstance.java │ │ └── SidecarInstanceImpl.java │ │ └── common │ │ ├── ApiEndpointsV1.java │ │ ├── DataObjectBuilder.java │ │ ├── data │ │ ├── ConsistencyConfig.java │ │ ├── ConsistencyLevel.java │ │ ├── ConsistencyVerificationResult.java │ │ ├── OperationalJobStatus.java │ │ ├── RestoreJobConstants.java │ │ ├── RestoreJobProgressFetchPolicy.java │ │ ├── RestoreJobSecrets.java │ │ ├── RestoreJobStatus.java │ │ ├── SSTableImportOptions.java │ │ └── StorageCredentials.java │ │ ├── exceptions │ │ └── RangeException.java │ │ ├── http │ │ ├── SidecarHttpHeaderNames.java │ │ └── SidecarHttpResponseStatus.java │ │ ├── request │ │ ├── AbortRestoreJobRequest.java │ │ ├── AllServicesConfigRequest.java │ │ ├── CassandraJmxHealthRequest.java │ │ ├── CassandraNativeHealthRequest.java │ │ ├── CleanSSTableUploadSessionRequest.java │ │ ├── ClearSnapshotRequest.java │ │ ├── ConnectedClientStatsRequest.java │ │ ├── CreateRestoreJobRequest.java │ │ ├── CreateRestoreJobSliceRequest.java │ │ ├── CreateSnapshotRequest.java │ │ ├── DeleteServiceConfigRequest.java │ │ ├── GossipHealthRequest.java │ │ ├── GossipInfoRequest.java │ │ ├── ImportSSTableRequest.java │ │ ├── JsonRequest.java │ │ ├── JsonResponseBytesDecoder.java │ │ ├── ListCdcSegmentsRequest.java │ │ ├── ListOperationalJobsRequest.java │ │ ├── ListSnapshotFilesRequest.java │ │ ├── NodeDecommissionRequest.java │ │ ├── NodeSettingsRequest.java │ │ ├── OperationalJobRequest.java │ │ ├── ReportSchemaRequest.java │ │ ├── Request.java │ │ ├── ResponseBytesDecoder.java │ │ ├── RestoreJobProgressRequest.java │ │ ├── RestoreJobSummaryRequest.java │ │ ├── RingRequest.java │ │ ├── SSTableComponentRequest.java │ │ ├── SchemaRequest.java │ │ ├── Service.java │ │ ├── SidecarHealthRequest.java │ │ ├── SnapshotRequest.java │ │ ├── StreamCdcSegmentRequest.java │ │ ├── StreamStatsRequest.java │ │ ├── TableStatsRequest.java │ │ ├── TimeSkewRequest.java │ │ ├── TokenRangeReplicasRequest.java │ │ ├── UpdateRestoreJobRequest.java │ │ ├── UpdateServiceConfigRequest.java │ │ ├── UploadSSTableRequest.java │ │ ├── UploadableRequest.java │ │ └── data │ │ │ ├── AbortRestoreJobRequestPayload.java │ │ │ ├── AllServicesConfigPayload.java │ │ │ ├── CreateRestoreJobRequestPayload.java │ │ │ ├── CreateSliceRequestPayload.java │ │ │ ├── Digest.java │ │ │ ├── MD5Digest.java │ │ │ ├── RestoreJobProgressRequestParams.java │ │ │ ├── UpdateCdcServiceConfigPayload.java │ │ │ ├── UpdateRestoreJobRequestPayload.java │ │ │ └── XXHash32Digest.java │ │ ├── response │ │ ├── ConnectedClientStatsResponse.java │ │ ├── GossipInfoResponse.java │ │ ├── HealthResponse.java │ │ ├── ListCdcSegmentsResponse.java │ │ ├── ListOperationalJobsResponse.java │ │ ├── ListSnapshotFilesResponse.java │ │ ├── NodeSettings.java │ │ ├── OperationalJobResponse.java │ │ ├── RingResponse.java │ │ ├── SSTableImportResponse.java │ │ ├── SSTableUploadResponse.java │ │ ├── SchemaResponse.java │ │ ├── StreamStatsResponse.java │ │ ├── TableStatsResponse.java │ │ ├── TimeSkewResponse.java │ │ ├── TokenRangeReplicasResponse.java │ │ └── data │ │ │ ├── CdcSegmentInfo.java │ │ │ ├── ClientConnectionEntry.java │ │ │ ├── CreateRestoreJobResponsePayload.java │ │ │ ├── RestoreJobProgressResponsePayload.java │ │ │ ├── RestoreJobSummaryResponsePayload.java │ │ │ ├── RestoreRangeJson.java │ │ │ ├── RingEntry.java │ │ │ └── StreamsProgressStats.java │ │ └── utils │ │ ├── HttpRange.java │ │ ├── Preconditions.java │ │ ├── StringUtils.java │ │ └── TimeUtils.java │ ├── test │ └── java │ │ └── org │ │ └── apache │ │ └── cassandra │ │ └── sidecar │ │ └── common │ │ ├── data │ │ ├── RestoreJobProgressFetchPolicyTest.java │ │ └── RestoreJobStatusTest.java │ │ ├── http │ │ └── SidecarHttpResponseStatusTest.java │ │ ├── request │ │ ├── CreateRestoreJobRequestPayloadTest.java │ │ ├── CreateSliceRequestPayloadTest.java │ │ ├── RestoreJobProgressRequestTest.java │ │ ├── UpdateRestoreJobRequestPayloadTest.java │ │ └── data │ │ │ └── AbortRestoreJobRequestPayloadTest.java │ │ ├── response │ │ ├── GossipInfoResponseTest.java │ │ ├── ListCdcSegmentsResponseTest.java │ │ └── RingResponseTest.java │ │ └── utils │ │ ├── HttpRangeTest.java │ │ └── TimeUtilsTest.java │ └── testFixtures │ └── java │ └── org │ └── apache │ └── cassandra │ └── sidecar │ └── foundation │ └── RestoreJobSecretsGen.java ├── analytics-sidecar-client ├── build.gradle └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── cassandra │ │ └── sidecar │ │ └── client │ │ ├── HttpClient.java │ │ ├── HttpClientConfig.java │ │ ├── HttpResponse.java │ │ ├── HttpResponseImpl.java │ │ ├── RequestContext.java │ │ ├── RequestExecutor.java │ │ ├── SidecarClient.java │ │ ├── SidecarClientBlobRestoreExtension.java │ │ ├── SidecarClientConfig.java │ │ ├── SidecarClientConfigImpl.java │ │ ├── SidecarInstancesProvider.java │ │ ├── SimpleSidecarInstancesProvider.java │ │ ├── StreamBuffer.java │ │ ├── StreamConsumer.java │ │ ├── exception │ │ ├── ResourceNotFoundException.java │ │ ├── RetriesExhaustedException.java │ │ └── UnexpectedStatusCodeException.java │ │ ├── retry │ │ ├── BasicRetryPolicy.java │ │ ├── CreateRestoreJobRetryPolicy.java │ │ ├── ExponentialBackoffRetryPolicy.java │ │ ├── IgnoreConflictRetryPolicy.java │ │ ├── NoRetryPolicy.java │ │ ├── OncePerInstanceRetryPolicy.java │ │ ├── RetryAction.java │ │ ├── RetryPolicy.java │ │ └── RunnableOnStatusCodeRetryPolicy.java │ │ └── selection │ │ ├── InstanceSelectionPolicy.java │ │ ├── OrderedInstanceSelectionPolicy.java │ │ ├── RandomInstanceSelectionPolicy.java │ │ └── SingleInstanceSelectionPolicy.java │ ├── test │ └── java │ │ └── org.apache.cassandra.sidecar.client │ │ ├── HttpClientConfigTest.java │ │ ├── SidecarClientConfigTest.java │ │ ├── SidecarInstanceImplTest.java │ │ ├── SidecarInstanceTest.java │ │ ├── SimpleSidecarInstancesProviderTest.java │ │ ├── StreamBufferTest.java │ │ ├── request │ │ ├── JsonRequestTest.java │ │ └── JsonResponseBytesDecoderTest.java │ │ ├── retry │ │ ├── BasicRetryPolicyTest.java │ │ ├── ExponentialBackoffRetryPolicyTest.java │ │ ├── IgnoreConflictRetryPolicyTest.java │ │ ├── NoRetryPolicyTest.java │ │ └── RunnableOnStatusCodeRetryPolicyTest.java │ │ └── selection │ │ ├── OrderedInstanceSelectionPolicyTest.java │ │ ├── RandomInstanceSelectionPolicyTest.java │ │ └── SingleInstanceSelectionPolicyTest.java │ └── testFixtures │ ├── java │ └── org │ │ └── apache │ │ └── cassandra │ │ └── sidecar │ │ └── client │ │ ├── SidecarClientTest.java │ │ └── request │ │ ├── BaseRequestTest.java │ │ ├── FullSchemaRequestTestParameters.java │ │ ├── GossipInfoRequestTestParameters.java │ │ ├── ListSnapshotFilesRequestTestParameters.java │ │ ├── NodeSettingsRequestTestParameters.java │ │ ├── RequestExecutorTest.java │ │ ├── RequestTestParameters.java │ │ ├── RingRequestForKeyspaceTestParameters.java │ │ ├── RingRequestTestParameters.java │ │ ├── SchemaRequestTestParameters.java │ │ └── TimeSkewRequestTestParameters.java │ └── resources │ └── sstables │ └── nb-1-big-TOC.txt ├── analytics-sidecar-vertx-client-shaded ├── build.gradle └── src │ └── test │ └── java │ └── org │ └── apache │ └── cassandra │ └── sidecar │ └── client │ └── test │ └── LibraryTest.java ├── analytics-sidecar-vertx-client ├── build.gradle └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── cassandra │ │ └── sidecar │ │ └── client │ │ ├── SidecarClientVertxRequestExecutor.java │ │ ├── StreamConsumerWriteStream.java │ │ ├── VertxHttpClient.java │ │ ├── VertxRequestExecutor.java │ │ └── VertxStreamBuffer.java │ └── test │ └── java │ └── org │ └── apache │ └── cassandra │ └── sidecar │ └── client │ ├── StreamConsumerWriteStreamTest.java │ ├── VertxSidecarClientTest.java │ ├── VertxStreamBufferTest.java │ └── request │ └── BaseVertxRequestTest.java ├── build.gradle ├── cassandra-analytics-cdc-codec ├── build.gradle ├── examples │ ├── basic.md │ ├── clusteringkeys.md │ ├── collections.md │ └── native_types.md └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── cassandra │ │ │ └── cdc │ │ │ ├── CdcEventTransformer.java │ │ │ ├── CdcLogMode.java │ │ │ ├── CdcLogger.java │ │ │ ├── avro │ │ │ ├── AvroBaseRecordTransformer.java │ │ │ ├── AvroByteRecordTransformer.java │ │ │ ├── AvroDataUtils.java │ │ │ ├── AvroGenericRecordTransformer.java │ │ │ ├── AvroSerializer.java │ │ │ ├── CdcEventAvroEncoder.java │ │ │ ├── CdcEventUtils.java │ │ │ ├── RecordReader.java │ │ │ ├── TypeConversion.java │ │ │ ├── TypeConversionRegistry.java │ │ │ └── msg │ │ │ │ ├── CdcEnvelope.java │ │ │ │ └── FieldValue.java │ │ │ ├── json │ │ │ ├── AvroJsonTransformer.java │ │ │ ├── CdcGenericData.java │ │ │ └── JsonSerializer.java │ │ │ ├── kafka │ │ │ ├── AvroGenericRecordSerializer.java │ │ │ ├── EventHasher.java │ │ │ ├── KafkaCdcSerializer.java │ │ │ ├── KafkaOptions.java │ │ │ ├── KafkaPublisher.java │ │ │ ├── KafkaStats.java │ │ │ ├── RecordProducer.java │ │ │ └── TopicSupplier.java │ │ │ └── schemastore │ │ │ ├── CachingSchemaStore.java │ │ │ ├── LocalTableSchemaStore.java │ │ │ ├── PublishSchemaResult.java │ │ │ ├── SchemaStore.java │ │ │ ├── SchemaStorePublisherFactory.java │ │ │ ├── SchemaStoreStats.java │ │ │ └── TableSchemaPublisher.java │ └── resources │ │ ├── cdc_bytes.avsc │ │ ├── cdc_generic_record.avsc │ │ └── table_schemas │ │ └── example.avsc │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── cassandra │ │ └── cdc │ │ ├── avro │ │ ├── AvroLogicalTypesTest.java │ │ ├── AvroSchemasTest.java │ │ ├── CqlToAvroSchemaConverterTest.java │ │ └── TypeConversionsTest.java │ │ ├── json │ │ └── JsonSerializerTests.java │ │ └── kafka │ │ ├── KafkaPublisherTest.java │ │ └── TopicSupplierTest.java │ └── resources │ ├── cql_schemas │ ├── test_ks.test_tbl1 │ ├── test_ks.test_tbl2 │ └── test_ks.test_tbl3 │ ├── expected.avro │ └── table_schemas │ ├── test_date.avsc │ ├── test_decimal.avsc │ ├── test_ks.test_tbl1.avsc │ ├── test_ks.test_tbl2.avsc │ ├── test_ks.test_tbl3.avsc │ ├── test_tbl_adv.avsc │ ├── test_tbl_basic.avsc │ ├── test_tbl_binary.avsc │ ├── test_tbl_collections.avsc │ ├── test_time-micros.avsc │ ├── test_timestamp-micros.avsc │ └── test_uuid.avsc ├── cassandra-analytics-cdc-sidecar ├── build.gradle └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── cassandra │ │ └── cdc │ │ ├── CassandraClientSource.java │ │ └── sidecar │ │ ├── ClusterConfigProvider.java │ │ ├── ReplicationFactorSupplier.java │ │ ├── SidecarCdc.java │ │ ├── SidecarCdcBuilder.java │ │ ├── SidecarCdcCassandraClient.java │ │ ├── SidecarCdcClient.java │ │ ├── SidecarCdcCommitLogSegment.java │ │ ├── SidecarCdcOptions.java │ │ ├── SidecarCdcStats.java │ │ ├── SidecarCommitLogProvider.java │ │ ├── SidecarDownMonitor.java │ │ └── SidecarStatePersister.java │ └── test │ └── java │ └── org │ └── apache │ └── cassandra │ └── cdc │ ├── CassandraClientSourceTest.java │ └── sidecar │ ├── SidecarCommitLogProviderTests.java │ └── SidecarStatePersisterTest.java ├── cassandra-analytics-cdc ├── build.gradle └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── cassandra │ │ ├── bridge │ │ └── CdcBridgeFactory.java │ │ └── cdc │ │ ├── Cdc.java │ │ ├── CdcBuilder.java │ │ ├── CdcKryoRegister.java │ │ ├── MicroBatchIterator.java │ │ ├── TypeCache.java │ │ ├── api │ │ ├── CommitLogProvider.java │ │ ├── EventConsumer.java │ │ ├── MessageConverter.java │ │ ├── SchemaSupplier.java │ │ ├── StatePersister.java │ │ └── TokenRangeSupplier.java │ │ ├── msg │ │ └── jdk │ │ │ ├── CdcMessage.java │ │ │ ├── Column.java │ │ │ ├── JdkMessageConverter.java │ │ │ └── RangeTombstoneMsg.java │ │ ├── scanner │ │ └── CdcScannerBuilder.java │ │ └── stats │ │ └── CdcStats.java │ └── test │ └── java │ └── org │ └── apache │ └── cassandra │ ├── cdc │ ├── CdcTester.java │ ├── CdcTests.java │ ├── CdcWriter.java │ ├── CollectionDeletionTests.java │ ├── LocalCommitLog.java │ ├── MicroBatchIteratorTests.java │ ├── PartitionDeletionTests.java │ ├── RangeDeletionTests.java │ ├── RowDeletionTests.java │ ├── TypeCacheTests.java │ ├── api │ │ └── CommitLogMarkerTests.java │ ├── model │ │ └── CdcKryoSerializationTests.java │ ├── msg │ │ └── jdk │ │ │ └── CdcMessageTests.java │ └── state │ │ └── CdcStateTests.java │ └── db │ └── commitlog │ └── BufferingCommitLogReaderTests.java ├── cassandra-analytics-common ├── build.gradle └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── cassandra │ │ ├── analytics │ │ └── stats │ │ │ └── Stats.java │ │ ├── bridge │ │ ├── BigNumberConfig.java │ │ ├── BigNumberConfigImpl.java │ │ ├── CassandraVersion.java │ │ ├── CassandraVersionFeatures.java │ │ ├── SSTableDescriptor.java │ │ ├── SSTableSummary.java │ │ ├── SSTableWriter.java │ │ ├── TokenRange.java │ │ └── type │ │ │ └── InternalDuration.java │ │ ├── cdc │ │ ├── api │ │ │ ├── CassandraSource.java │ │ │ ├── CdcOptions.java │ │ │ ├── CommitLog.java │ │ │ ├── CommitLogInstance.java │ │ │ ├── CommitLogMarkers.java │ │ │ ├── CommitLogReader.java │ │ │ ├── KeyspaceTypeKey.java │ │ │ ├── Marker.java │ │ │ ├── Mutation.java │ │ │ ├── PerInstanceCommitLogMarkers.java │ │ │ ├── PerRangeCommitLogMarkers.java │ │ │ ├── RangeTombstoneData.java │ │ │ ├── Row.java │ │ │ └── TableIdLookup.java │ │ ├── msg │ │ │ ├── CdcEvent.java │ │ │ ├── CdcEventBuilder.java │ │ │ ├── RangeTombstone.java │ │ │ ├── RangeTombstoneBuilder.java │ │ │ └── Value.java │ │ ├── scanner │ │ │ └── CdcStreamScanner.java │ │ ├── state │ │ │ └── CdcState.java │ │ └── stats │ │ │ └── ICdcStats.java │ │ ├── db │ │ └── commitlog │ │ │ └── PartitionUpdateWrapper.java │ │ ├── secrets │ │ ├── SecretsProvider.java │ │ ├── SslConfig.java │ │ └── SslConfigSecretsProvider.java │ │ ├── spark │ │ ├── common │ │ │ ├── SSTables.java │ │ │ ├── model │ │ │ │ ├── BulkFeatures.java │ │ │ │ ├── CassandraInstance.java │ │ │ │ ├── NodeState.java │ │ │ │ └── NodeStatus.java │ │ │ └── stats │ │ │ │ ├── JobStatsPublisher.java │ │ │ │ └── LogStatsPublisher.java │ │ ├── data │ │ │ ├── BasicSupplier.java │ │ │ ├── BridgeUdtValue.java │ │ │ ├── CassandraTypes.java │ │ │ ├── CqlField.java │ │ │ ├── CqlTable.java │ │ │ ├── DefaultSizing.java │ │ │ ├── FileSystemSSTable.java │ │ │ ├── FileSystemSource.java │ │ │ ├── FileType.java │ │ │ ├── IncompleteSSTableException.java │ │ │ ├── QualifiedTableName.java │ │ │ ├── ReplicationFactor.java │ │ │ ├── SSTable.java │ │ │ ├── SSTablesSupplier.java │ │ │ ├── Sizing.java │ │ │ ├── TypeConverter.java │ │ │ ├── model │ │ │ │ └── TokenOwner.java │ │ │ └── partitioner │ │ │ │ ├── CassandraInstance.java │ │ │ │ ├── CassandraRing.java │ │ │ │ ├── ConsistencyLevel.java │ │ │ │ ├── MurmurHash.java │ │ │ │ ├── NotEnoughReplicasException.java │ │ │ │ ├── Partitioner.java │ │ │ │ └── TokenPartitioner.java │ │ ├── exception │ │ │ ├── AnalyticsException.java │ │ │ ├── ConsistencyNotSatisfiedException.java │ │ │ ├── ImportFailedException.java │ │ │ ├── S3ApiCallException.java │ │ │ ├── SidecarApiCallException.java │ │ │ ├── TimeSkewTooLargeException.java │ │ │ └── UnsupportedAnalyticsOperationException.java │ │ ├── exceptions │ │ │ └── TransportFailureException.java │ │ ├── reader │ │ │ ├── EmptyStreamScanner.java │ │ │ ├── IndexConsumer.java │ │ │ ├── IndexEntry.java │ │ │ ├── RowData.java │ │ │ ├── SparkSSTableReader.java │ │ │ ├── StreamScanner.java │ │ │ └── common │ │ │ │ ├── AbstractCompressionMetadata.java │ │ │ │ ├── BigLongArray.java │ │ │ │ ├── ChunkCorruptException.java │ │ │ │ ├── IIndexReader.java │ │ │ │ └── SSTableStreamException.java │ │ ├── sparksql │ │ │ ├── Cell.java │ │ │ ├── CellIterator.java │ │ │ ├── FullRowBuilder.java │ │ │ ├── NoMatchFoundException.java │ │ │ ├── PartialRowBuilder.java │ │ │ ├── RowBuilder.java │ │ │ ├── RowIterator.java │ │ │ └── filters │ │ │ │ ├── PartitionKeyFilter.java │ │ │ │ ├── PruneColumnFilter.java │ │ │ │ └── SparkRangeFilter.java │ │ ├── stats │ │ │ └── BufferingInputStreamStats.java │ │ └── utils │ │ │ ├── ArrayUtils.java │ │ │ ├── AsyncExecutor.java │ │ │ ├── BuildInfo.java │ │ │ ├── ByteBufferUtils.java │ │ │ ├── ComparisonUtils.java │ │ │ ├── CqlUtils.java │ │ │ ├── FutureUtils.java │ │ │ ├── IOUtils.java │ │ │ ├── KryoUtils.java │ │ │ ├── LoggerHelper.java │ │ │ ├── MapUtils.java │ │ │ ├── Pair.java │ │ │ ├── Preconditions.java │ │ │ ├── Properties.java │ │ │ ├── RandomUtils.java │ │ │ ├── RangeUtils.java │ │ │ ├── ReaderTimeProvider.java │ │ │ ├── TableIdentifier.java │ │ │ ├── ThrowableUtils.java │ │ │ ├── Throwing.java │ │ │ ├── TimeProvider.java │ │ │ ├── TimeUtils.java │ │ │ ├── UUIDs.java │ │ │ ├── streaming │ │ │ ├── BufferingInputStream.java │ │ │ ├── CassandraFile.java │ │ │ ├── CassandraFileSource.java │ │ │ ├── StreamBuffer.java │ │ │ └── StreamConsumer.java │ │ │ └── test │ │ │ └── TestSSTable.java │ │ └── util │ │ ├── CompressionUtil.java │ │ ├── StatsUtil.java │ │ └── ThreadUtil.java │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── cassandra │ │ ├── bridge │ │ ├── BigNumberConfigImplTest.java │ │ └── type │ │ │ └── InternalDurationTest.java │ │ ├── secrets │ │ ├── SslConfigSecretsProviderTest.java │ │ └── SslConfigTest.java │ │ └── spark │ │ ├── data │ │ ├── ReplicationFactorTests.java │ │ └── partitioner │ │ │ ├── CassandraRingTests.java │ │ │ └── ConsistencyLevelTests.java │ │ ├── reader │ │ ├── OverlapTests.java │ │ └── RowDataTests.java │ │ ├── sparksql │ │ └── filters │ │ │ └── PartitionKeyFilterTests.java │ │ └── utils │ │ ├── ArrayUtilsTest.java │ │ ├── ByteBufferUtilsTests.java │ │ ├── LoggerHelperTests.java │ │ ├── RandomUtilsTest.java │ │ ├── RangeUtilsTest.java │ │ └── ThrowableUtilTests.java │ └── resources │ └── secrets │ └── fakecerts │ ├── client-keystore-password │ ├── client-keystore.p12 │ ├── client-truststore-password │ └── client-truststore.jks ├── cassandra-analytics-core-example ├── README.md ├── build.gradle └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── cassandra │ │ └── spark │ │ └── example │ │ ├── AbstractCassandraJob.java │ │ ├── DirectWriteAndReadJob.java │ │ ├── ExampleStorageTransportExtension.java │ │ ├── JobSelector.java │ │ ├── LocalS3WriteAndReadJob.java │ │ └── LocalStorageTransportExtension.java │ └── resources │ └── keystore-private.p12 ├── cassandra-analytics-core ├── build.gradle └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── cassandra │ │ │ ├── bridge │ │ │ └── CassandraBridgeFactory.java │ │ │ ├── clients │ │ │ └── AnalyticsSidecarClient.java │ │ │ └── spark │ │ │ ├── KryoRegister.java │ │ │ ├── bulkwriter │ │ │ ├── AbstractBulkWriterContext.java │ │ │ ├── BulkSparkConf.java │ │ │ ├── BulkWriteValidator.java │ │ │ ├── BulkWriterContext.java │ │ │ ├── BulkWriterContextFactory.java │ │ │ ├── CancelJobEvent.java │ │ │ ├── CassandraBulkSourceRelation.java │ │ │ ├── CassandraBulkWriterContext.java │ │ │ ├── CassandraClusterInfo.java │ │ │ ├── CassandraContext.java │ │ │ ├── CassandraDirectDataTransportContext.java │ │ │ ├── CassandraJobInfo.java │ │ │ ├── CassandraSchemaInfo.java │ │ │ ├── ClusterInfo.java │ │ │ ├── CommitCoordinator.java │ │ │ ├── CommitError.java │ │ │ ├── CommitResult.java │ │ │ ├── CqlTableInfoProvider.java │ │ │ ├── DataTransport.java │ │ │ ├── DataTransportInfo.java │ │ │ ├── DecoratedKey.java │ │ │ ├── DigestAlgorithmSupplier.java │ │ │ ├── DigestAlgorithms.java │ │ │ ├── DirectDataTransferApi.java │ │ │ ├── DirectStreamResult.java │ │ │ ├── DirectStreamSession.java │ │ │ ├── JobInfo.java │ │ │ ├── RecordWriter.java │ │ │ ├── RingInstance.java │ │ │ ├── SSTableWriterFactory.java │ │ │ ├── SchemaInfo.java │ │ │ ├── SidecarDataTransferApi.java │ │ │ ├── SimpleTaskScheduler.java │ │ │ ├── SortedSSTableWriter.java │ │ │ ├── SqlToCqlTypeConverter.java │ │ │ ├── StreamError.java │ │ │ ├── StreamResult.java │ │ │ ├── StreamSession.java │ │ │ ├── TTLOption.java │ │ │ ├── TableInfoProvider.java │ │ │ ├── TableSchema.java │ │ │ ├── TimestampOption.java │ │ │ ├── TokenPartitioner.java │ │ │ ├── Tokenizer.java │ │ │ ├── TransportContext.java │ │ │ ├── WriteAvailability.java │ │ │ ├── WriteMode.java │ │ │ ├── WriteResult.java │ │ │ ├── WriterOption.java │ │ │ ├── WriterOptions.java │ │ │ ├── cloudstorage │ │ │ │ ├── Bundle.java │ │ │ │ ├── BundleManifest.java │ │ │ │ ├── BundleNameGenerator.java │ │ │ │ ├── BundleStorageObject.java │ │ │ │ ├── CassandraCloudStorageTransportContext.java │ │ │ │ ├── CassandraTopologyMonitor.java │ │ │ │ ├── CloudStorageDataTransferApi.java │ │ │ │ ├── CloudStorageDataTransferApiFactory.java │ │ │ │ ├── CloudStorageDataTransferApiImpl.java │ │ │ │ ├── CloudStorageStreamResult.java │ │ │ │ ├── CloudStorageStreamSession.java │ │ │ │ ├── CreatedRestoreSlice.java │ │ │ │ ├── DataChunker.java │ │ │ │ ├── ImportCompletionCoordinator.java │ │ │ │ ├── ImportCoordinator.java │ │ │ │ ├── SSTableBundleSpec.md │ │ │ │ ├── SSTableCollector.java │ │ │ │ ├── SSTableLister.java │ │ │ │ ├── SSTablesBundler.java │ │ │ │ ├── StorageClient.java │ │ │ │ ├── StorageClientConfig.java │ │ │ │ └── coordinated │ │ │ │ │ ├── CassandraClusterInfoGroup.java │ │ │ │ │ ├── CassandraCoordinatedBulkWriterContext.java │ │ │ │ │ ├── CoordinatedCloudStorageDataTransferApi.java │ │ │ │ │ ├── CoordinatedCloudStorageDataTransferApiExtension.java │ │ │ │ │ ├── CoordinatedImportCoordinator.java │ │ │ │ │ ├── CoordinatedWriteConf.java │ │ │ │ │ ├── MultiClusterContainer.java │ │ │ │ │ └── MultiClusterSupport.java │ │ │ ├── token │ │ │ │ ├── ConsistencyLevel.java │ │ │ │ ├── MultiClusterReplicaAwareFailureHandler.java │ │ │ │ ├── ReplicaAwareFailureHandler.java │ │ │ │ ├── SingleClusterReplicaAwareFailureHandler.java │ │ │ │ ├── TokenRangeMapping.java │ │ │ │ └── TokenUtils.java │ │ │ └── util │ │ │ │ ├── FastByteOperations.java │ │ │ │ ├── IOUtils.java │ │ │ │ ├── SbwJavaSerializer.java │ │ │ │ ├── SbwKryoRegistrator.java │ │ │ │ └── TaskContextUtils.java │ │ │ ├── common │ │ │ ├── DataObjectBuilder.java │ │ │ ├── Digest.java │ │ │ ├── MD5Digest.java │ │ │ ├── SidecarInstanceFactory.java │ │ │ ├── XXHash32Digest.java │ │ │ └── schema │ │ │ │ ├── BooleanType.java │ │ │ │ ├── BytesType.java │ │ │ │ ├── CollectionType.java │ │ │ │ ├── ColumnType.java │ │ │ │ ├── ColumnTypes.java │ │ │ │ ├── ColumnUtil.java │ │ │ │ ├── DoubleType.java │ │ │ │ ├── IntegerType.java │ │ │ │ ├── ListType.java │ │ │ │ ├── LongType.java │ │ │ │ ├── MapType.java │ │ │ │ ├── SetType.java │ │ │ │ ├── StringType.java │ │ │ │ ├── StringUuidType.java │ │ │ │ ├── TimestampType.java │ │ │ │ └── UuidType.java │ │ │ ├── data │ │ │ ├── CassandraDataLayer.java │ │ │ ├── CassandraDataSourceHelper.java │ │ │ ├── ClientConfig.java │ │ │ ├── DataLayer.java │ │ │ ├── LocalDataLayer.java │ │ │ ├── PartitionedDataLayer.java │ │ │ ├── SidecarProvisionedSSTable.java │ │ │ └── partitioner │ │ │ │ ├── MultipleReplicas.java │ │ │ │ └── SingleReplica.java │ │ │ ├── sparksql │ │ │ ├── AbstractSparkRowIterator.java │ │ │ ├── CassandraDataSink.java │ │ │ └── SparkCellIterator.java │ │ │ ├── transports │ │ │ └── storage │ │ │ │ ├── StorageAccessConfiguration.java │ │ │ │ ├── StorageCredentialPair.java │ │ │ │ ├── StorageCredentials.java │ │ │ │ └── extensions │ │ │ │ ├── CommonStorageTransportExtension.java │ │ │ │ ├── CoordinatedTransportExtension.java │ │ │ │ ├── CoordinationSignalListener.java │ │ │ │ ├── CredentialChangeListener.java │ │ │ │ ├── DriverStorageTransportExtension.java │ │ │ │ ├── ExecutorStorageTransportExtension.java │ │ │ │ ├── ObjectFailureListener.java │ │ │ │ ├── StorageTransportConfiguration.java │ │ │ │ ├── StorageTransportExtension.java │ │ │ │ ├── StorageTransportHandler.java │ │ │ │ └── TransportExtensionUtils.java │ │ │ ├── utils │ │ │ ├── DigestAlgorithm.java │ │ │ ├── FilterUtils.java │ │ │ ├── MD5DigestAlgorithm.java │ │ │ ├── ScalaFunctions.java │ │ │ └── XXHash32DigestAlgorithm.java │ │ │ └── validation │ │ │ ├── BulkWriterKeyStoreValidation.java │ │ │ ├── BulkWriterTrustStoreValidation.java │ │ │ ├── CassandraValidation.java │ │ │ ├── SidecarValidation.java │ │ │ ├── SslValidation.java │ │ │ └── StartupValidatable.java │ ├── resources │ │ └── META-INF │ │ │ └── services │ │ │ └── org.apache.spark.sql.sources.DataSourceRegister │ └── spark3 │ │ └── org │ │ └── apache │ │ └── cassandra │ │ └── spark │ │ └── sparksql │ │ ├── CassandraDataSource.java │ │ ├── CassandraInputPartition.java │ │ ├── CassandraPartitionReaderFactory.java │ │ ├── CassandraPartitioning.java │ │ ├── CassandraScanBuilder.java │ │ ├── CassandraTable.java │ │ ├── CassandraTableProvider.java │ │ ├── LocalDataSource.java │ │ ├── LocalPartitionSizeSource.java │ │ ├── PartitionSizeIterator.java │ │ ├── PartitionSizeTableProvider.java │ │ └── SparkRowIterator.java │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── cassandra │ │ ├── bridge │ │ ├── CassandraVersionFeaturesTest.java │ │ ├── CompressionTests.java │ │ └── VersionRunner.java │ │ ├── clients │ │ ├── SidecarClientConfigTest.java │ │ ├── SidecarInstanceTest.java │ │ └── SidecarNativeLibrariesTest.java │ │ ├── secrets │ │ └── TestSecretsProvider.java │ │ └── spark │ │ ├── EndToEndTests.java │ │ ├── KryoSerializationTests.java │ │ ├── SSTableReaderTests.java │ │ ├── TestDataLayer.java │ │ ├── TestRunnable.java │ │ ├── TestUtils.java │ │ ├── Tester.java │ │ ├── bulkwriter │ │ ├── BulkSparkConfTest.java │ │ ├── BulkWriteValidatorTest.java │ │ ├── BulkWriterContextFactoryTest.java │ │ ├── CassandraClusterInfoTest.java │ │ ├── CommitCoordinatorTest.java │ │ ├── CqlTableInfoProviderTest.java │ │ ├── DecoratedKeyTest.java │ │ ├── DirectStreamSessionTest.java │ │ ├── MockBulkWriterContext.java │ │ ├── MockScheduledExecutorService.java │ │ ├── MockTableWriter.java │ │ ├── NonValidatingTestSortedSSTableWriter.java │ │ ├── RecordWriterTest.java │ │ ├── RingInstanceSerializationTest.java │ │ ├── RingInstanceTest.java │ │ ├── SSTables.java │ │ ├── SimpleTaskSchedulerTest.java │ │ ├── SortedSSTableWriterTest.java │ │ ├── SqlToCqlTypeConverterTest.java │ │ ├── StreamSessionConsistencyTest.java │ │ ├── TableSchemaNormalizeTest.java │ │ ├── TableSchemaTest.java │ │ ├── TableSchemaTestCommon.java │ │ ├── TokenPartitionerTest.java │ │ ├── TokenRangeMappingUtils.java │ │ ├── UploadRequest.java │ │ ├── WriteAvailabilityTest.java │ │ ├── benchmarks │ │ │ └── ByteOperationsBenchmark.java │ │ ├── cloudstorage │ │ │ ├── BundleManifestTest.java │ │ │ ├── BundleNameGeneratorTest.java │ │ │ ├── BundleTest.java │ │ │ ├── CassandraTopologyMonitorTest.java │ │ │ ├── CloudStorageDataTransferApiFactoryTest.java │ │ │ ├── CloudStorageStreamSessionTest.java │ │ │ ├── CreatedRestoreSliceTest.java │ │ │ ├── DataChunkerTest.java │ │ │ ├── ImportCompletionCoordinatorTest.java │ │ │ ├── SSTableListerTest.java │ │ │ ├── SSTablesBundlerTest.java │ │ │ └── coordinated │ │ │ │ ├── CassandraClusterInfoGroupTest.java │ │ │ │ ├── CoordinatedImportCoordinatorTest.java │ │ │ │ ├── CoordinatedWriteConfTest.java │ │ │ │ └── MultiClusterContainerTest.java │ │ └── token │ │ │ ├── BulkWriterConsistencyLevelTest.java │ │ │ ├── FailureHandlerTextContext.java │ │ │ ├── MultiClusterReplicaAwareFailureHandlerTest.java │ │ │ ├── SingleClusterReplicaAwareFailureHandlerTest.java │ │ │ └── TokenRangeMappingTest.java │ │ ├── data │ │ ├── CassandraDataLayerTests.java │ │ ├── CassandraDataSourceHelperCacheTest.java │ │ ├── ClientConfigTests.java │ │ ├── CqlFieldComparatorTests.java │ │ ├── CqlFieldTests.java │ │ ├── DataLayerUnsupportedPushDownFiltersTest.java │ │ ├── DefaultSizingTest.java │ │ ├── LocalDataLayerTests.java │ │ ├── PartitionedDataLayerTests.java │ │ ├── SidecarProvisionedSSTableTest.java │ │ ├── VersionRunner.java │ │ └── partitioner │ │ │ ├── JDKSerializationTests.java │ │ │ ├── MultipleReplicasTests.java │ │ │ ├── SingleReplicaTests.java │ │ │ └── TokenPartitionerTests.java │ │ ├── reader │ │ ├── CassandraBridgeTests.java │ │ ├── CassandraBridgeUtilTests.java │ │ ├── DataTypeSerializationTests.java │ │ ├── PartitionKeyTests.java │ │ ├── SchemaTests.java │ │ ├── TombstoneWriterTests.java │ │ └── common │ │ │ └── CompressionInputStreamTests.java │ │ ├── sparksql │ │ └── SparkRowIteratorTests.java │ │ ├── transports │ │ └── storage │ │ │ ├── StorageAccessConfigurationTest.java │ │ │ └── StorageCredentialPairTest.java │ │ ├── utils │ │ ├── BufferingInputStreamHttpTest.java │ │ ├── BufferingInputStreamTests.java │ │ ├── BuildInfoTest.java │ │ ├── CqlUtilsTest.java │ │ ├── FilterUtilsTests.java │ │ ├── IOUtilsTest.java │ │ ├── MD5DigestAlgorithmTest.java │ │ ├── ResourceUtils.java │ │ ├── SerializationUtils.java │ │ └── XXHash32DigestAlgorithmTest.java │ │ └── validation │ │ ├── KeyStoreValidationTests.java │ │ ├── StartupValidatorTests.java │ │ ├── TestValidation.java │ │ └── TrustStoreValidationTests.java │ ├── resources │ ├── cql │ │ └── fullSchema.cql │ ├── data │ │ └── ks │ │ │ └── table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0 │ │ │ ├── na-1-big-Data.db │ │ │ ├── na-1-big-Filter.db │ │ │ ├── na-1-big-Index.db │ │ │ ├── na-1-big-Statistics.db │ │ │ ├── na-1-big-Summary.db │ │ │ ├── na-1-big-TOC.txt │ │ │ ├── na-2-big-Data.db │ │ │ ├── na-2-big-Filter.db │ │ │ ├── na-2-big-Index.db │ │ │ ├── na-2-big-Statistics.db │ │ │ ├── na-2-big-Summary.db │ │ │ └── na-2-big-TOC.txt │ ├── digest │ │ ├── file1.txt │ │ ├── file2.txt │ │ └── file3.txt │ └── validation │ │ ├── keystore-certificate.p12 │ │ ├── keystore-empty.p12 │ │ ├── keystore-expired.p12 │ │ ├── keystore-malformed.p12 │ │ ├── keystore-private.p12 │ │ └── keystore-secret.p12 │ └── spark3 │ └── org │ └── apache │ └── cassandra │ └── spark │ ├── PartitionSizeTests.java │ ├── bulkwriter │ └── TestTaskContext.java │ └── common │ └── SidecarInstanceFactoryTest.java ├── cassandra-analytics-integration-framework ├── build.gradle └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── cassandra │ │ ├── distributed │ │ └── impl │ │ │ └── CassandraCluster.java │ │ ├── sidecar │ │ └── testing │ │ │ ├── LocalhostResolver.java │ │ │ ├── MtlsTestHelper.java │ │ │ ├── QualifiedName.java │ │ │ ├── SharedClusterIntegrationTestBase.java │ │ │ ├── SharedExecutorNettyOptions.java │ │ │ └── TemporaryCqlSessionProvider.java │ │ └── testing │ │ ├── ClusterBuilderConfiguration.java │ │ ├── IClusterExtension.java │ │ ├── IsolatedDTestClassLoaderWrapper.java │ │ ├── Partitioner.java │ │ ├── TestTokenSupplier.java │ │ ├── TestUtils.java │ │ ├── TestVersion.java │ │ ├── TestVersionSupplier.java │ │ └── utils │ │ ├── ClusterUtils.java │ │ ├── WithProperties.java │ │ └── tls │ │ ├── CertificateBuilder.java │ │ └── CertificateBundle.java │ └── test │ └── java │ └── org │ └── apache │ └── cassandra │ └── sidecar │ └── testing │ └── LocalhostResolverTest.java ├── cassandra-analytics-integration-tests ├── build.gradle └── src │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── cassandra │ │ └── analytics │ │ ├── BulkReaderTest.java │ │ ├── BulkWriteDataTypesTest.java │ │ ├── BulkWriteDownInstanceMultipleTokensTest.java │ │ ├── BulkWriteDownInstanceTest.java │ │ ├── BulkWriteDownSidecarMultipleTokensTest.java │ │ ├── BulkWriteDownSidecarTest.java │ │ ├── BulkWriteTtlTest.java │ │ ├── BulkWriteUdtTest.java │ │ ├── CassandraAnalyticsSimpleMultipleTokensTest.java │ │ ├── CassandraAnalyticsSimpleTest.java │ │ ├── DataGenerationUtils.java │ │ ├── PartitionKeyIntegrationTest.java │ │ ├── QuoteIdentifiersReadTest.java │ │ ├── QuoteIdentifiersWriteTest.java │ │ ├── RandomPartitionerTest.java │ │ ├── ResiliencyTestBase.java │ │ ├── SharedClusterSparkIntegrationTestBase.java │ │ ├── SparkTestUtils.java │ │ ├── SparkTestUtilsProvider.java │ │ ├── TestConsistencyLevel.java │ │ ├── TestUninterruptibles.java │ │ ├── TimestampIntegrationTest.java │ │ ├── WriterDigestIntegrationTest.java │ │ ├── correctness │ │ └── BulkWriteCorruptionTest.java │ │ ├── data │ │ └── ClearSnapshotTest.java │ │ ├── expansion │ │ ├── JoiningDoubleClusterFailureTest.java │ │ ├── JoiningDoubleClusterTest.java │ │ ├── JoiningMultiDCFailureTest.java │ │ ├── JoiningMultiDCSingleReplicatedFailureTest.java │ │ ├── JoiningMultiDCSingleReplicatedTest.java │ │ ├── JoiningMultiDCTest.java │ │ ├── JoiningMultipleNodesFailureTest.java │ │ ├── JoiningMultipleNodesTest.java │ │ ├── JoiningSingleNodeFailureTest.java │ │ ├── JoiningSingleNodeTest.java │ │ └── JoiningTestBase.java │ │ ├── movement │ │ ├── NodeMovementFailureTest.java │ │ ├── NodeMovementMultiDCFailureTest.java │ │ ├── NodeMovementMultiDCTest.java │ │ ├── NodeMovementTest.java │ │ └── NodeMovementTestBase.java │ │ ├── replacement │ │ ├── HostReplacementFailureTest.java │ │ ├── HostReplacementMultiDCFailureTest.java │ │ ├── HostReplacementMultiDCInsufficientReplicasTest.java │ │ ├── HostReplacementMultiDCTest.java │ │ ├── HostReplacementTest.java │ │ └── HostReplacementTestBase.java │ │ ├── shrink │ │ ├── LeavingHalfTest.java │ │ ├── LeavingMultiDCFailureTest.java │ │ ├── LeavingMultiDCHalveClusterFailureTest.java │ │ ├── LeavingMultiDCHalveClusterTest.java │ │ ├── LeavingMultiDCTest.java │ │ ├── LeavingMultipleFailureTest.java │ │ ├── LeavingMultipleTest.java │ │ ├── LeavingSingleFailureTest.java │ │ ├── LeavingSingleTest.java │ │ └── LeavingTestBase.java │ │ └── testcontainer │ │ ├── BulkWriteS3CompatModeSimpleMultipleTokensTest.java │ │ ├── BulkWriteS3CompatModeSimpleTest.java │ │ ├── CoordinatedBulkWriteSimpleTest.java │ │ ├── CoordinatedWriteTestBase.java │ │ ├── LocalCoordinatedStorageTransportExtension.java │ │ └── LocalStorageTransportExtension.java │ └── resources │ └── logback-test.xml ├── cassandra-analytics-sidecar-client ├── build.gradle └── src │ └── main │ └── java │ └── org │ └── apache │ └── cassandra │ ├── clients │ ├── ExecutorHolder.java │ ├── Sidecar.java │ └── SidecarStreamConsumerAdapter.java │ └── spark │ └── validation │ ├── KeyStoreValidation.java │ ├── StartupValidation.java │ ├── StartupValidator.java │ └── TrustStoreValidation.java ├── cassandra-analytics-spark-converter ├── build.gradle └── src │ └── main │ ├── java │ └── org │ │ └── apache │ │ └── cassandra │ │ └── spark │ │ ├── config │ │ ├── SchemaFeature.java │ │ └── SchemaFeatureSet.java │ │ ├── data │ │ └── converter │ │ │ ├── SparkSqlTypeConverter.java │ │ │ └── types │ │ │ ├── BinaryFeatures.java │ │ │ ├── DecimalFeatures.java │ │ │ ├── Empty.java │ │ │ ├── IntFeatures.java │ │ │ ├── LongFeatures.java │ │ │ ├── NotImplementedFeatures.java │ │ │ ├── SparkAscii.java │ │ │ ├── SparkBigInt.java │ │ │ ├── SparkBlob.java │ │ │ ├── SparkBoolean.java │ │ │ ├── SparkCounter.java │ │ │ ├── SparkDate.java │ │ │ ├── SparkDecimal.java │ │ │ ├── SparkDouble.java │ │ │ ├── SparkDuration.java │ │ │ ├── SparkFloat.java │ │ │ ├── SparkInet.java │ │ │ ├── SparkInt.java │ │ │ ├── SparkSmallInt.java │ │ │ ├── SparkText.java │ │ │ ├── SparkTime.java │ │ │ ├── SparkTimeUUID.java │ │ │ ├── SparkTimestamp.java │ │ │ ├── SparkTinyInt.java │ │ │ ├── SparkType.java │ │ │ ├── SparkUUID.java │ │ │ ├── SparkVarChar.java │ │ │ ├── SparkVarInt.java │ │ │ ├── StringFeatures.java │ │ │ ├── UUIDFeatures.java │ │ │ └── complex │ │ │ ├── CollectionFeatures.java │ │ │ ├── MapFeatures.java │ │ │ ├── SparkFrozen.java │ │ │ ├── SparkList.java │ │ │ ├── SparkMap.java │ │ │ ├── SparkSet.java │ │ │ ├── SparkTuple.java │ │ │ └── SparkUdt.java │ │ ├── sparksql │ │ ├── LastModifiedTimestampDecorator.java │ │ └── RowBuilderDecorator.java │ │ └── utils │ │ ├── FastThreadLocalUtf8Decoder.java │ │ ├── SparkClassLoaderOverride.java │ │ └── TemporaryDirectory.java │ ├── scala-2.11-spark-2 │ └── org │ │ └── apache │ │ └── cassandra │ │ └── spark │ │ └── utils │ │ ├── ScalaConversionUtils.java │ │ └── SparkTypeUtils.java │ ├── scala-2.12-spark-2 │ └── org │ │ └── apache │ │ └── cassandra │ │ └── spark │ │ └── utils │ │ ├── ScalaConversionUtils.java │ │ └── SparkTypeUtils.java │ ├── scala-2.12-spark-3 │ └── org │ │ └── apache │ │ └── cassandra │ │ └── spark │ │ └── utils │ │ ├── ScalaConversionUtils.java │ │ └── SparkTypeUtils.java │ └── scala-2.13-spark-3 │ └── org │ └── apache │ └── cassandra │ └── spark │ └── utils │ ├── ScalaConversionUtils.java │ └── SparkTypeUtils.java ├── cassandra-analytics-spark-four-zero-converter ├── build.gradle └── src │ └── main │ └── java │ └── org │ └── apache │ └── cassandra │ └── spark │ └── data │ └── converter │ └── SparkSqlTypeConverterImplementation.java ├── cassandra-avro-converter ├── build.gradle └── src │ └── main │ └── java │ └── org │ └── apache │ └── cassandra │ └── cdc │ └── avro │ ├── AvroConstants.java │ ├── AvroSchemas.java │ └── CqlToAvroSchemaConverter.java ├── cassandra-bridge ├── build.gradle └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── cassandra │ │ ├── analytics │ │ └── reader │ │ │ └── common │ │ │ ├── IndexIterator.java │ │ │ └── RawInputStream.java │ │ └── bridge │ │ ├── BaseCassandraBridgeFactory.java │ │ ├── BloomFilter.java │ │ ├── CassandraBridge.java │ │ ├── CdcBridge.java │ │ ├── PostDelegationClassLoader.java │ │ └── Tokenizer.java │ └── testFixtures │ └── java │ └── org │ └── apache │ └── cassandra │ └── spark │ ├── CommonTestUtils.java │ └── utils │ └── test │ └── TestSchema.java ├── cassandra-four-zero-avro-converter ├── build.gradle └── src │ └── main │ └── java │ └── org │ └── apache │ └── cassandra │ └── cdc │ └── avro │ └── CqlToAvroSchemaConverterImplementation.java ├── cassandra-four-zero-bridge ├── build.gradle └── src │ ├── main │ └── java │ │ └── org │ │ └── apache │ │ └── cassandra │ │ ├── bridge │ │ ├── CassandraBridgeImplementation.java │ │ ├── CdcBridgeImplementation.java │ │ └── SSTableWriterImplementation.java │ │ ├── cdc │ │ ├── FourZeroCommitLog.java │ │ ├── FourZeroMutation.java │ │ ├── msg │ │ │ ├── FourZeroCdcEventBuilder.java │ │ │ └── FourZeroRangeTombstoneBuilder.java │ │ └── scanner │ │ │ ├── CdcSortedStreamScanner.java │ │ │ └── HybridUnfilteredPartitionIterator.java │ │ ├── db │ │ └── commitlog │ │ │ ├── BufferingCommitLogReader.java │ │ │ ├── FourZeroPartitionUpdateWrapper.java │ │ │ └── SeekableCommitLogSegmentReader.java │ │ ├── io │ │ ├── sstable │ │ │ └── SSTableTombstoneWriter.java │ │ └── util │ │ │ └── CdcRandomAccessReader.java │ │ ├── spark │ │ └── reader │ │ │ ├── AbstractStreamScanner.java │ │ │ ├── CompactionStreamScanner.java │ │ │ ├── CompressedRawInputStream.java │ │ │ ├── CompressionMetadata.java │ │ │ ├── CompressionUtils.java │ │ │ ├── IndexDbUtils.java │ │ │ ├── IndexReader.java │ │ │ ├── ReaderUtils.java │ │ │ ├── SSTableCache.java │ │ │ ├── SSTableReader.java │ │ │ ├── Scannable.java │ │ │ └── SummaryDbUtils.java │ │ └── utils │ │ └── CompressionUtilImplementation.java │ └── test │ └── java │ └── org │ └── apache │ └── cassandra │ ├── bridge │ ├── CassandraSchemaTests.java │ └── SSTableWriterImplementationTest.java │ ├── spark │ ├── TestUtils.java │ ├── data │ │ └── converter │ │ │ └── types │ │ │ └── DateTypeTests.java │ └── reader │ │ ├── IndexDbTests.java │ │ ├── IndexOffsetTests.java │ │ ├── IndexReaderTests.java │ │ ├── PartitionKeyTests.java │ │ ├── ReaderUtilsTests.java │ │ ├── SSTableCacheTests.java │ │ ├── SSTableReaderTests.java │ │ ├── SchemaBuilderTests.java │ │ └── SummaryDbTests.java │ └── utils │ └── ReflectionUtils.java ├── cassandra-four-zero-types ├── build.gradle └── src │ └── main │ └── java │ └── org │ └── apache │ └── cassandra │ ├── bridge │ ├── CassandraSchema.java │ ├── CassandraTypesImplementation.java │ └── CollectionElement.java │ ├── cql3 │ └── functions │ │ └── types │ │ ├── TupleHelper.java │ │ └── UserTypeHelper.java │ ├── spark │ ├── data │ │ ├── CqlType.java │ │ ├── NativeType.java │ │ ├── complex │ │ │ ├── CqlCollection.java │ │ │ ├── CqlFrozen.java │ │ │ ├── CqlList.java │ │ │ ├── CqlMap.java │ │ │ ├── CqlSet.java │ │ │ ├── CqlTuple.java │ │ │ └── CqlUdt.java │ │ └── types │ │ │ ├── Ascii.java │ │ │ ├── BigInt.java │ │ │ ├── BinaryBased.java │ │ │ ├── Blob.java │ │ │ ├── Boolean.java │ │ │ ├── Counter.java │ │ │ ├── Date.java │ │ │ ├── Decimal.java │ │ │ ├── Double.java │ │ │ ├── Duration.java │ │ │ ├── Empty.java │ │ │ ├── Float.java │ │ │ ├── Inet.java │ │ │ ├── Int.java │ │ │ ├── LongBased.java │ │ │ ├── SmallInt.java │ │ │ ├── StringBased.java │ │ │ ├── Text.java │ │ │ ├── Time.java │ │ │ ├── TimeUUID.java │ │ │ ├── Timestamp.java │ │ │ ├── TinyInt.java │ │ │ ├── UUID.java │ │ │ ├── VarChar.java │ │ │ └── VarInt.java │ └── reader │ │ ├── ComplexTypeBuffer.java │ │ ├── ListBuffer.java │ │ ├── MapBuffer.java │ │ ├── SchemaBuilder.java │ │ ├── SetBuffer.java │ │ └── UdtBuffer.java │ └── utils │ ├── ColumnTypes.java │ └── TokenUtils.java ├── cassandra-four-zero └── build.gradle ├── cassandra-three-zero └── TODO.md ├── code_version.sh ├── config └── checkstyle │ ├── checkstyle.xml │ └── suppressions.xml ├── githooks └── pre-push ├── gradle.properties ├── gradle └── wrapper │ ├── gradle-wrapper.jar │ └── gradle-wrapper.properties ├── gradlew ├── ide └── idea │ ├── Project_Default.xml │ └── codeStyleSettings.xml ├── profiles ├── scala-2.12-spark-3-jdk-11.gradle └── scala-2.13-spark-3-jdk-11.gradle ├── scripts ├── .mvn │ └── wrapper │ │ ├── maven-wrapper.jar │ │ └── maven-wrapper.properties ├── build-dependencies.sh ├── build-dtest-jars.sh ├── build-shaded-dtest-jar-local.sh ├── functions.sh ├── mvnw └── relocate-dtest-dependencies.pom └── settings.gradle /.asf.yaml: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | # 18 | 19 | github: 20 | enabled_merge_buttons: 21 | squash: true 22 | merge: false 23 | rebase: true 24 | 25 | notifications: 26 | commits: commits@cassandra.apache.org 27 | issues: commits@cassandra.apache.org 28 | pullrequests: commits@cassandra.apache.org 29 | jira_options: link worklog 30 | -------------------------------------------------------------------------------- /NOTICE.txt: -------------------------------------------------------------------------------- 1 | Apache Cassandra Analytics 2 | Copyright 2023- The Apache Software Foundation 3 | 4 | This product includes software developed by The Apache Software 5 | Foundation (http://www.apache.org/). 6 | 7 | -------------------------------------------------------------------------------- /analytics-sidecar-client-common/src/main/java/org/apache/cassandra/sidecar/client/SidecarInstance.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.client; 20 | 21 | /** 22 | * Holds information about the Cassandra Sidecar Instance host and port 23 | */ 24 | public interface SidecarInstance 25 | { 26 | /** 27 | * @return the port where the Cassandra Sidecar instance is listening 28 | */ 29 | int port(); 30 | 31 | /** 32 | * @return the hostname where the Cassandra Sidecar instance is running 33 | */ 34 | String hostname(); 35 | } 36 | -------------------------------------------------------------------------------- /analytics-sidecar-client-common/src/main/java/org/apache/cassandra/sidecar/common/data/ConsistencyVerificationResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.common.data; 20 | 21 | /** 22 | * Verification result 23 | */ 24 | public enum ConsistencyVerificationResult 25 | { 26 | /** 27 | * the passed replicas have satisfied the consistency level 28 | */ 29 | SATISFIED, 30 | /** 31 | * no conclusion can be made yet 32 | */ 33 | PENDING, 34 | /** 35 | * the failed replicas have failed the consistency level 36 | */ 37 | FAILED, 38 | } 39 | -------------------------------------------------------------------------------- /analytics-sidecar-client-common/src/main/java/org/apache/cassandra/sidecar/common/data/OperationalJobStatus.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.common.data; 20 | 21 | /** 22 | * Encapsulates the states of the job lifecycle. 23 | * Operational jobs are the ones running on Cassandra, e.g. decommission, etc. 24 | */ 25 | public enum OperationalJobStatus 26 | { 27 | /** 28 | * The operational job is created 29 | */ 30 | CREATED, 31 | /** 32 | * The operational job is running on Cassandra 33 | */ 34 | RUNNING, 35 | /** 36 | * The operational job succeeds 37 | */ 38 | SUCCEEDED, 39 | /** 40 | * The operational job fails 41 | */ 42 | FAILED; 43 | 44 | public boolean isCompleted() 45 | { 46 | return this == SUCCEEDED || this == FAILED; 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /analytics-sidecar-client-common/src/main/java/org/apache/cassandra/sidecar/common/exceptions/RangeException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.common.exceptions; 20 | 21 | /** 22 | * Custom exception 23 | */ 24 | public class RangeException extends RuntimeException 25 | { 26 | public RangeException(String message) 27 | { 28 | super(message); 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /analytics-sidecar-client-common/src/main/java/org/apache/cassandra/sidecar/common/http/SidecarHttpHeaderNames.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.common.http; 20 | 21 | /** 22 | * Custom header names for sidecar 23 | */ 24 | public final class SidecarHttpHeaderNames 25 | { 26 | /** 27 | * {@code "cassandra-content-xxhash32"} 28 | */ 29 | public static final String CONTENT_XXHASH32 = "cassandra-content-xxhash32"; 30 | /** 31 | * {@code "cassandra-content-xxhash32-seed"} 32 | */ 33 | public static final String CONTENT_XXHASH32_SEED = "cassandra-content-xxhash32-seed"; 34 | 35 | private SidecarHttpHeaderNames() 36 | { 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /analytics-sidecar-client-common/src/main/java/org/apache/cassandra/sidecar/common/request/GossipHealthRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.common.request; 20 | 21 | import io.netty.handler.codec.http.HttpMethod; 22 | import org.apache.cassandra.sidecar.common.ApiEndpointsV1; 23 | import org.apache.cassandra.sidecar.common.response.HealthResponse; 24 | 25 | /** 26 | * Request to get gossip health 27 | */ 28 | public class GossipHealthRequest extends JsonRequest 29 | { 30 | /** 31 | * Constructs a request to GET gossip health 32 | */ 33 | public GossipHealthRequest() 34 | { 35 | super(ApiEndpointsV1.GOSSIP_HEALTH_ROUTE); 36 | } 37 | 38 | /** 39 | * {@inheritDoc} 40 | */ 41 | @Override 42 | public HttpMethod method() 43 | { 44 | return HttpMethod.GET; 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /analytics-sidecar-client-common/src/main/java/org/apache/cassandra/sidecar/common/request/ListCdcSegmentsRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package org.apache.cassandra.sidecar.common.request; 20 | 21 | import io.netty.handler.codec.http.HttpMethod; 22 | import org.apache.cassandra.sidecar.common.ApiEndpointsV1; 23 | import org.apache.cassandra.sidecar.common.response.ListCdcSegmentsResponse; 24 | 25 | /** 26 | * Represents a request for listing commit log files on an instance 27 | */ 28 | public class ListCdcSegmentsRequest extends JsonRequest 29 | { 30 | public ListCdcSegmentsRequest() 31 | { 32 | super(ApiEndpointsV1.LIST_CDC_SEGMENTS_ROUTE); 33 | } 34 | 35 | @Override 36 | public HttpMethod method() 37 | { 38 | return HttpMethod.GET; 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /analytics-sidecar-client-common/src/main/java/org/apache/cassandra/sidecar/common/request/NodeSettingsRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.common.request; 20 | 21 | import io.netty.handler.codec.http.HttpMethod; 22 | import org.apache.cassandra.sidecar.common.ApiEndpointsV1; 23 | import org.apache.cassandra.sidecar.common.response.NodeSettings; 24 | 25 | /** 26 | * Represents a request to retrieve the node settings 27 | */ 28 | public class NodeSettingsRequest extends JsonRequest 29 | { 30 | /** 31 | * Constructs a request to retrieve the node settings 32 | */ 33 | public NodeSettingsRequest() 34 | { 35 | super(ApiEndpointsV1.NODE_SETTINGS_ROUTE); 36 | } 37 | 38 | /** 39 | * {@inheritDoc} 40 | */ 41 | @Override 42 | public HttpMethod method() 43 | { 44 | return HttpMethod.GET; 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /analytics-sidecar-client-common/src/main/java/org/apache/cassandra/sidecar/common/request/ReportSchemaRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.common.request; 20 | 21 | import io.netty.handler.codec.http.HttpMethod; 22 | import org.apache.cassandra.sidecar.common.ApiEndpointsV1; 23 | import org.apache.cassandra.sidecar.common.response.HealthResponse; 24 | 25 | /** 26 | * A request to trigger an immediate, synchronous schema conversion 27 | * and report regardless of the periodic task schedule or status 28 | */ 29 | public class ReportSchemaRequest extends JsonRequest 30 | { 31 | public ReportSchemaRequest() 32 | { 33 | super(ApiEndpointsV1.REPORT_SCHEMA_ROUTE); 34 | } 35 | 36 | @Override 37 | public HttpMethod method() 38 | { 39 | return HttpMethod.PUT; 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /analytics-sidecar-client-common/src/main/java/org/apache/cassandra/sidecar/common/request/ResponseBytesDecoder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.common.request; 20 | 21 | import java.io.IOException; 22 | 23 | /** 24 | * Decode response bytes into given java type {@code } 25 | * @param expected java type 26 | */ 27 | public interface ResponseBytesDecoder 28 | { 29 | /** 30 | * Decodes the provided {@code bytes} to an instance of the type {@code } 31 | * 32 | * @param bytes the raw bytes of the response 33 | * @return the decoded instance for the given {@code bytes}, or null for null input 34 | * @throws IOException when the decoder is unable to decode successfully 35 | */ 36 | T decode(byte[] bytes) throws IOException; 37 | } 38 | -------------------------------------------------------------------------------- /analytics-sidecar-client-common/src/main/java/org/apache/cassandra/sidecar/common/request/Service.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package org.apache.cassandra.sidecar.common.request; 19 | 20 | import java.util.Locale; 21 | 22 | /** 23 | * "configs" table inside internal sidecar keyspace stores configs for Kafka and CDC. ServiceConfig identifies 24 | * the service of the configs. 25 | */ 26 | public enum Service 27 | { 28 | KAFKA, 29 | CDC; 30 | public final String serviceName; 31 | 32 | Service() 33 | { 34 | this.serviceName = this.toString().toLowerCase(Locale.ENGLISH); 35 | } 36 | 37 | public static Service withName(String serviceName) 38 | { 39 | return valueOf(serviceName.toUpperCase(Locale.ENGLISH)); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /analytics-sidecar-client-common/src/main/java/org/apache/cassandra/sidecar/common/request/UploadableRequest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.common.request; 20 | 21 | /** 22 | * A request that will upload the provided file to the service 23 | */ 24 | public interface UploadableRequest 25 | { 26 | /** 27 | * @return the path to the file to be uploaded 28 | */ 29 | String filename(); 30 | } 31 | -------------------------------------------------------------------------------- /analytics-sidecar-client-common/src/main/java/org/apache/cassandra/sidecar/common/request/data/Digest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.common.request.data; 20 | 21 | import java.util.Map; 22 | 23 | /** 24 | * Interface that represents a checksum digest 25 | */ 26 | public interface Digest 27 | { 28 | /** 29 | * @return headers to be used in the HTTP request 30 | */ 31 | Map headers(); 32 | 33 | /** 34 | * @return the string representation of the digest 35 | */ 36 | String value(); 37 | 38 | /** 39 | * @return the name of the digest's algorithm 40 | */ 41 | String algorithm(); 42 | } 43 | -------------------------------------------------------------------------------- /analytics-sidecar-client-common/src/test/java/org/apache/cassandra/sidecar/common/http/SidecarHttpResponseStatusTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.common.http; 20 | 21 | import org.junit.jupiter.api.Test; 22 | 23 | import static org.apache.cassandra.sidecar.common.http.SidecarHttpResponseStatus.CHECKSUM_MISMATCH; 24 | import static org.assertj.core.api.Assertions.assertThat; 25 | 26 | /** 27 | * Unit tests for {@link SidecarHttpResponseStatus} 28 | */ 29 | class SidecarHttpResponseStatusTest 30 | { 31 | 32 | @Test 33 | void testInvalidChecksum() 34 | { 35 | assertThat(CHECKSUM_MISMATCH.code()).isEqualTo(455); 36 | assertThat(CHECKSUM_MISMATCH.reasonPhrase()).isEqualTo("Checksum Mismatch"); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /analytics-sidecar-client/src/main/java/org/apache/cassandra/sidecar/client/SidecarInstancesProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.client; 20 | 21 | import java.util.List; 22 | 23 | /** 24 | * A class that provides the list of {@link SidecarInstance}s. This class allows for statically or dynamically 25 | * providing a list of instances. It is meant to support expansions and shrink of Cassandra clusters. 26 | */ 27 | public interface SidecarInstancesProvider 28 | { 29 | /** 30 | * @return the list of {@link SidecarInstance}s 31 | */ 32 | List instances(); 33 | } 34 | -------------------------------------------------------------------------------- /analytics-sidecar-client/src/main/java/org/apache/cassandra/sidecar/client/exception/ResourceNotFoundException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.client.exception; 20 | 21 | import org.apache.cassandra.sidecar.common.request.Request; 22 | 23 | /** 24 | * An exception raised when the HTTP resource is Not Found (A 404 Status Code response). 25 | */ 26 | public class ResourceNotFoundException extends RuntimeException 27 | { 28 | /** 29 | * Constructs a new exception for a request where the URI is not found on the remote server 30 | * 31 | * @param request the HTTP request 32 | */ 33 | public ResourceNotFoundException(Request request) 34 | { 35 | super(String.format("The resource for the request '%s' does not exist", request.requestURI())); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /analytics-sidecar-client/src/main/java/org/apache/cassandra/sidecar/client/retry/RetryAction.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.client.retry; 20 | 21 | /** 22 | * An action called when a request is retried 23 | */ 24 | public interface RetryAction 25 | { 26 | /** 27 | * Notify that the request must be retried with {@code retryDelayMillis}. 28 | * 29 | * @param attempts the number of attempts for this request 30 | * @param retryDelayMillis the delay between retries in milliseconds 31 | */ 32 | void retry(int attempts, long retryDelayMillis); 33 | } 34 | -------------------------------------------------------------------------------- /analytics-sidecar-client/src/main/java/org/apache/cassandra/sidecar/client/selection/InstanceSelectionPolicy.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.client.selection; 20 | 21 | import org.apache.cassandra.sidecar.client.SidecarInstance; 22 | 23 | /** 24 | * Defines the selection policy for an instance 25 | */ 26 | public interface InstanceSelectionPolicy extends Iterable 27 | { 28 | } 29 | -------------------------------------------------------------------------------- /analytics-sidecar-client/src/test/java/org.apache.cassandra.sidecar.client/SidecarInstanceImplTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.client; 20 | 21 | /** 22 | * Unit tests for the {@link SidecarInstanceImpl} class 23 | */ 24 | class SidecarInstanceImplTest extends SidecarInstanceTest 25 | { 26 | @Override 27 | protected SidecarInstance newInstance(String hostname, int port) 28 | { 29 | return new SidecarInstanceImpl(hostname, port); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /analytics-sidecar-client/src/testFixtures/java/org/apache/cassandra/sidecar/client/request/RequestTestParameters.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.client.request; 20 | 21 | import org.apache.cassandra.sidecar.client.RequestContext; 22 | 23 | /** 24 | * An interface defining the request testing for the client 25 | * 26 | * @param the type 27 | */ 28 | public interface RequestTestParameters 29 | { 30 | RequestContext.Builder specificRequest(RequestContext.Builder requestContextBuilder); 31 | 32 | String okResponseBody(); 33 | 34 | String expectedEndpointPath(); 35 | 36 | void validateResponse(T responseObject); 37 | 38 | default String serverErrorResponseBody() 39 | { 40 | return "{\"error\":\"some error\"}"; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /analytics-sidecar-client/src/testFixtures/resources/sstables/nb-1-big-TOC.txt: -------------------------------------------------------------------------------- 1 | Summary.db 2 | TOC.txt 3 | Statistics.db 4 | Filter.db 5 | Data.db 6 | CRC.db 7 | Digest.crc32 8 | Index.db 9 | -------------------------------------------------------------------------------- /analytics-sidecar-vertx-client-shaded/src/test/java/org/apache/cassandra/sidecar/client/test/LibraryTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.sidecar.client.test; 20 | 21 | import org.junit.jupiter.api.DisplayName; 22 | import org.junit.jupiter.api.Test; 23 | 24 | import o.a.c.sidecar.client.shaded.io.vertx.core.net.OpenSSLEngineOptions; 25 | 26 | import static org.junit.jupiter.api.Assertions.assertTrue; 27 | 28 | /** 29 | * Tests for the vertx-client-shaded project 30 | */ 31 | class LibraryTest 32 | { 33 | @DisplayName("Ensures that the shading is correct for the vertx-client-shaded project") 34 | @Test 35 | void openSslIsAvailable() 36 | { 37 | assertTrue(OpenSSLEngineOptions.isAvailable()); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/main/java/org/apache/cassandra/cdc/CdcEventTransformer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc; 21 | 22 | import org.apache.cassandra.cdc.msg.CdcEvent; 23 | 24 | public interface CdcEventTransformer 25 | { 26 | /** 27 | * Transform CdcEvent into another type. 28 | * 29 | * @param event the CdcEvent 30 | * @return transformed event of type `T` 31 | */ 32 | T transform(CdcEvent event); 33 | } 34 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/main/java/org/apache/cassandra/cdc/avro/msg/CdcEnvelope.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.avro.msg; 21 | 22 | import org.apache.avro.generic.GenericRecord; 23 | 24 | public class CdcEnvelope 25 | { 26 | // Contains the metadata. Structure is defined in cdc.avsc 27 | public final GenericRecord header; 28 | 29 | // Contains the actual change captured for a table 30 | public final GenericRecord payload; 31 | 32 | public CdcEnvelope(GenericRecord header, GenericRecord payload) 33 | { 34 | this.header = header; 35 | this.payload = payload; 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/main/java/org/apache/cassandra/cdc/kafka/KafkaCdcSerializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.kafka; 21 | 22 | import org.apache.cassandra.cdc.avro.CdcEventAvroEncoder; 23 | import org.apache.kafka.common.serialization.Serializer; 24 | 25 | public interface KafkaCdcSerializer extends Serializer 26 | { 27 | CdcEventAvroEncoder getTransformer(); 28 | } 29 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/main/java/org/apache/cassandra/cdc/kafka/KafkaOptions.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.kafka; 21 | 22 | import java.util.Map; 23 | 24 | public interface KafkaOptions 25 | { 26 | Map kafkaConfigs(); 27 | } 28 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/main/java/org/apache/cassandra/cdc/kafka/KafkaStats.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.kafka; 21 | 22 | import org.apache.cassandra.cdc.msg.CdcEvent; 23 | 24 | public interface KafkaStats 25 | { 26 | KafkaStats STUB = new KafkaStats() 27 | { 28 | }; 29 | 30 | default void registerKafkaPublishErrorKpi() 31 | { 32 | 33 | } 34 | 35 | default void reportJobFailure() 36 | { 37 | } 38 | 39 | default void reportKafkaPublishError() 40 | { 41 | 42 | } 43 | 44 | default void reportKafkaRecordTooLarge() 45 | { 46 | 47 | } 48 | 49 | default void changePublished(CdcEvent event) 50 | { 51 | 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/main/java/org/apache/cassandra/cdc/schemastore/PublishSchemaResult.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.schemastore; 21 | 22 | /** 23 | * Object representing the result of publishing a schema on a schema store. 24 | */ 25 | public class PublishSchemaResult 26 | { 27 | 28 | /** 29 | * The id of the schema that has been published. 30 | */ 31 | private final String schemaId; 32 | 33 | public PublishSchemaResult(String schemaId) 34 | { 35 | this.schemaId = schemaId; 36 | } 37 | 38 | public String getSchemaId() 39 | { 40 | return schemaId; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/main/java/org/apache/cassandra/cdc/schemastore/SchemaStorePublisherFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.schemastore; 21 | 22 | import org.apache.cassandra.cdc.kafka.KafkaOptions; 23 | import org.jetbrains.annotations.Nullable; 24 | 25 | public interface SchemaStorePublisherFactory 26 | { 27 | SchemaStorePublisherFactory DEFAULT = new SchemaStorePublisherFactory() 28 | { 29 | @Nullable 30 | @Override 31 | public TableSchemaPublisher buildPublisher(KafkaOptions kafkaOptions) 32 | { 33 | return null; 34 | } 35 | }; 36 | 37 | @Nullable 38 | TableSchemaPublisher buildPublisher(KafkaOptions kafkaOptions); 39 | } 40 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/main/java/org/apache/cassandra/cdc/schemastore/SchemaStoreStats.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.schemastore; 21 | 22 | public interface SchemaStoreStats 23 | { 24 | /** 25 | * Schema has been published. 26 | */ 27 | default void capturePublishedSchema() 28 | { 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/main/java/org/apache/cassandra/cdc/schemastore/TableSchemaPublisher.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.schemastore; 21 | 22 | import java.util.HashMap; 23 | 24 | /** 25 | * Interface representing a CDC schema publisher. 26 | */ 27 | public interface TableSchemaPublisher extends AutoCloseable 28 | { 29 | 30 | /** 31 | * Publishes the schema producing a result. 32 | * 33 | * @param schema A string containing a valid schema. 34 | * @param metadata All the needed metadata associated to the schema. 35 | * @return The publishing result. 36 | */ 37 | PublishSchemaResult publishSchema(String schema, SchemaPublishMetadata metadata); 38 | 39 | class SchemaPublishMetadata extends HashMap 40 | { 41 | 42 | } 43 | 44 | default void close() 45 | { 46 | 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/main/resources/table_schemas/example.avsc: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | { 21 | "type" : "record", 22 | "name" : "example", 23 | "namespace" : "foo.namespace", 24 | "doc" : "This is an example avro schema for a foo.namespace.example table", 25 | "fields" : [ { 26 | "name" : "id", 27 | "type" : [ { 28 | "type" : "string", 29 | "logicalType" : "uuid" 30 | }, "null" ], 31 | "doc" : "doc" 32 | }, { 33 | "name" : "details", 34 | "type" : [ "bytes", "null" ], 35 | "doc" : "doc" 36 | } ], 37 | "primary_keys" : [ "id" ], 38 | "partition_keys" : [ "id" ] 39 | } -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/test/resources/cql_schemas/test_ks.test_tbl1: -------------------------------------------------------------------------------- 1 | CREATE TABLE test_ks.test_tbl1 ( 2 | a text PRIMARY KEY, 3 | b text, 4 | c text, 5 | d text, 6 | e timestamp, 7 | f text, 8 | g bigint, 9 | h timestamp, 10 | i text, 11 | j boolean 12 | ) -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/test/resources/cql_schemas/test_ks.test_tbl2: -------------------------------------------------------------------------------- 1 | CREATE TABLE test_ks.test_tbl2 ( 2 | a text PRIMARY KEY, 3 | b text, 4 | c text, 5 | d text, 6 | e timestamp, 7 | f text, 8 | g bigint, 9 | h timestamp 10 | ) -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/test/resources/cql_schemas/test_ks.test_tbl3: -------------------------------------------------------------------------------- 1 | CREATE TABLE test_ks.test_tbl3 ( 2 | a uuid, 3 | b timeuuid, 4 | c timestamp static, 5 | d text static, 6 | e text static, 7 | f text, 8 | g uuid, 9 | h text, 10 | i text, 11 | PRIMARY KEY (a, b) 12 | ) WITH CLUSTERING ORDER BY (b DESC) -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/test/resources/table_schemas/test_date.avsc: -------------------------------------------------------------------------------- 1 | { 2 | "type" : "record", 3 | "name" : "date", 4 | "namespace" : "test.date", 5 | "doc" : "doc", 6 | "fields" : [ { 7 | "name" : "a", 8 | "type" : [ { 9 | "type": "int", 10 | "logicalType": "date" 11 | }, "null" ], 12 | "doc" : "doc" 13 | } ] 14 | } 15 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/test/resources/table_schemas/test_decimal.avsc: -------------------------------------------------------------------------------- 1 | { 2 | "type" : "record", 3 | "name" : "decimal", 4 | "namespace" : "test.decimal", 5 | "doc" : "doc", 6 | "fields" : [ { 7 | "name" : "a", 8 | "type" : [ { 9 | "type": "bytes", 10 | "logicalType": "decimal", 11 | "precision": 30, 12 | "scale": 8 13 | }, "null" ], 14 | "doc" : "doc" 15 | } ] 16 | } 17 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/test/resources/table_schemas/test_ks.test_tbl2.avsc: -------------------------------------------------------------------------------- 1 | { 2 | "type" : "record", 3 | "name" : "test_tbl2", 4 | "namespace" : "test_ks.test_tbl2", 5 | "doc" : "doc", 6 | "fields" : [ { 7 | "name" : "a", 8 | "type" : [ { 9 | "type" : "string", 10 | "cqlType" : "text" 11 | }, "null" ], 12 | "doc" : "doc" 13 | }, { 14 | "name" : "b", 15 | "type" : [ { 16 | "type" : "string", 17 | "cqlType" : "text" 18 | }, "null" ], 19 | "doc" : "doc" 20 | }, { 21 | "name" : "c", 22 | "type" : [ { 23 | "type" : "string", 24 | "cqlType" : "text" 25 | }, "null" ], 26 | "doc" : "doc" 27 | }, { 28 | "name" : "d", 29 | "type" : [ { 30 | "type" : "string", 31 | "cqlType" : "text" 32 | }, "null" ], 33 | "doc" : "doc" 34 | }, { 35 | "name" : "e", 36 | "type" : [ { 37 | "type" : "long", 38 | "logicalType" : "timestamp-micros", 39 | "cqlType" : "timestamp" 40 | }, "null" ], 41 | "doc" : "doc" 42 | }, { 43 | "name" : "f", 44 | "type" : [ { 45 | "type" : "string", 46 | "cqlType" : "text" 47 | }, "null" ], 48 | "doc" : "doc" 49 | }, { 50 | "name" : "g", 51 | "type" : [ { 52 | "type" : "long", 53 | "cqlType" : "bigint" 54 | }, "null" ], 55 | "doc" : "doc" 56 | }, { 57 | "name" : "h", 58 | "type" : [ { 59 | "type" : "long", 60 | "logicalType" : "timestamp-micros", 61 | "cqlType" : "timestamp" 62 | }, "null" ], 63 | "doc" : "doc" 64 | } ], 65 | "primary_keys" : [ "a" ], 66 | "partition_keys" : [ "a" ], 67 | "clustering_keys" : [ ], 68 | "static_columns":[] 69 | } 70 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/test/resources/table_schemas/test_tbl_basic.avsc: -------------------------------------------------------------------------------- 1 | { 2 | "type" : "record", 3 | "name" : "test_tbl_basic", 4 | "namespace" : "test_ks.test_tbl_basic", 5 | "doc" : "doc", 6 | "fields" : [ { 7 | "name" : "a", 8 | "type" : [ "int", "null" ], 9 | "doc" : "doc" 10 | }, { 11 | "name" : "b", 12 | "type" : [ "int", "null" ], 13 | "doc" : "doc" 14 | }, { 15 | "name" : "c", 16 | "type" : [ "int", "null" ], 17 | "doc" : "doc" 18 | }, { 19 | "name" : "e", 20 | "type" : [ "int", "null" ], 21 | "doc" : "doc" 22 | }, { 23 | "name" : "f", 24 | "type" : [ "int", "null" ], 25 | "doc" : "doc" 26 | } ], 27 | "primary_keys" : [ "a" ], 28 | "partition_keys" : [ "a" ] 29 | } 30 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/test/resources/table_schemas/test_tbl_binary.avsc: -------------------------------------------------------------------------------- 1 | { 2 | "type" : "record", 3 | "name" : "test_tbl_binary", 4 | "namespace" : "test_ks.test_tbl_binary", 5 | "doc" : "doc", 6 | "fields" : [ { 7 | "name" : "a", 8 | "type" : [ "int", "null" ], 9 | "doc" : "doc" 10 | }, { 11 | "name" : "b", 12 | "type" : [ "bytes", "null" ], 13 | "doc" : "doc" 14 | }, { 15 | "name" : "c", 16 | "type" : [ "bytes", "null" ], 17 | "doc" : "doc" 18 | } ], 19 | "primary_keys" : [ "a" ], 20 | "partition_keys" : [ "a" ] 21 | } 22 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/test/resources/table_schemas/test_time-micros.avsc: -------------------------------------------------------------------------------- 1 | { 2 | "type" : "record", 3 | "name" : "time_micros", 4 | "namespace" : "test.time_micros", 5 | "doc" : "doc", 6 | "fields" : [ { 7 | "name" : "a", 8 | "type" : [ { 9 | "type": "long", 10 | "logicalType": "time-micros" 11 | }, "null" ], 12 | "doc" : "doc" 13 | } ] 14 | } 15 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/test/resources/table_schemas/test_timestamp-micros.avsc: -------------------------------------------------------------------------------- 1 | { 2 | "type" : "record", 3 | "name" : "timestamp_micros", 4 | "namespace" : "test.timestamp_micros", 5 | "doc" : "doc", 6 | "fields" : [ { 7 | "name" : "a", 8 | "type" : [ { 9 | "type": "long", 10 | "logicalType": "timestamp-micros" 11 | }, "null" ], 12 | "doc" : "doc" 13 | } ] 14 | } 15 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc-codec/src/test/resources/table_schemas/test_uuid.avsc: -------------------------------------------------------------------------------- 1 | { 2 | "type" : "record", 3 | "name" : "uuid", 4 | "namespace" : "test.uuid", 5 | "doc" : "doc", 6 | "fields" : [ { 7 | "name" : "a", 8 | "type" : [ { 9 | "type": "string", 10 | "logicalType": "uuid" 11 | }, "null" ], 12 | "doc" : "doc" 13 | } ] 14 | } 15 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc/src/main/java/org/apache/cassandra/cdc/api/EventConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.api; 21 | 22 | import java.util.function.Consumer; 23 | 24 | import org.apache.cassandra.cdc.msg.CdcEvent; 25 | 26 | public interface EventConsumer extends Consumer 27 | { 28 | } 29 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc/src/main/java/org/apache/cassandra/cdc/api/MessageConverter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.api; 21 | 22 | import org.apache.cassandra.cdc.msg.CdcEvent; 23 | import org.apache.cassandra.cdc.msg.RangeTombstone; 24 | import org.apache.cassandra.cdc.msg.Value; 25 | 26 | public interface MessageConverter 27 | { 28 | T toCdcMessage(Value value); 29 | 30 | R toCdcMessage(RangeTombstone value); 31 | 32 | E toCdcMessage(CdcEvent event); 33 | } 34 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc/src/main/java/org/apache/cassandra/cdc/api/SchemaSupplier.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.api; 21 | 22 | import java.util.Set; 23 | import java.util.concurrent.CompletableFuture; 24 | 25 | import org.apache.cassandra.spark.data.CqlTable; 26 | 27 | /** 28 | * Supplies all CDC enabled tables 29 | */ 30 | public interface SchemaSupplier 31 | { 32 | CompletableFuture> getCdcEnabledTables(); 33 | } 34 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc/src/main/java/org/apache/cassandra/cdc/api/TokenRangeSupplier.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.api; 21 | 22 | import java.util.function.Supplier; 23 | 24 | import org.apache.cassandra.bridge.TokenRange; 25 | 26 | public interface TokenRangeSupplier extends Supplier 27 | { 28 | } 29 | -------------------------------------------------------------------------------- /cassandra-analytics-cdc/src/test/java/org/apache/cassandra/cdc/CdcWriter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc; 21 | 22 | import java.util.Map; 23 | import java.util.function.BiConsumer; 24 | 25 | import org.apache.cassandra.spark.data.CqlTable; 26 | import org.apache.cassandra.spark.utils.test.TestSchema; 27 | 28 | public interface CdcWriter 29 | { 30 | void write(CdcTester tester, Map rows, BiConsumer writer); 31 | 32 | default CqlTable cqlTable(CdcTester tester) 33 | { 34 | return tester.cqlTable; 35 | } 36 | } 37 | 38 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/bridge/BigNumberConfig.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.bridge; 21 | 22 | public interface BigNumberConfig 23 | { 24 | BigNumberConfig DEFAULT = new BigNumberConfig() 25 | { 26 | public int bigIntegerPrecision() 27 | { 28 | return 38; 29 | } 30 | 31 | public int bigIntegerScale() 32 | { 33 | return 0; 34 | } 35 | 36 | public int bigDecimalPrecision() 37 | { 38 | return 38; 39 | } 40 | 41 | public int bigDecimalScale() 42 | { 43 | return 19; 44 | } 45 | }; 46 | 47 | int bigIntegerPrecision(); 48 | 49 | int bigIntegerScale(); 50 | 51 | int bigDecimalPrecision(); 52 | 53 | int bigDecimalScale(); 54 | } 55 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/bridge/SSTableSummary.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.bridge; 21 | 22 | import java.math.BigInteger; 23 | 24 | public class SSTableSummary 25 | { 26 | public final BigInteger firstToken; 27 | public final BigInteger lastToken; 28 | public final String sstableId; 29 | 30 | public SSTableSummary(BigInteger firstToken, BigInteger lastToken, String sstableId) 31 | { 32 | this.firstToken = firstToken; 33 | this.lastToken = lastToken; 34 | this.sstableId = sstableId; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/cdc/api/CommitLogInstance.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.api; 21 | 22 | public interface CommitLogInstance 23 | { 24 | void start(); 25 | 26 | void stop(); 27 | 28 | void clear(); 29 | 30 | void add(Mutation mutation); 31 | 32 | void sync(); 33 | } 34 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/cdc/api/Mutation.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.api; 21 | 22 | public interface Mutation 23 | { 24 | 25 | } 26 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/cdc/api/RangeTombstoneData.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.api; 21 | 22 | public class RangeTombstoneData 23 | { 24 | public final Bound open; 25 | public final Bound close; 26 | 27 | public RangeTombstoneData(Bound open, Bound close) 28 | { 29 | this.open = open; 30 | this.close = close; 31 | } 32 | 33 | public static class Bound 34 | { 35 | public final Object[] values; 36 | public final boolean inclusive; 37 | 38 | public Bound(Object[] values, boolean inclusive) 39 | { 40 | this.values = values; 41 | this.inclusive = inclusive; 42 | } 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/cdc/scanner/CdcStreamScanner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc.scanner; 21 | 22 | import org.apache.cassandra.cdc.msg.CdcEvent; 23 | import org.apache.cassandra.cdc.state.CdcState; 24 | import org.apache.cassandra.spark.reader.StreamScanner; 25 | 26 | public interface CdcStreamScanner extends StreamScanner 27 | { 28 | CdcState endState(); 29 | } 30 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/common/model/BulkFeatures.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.common.model; 21 | 22 | @SuppressWarnings("unused") // Part of this library's API used by the consumers 23 | public final class BulkFeatures 24 | { 25 | public static final String BULK_WRITER = "cassandra-spark-bulk-writer"; 26 | public static final String BULK_READER = "cassandra-spark-bulk-reader"; 27 | 28 | private BulkFeatures() 29 | { 30 | throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated"); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/common/model/NodeState.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.common.model; 21 | 22 | public enum NodeState 23 | { 24 | NORMAL(false), 25 | JOINING(true), 26 | LEAVING(true), 27 | MOVING(true), 28 | REPLACING(true); // deprecated state. Remove once Sidecar's TokenRangeReplicasResponse gets rid of it 29 | 30 | public final boolean isPending; 31 | 32 | NodeState(boolean isPending) 33 | { 34 | this.isPending = isPending; 35 | } 36 | 37 | public static NodeState fromNameIgnoreCase(String name) 38 | { 39 | String uppercase = name.toUpperCase(); 40 | return valueOf(uppercase); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/common/model/NodeStatus.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.common.model; 21 | 22 | public enum NodeStatus 23 | { 24 | UP, 25 | DOWN, 26 | UNKNOWN; 27 | 28 | public static NodeStatus fromNameIgnoreCase(String name) 29 | { 30 | String uppercase = name.toUpperCase(); 31 | try 32 | { 33 | return valueOf(uppercase); 34 | } 35 | catch (Exception ex) 36 | { 37 | // default to UNKNOWN 38 | return UNKNOWN; 39 | } 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/common/stats/JobStatsPublisher.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.common.stats; 21 | 22 | import java.util.Map; 23 | 24 | /** 25 | * Interface to provide functionality to report Spark Job Statistics and/or properties 26 | * that can optionally be instrumented. The default implementation merely logs these 27 | * stats at the end of the job. 28 | */ 29 | public interface JobStatsPublisher 30 | { 31 | /** 32 | * Publish the job attributes to be persisted and summarized 33 | * 34 | * @param stats the stats to publish 35 | */ 36 | void publish(Map stats); 37 | } 38 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/common/stats/LogStatsPublisher.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.common.stats; 21 | 22 | import java.util.Map; 23 | 24 | import org.slf4j.Logger; 25 | import org.slf4j.LoggerFactory; 26 | 27 | /** 28 | * Implementation of {@link JobStatsPublisher} that is used to publish job statistics during the 29 | * Spark job execution. This implementation logs the stats when published. 30 | */ 31 | public class LogStatsPublisher implements JobStatsPublisher 32 | { 33 | private static final Logger LOGGER = LoggerFactory.getLogger(LogStatsPublisher.class); 34 | 35 | @Override 36 | public void publish(Map stats) 37 | { 38 | LOGGER.info("Job Stats: {}", stats); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/DefaultSizing.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data; 21 | 22 | /** 23 | * Default sizing based on the number of cores option 24 | */ 25 | public class DefaultSizing implements Sizing 26 | { 27 | private final int numCores; 28 | 29 | /** 30 | * Constructs a new object with the given {@code numCores} 31 | * 32 | * @param numCores the number of cores for the execution 33 | */ 34 | public DefaultSizing(int numCores) 35 | { 36 | this.numCores = numCores; 37 | } 38 | 39 | /** 40 | * {@inheritDoc} 41 | */ 42 | @Override 43 | public int getEffectiveNumberOfCores() 44 | { 45 | return numCores; 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/Sizing.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data; 21 | 22 | /** 23 | * Sizing options to determine the number of cores 24 | */ 25 | public interface Sizing 26 | { 27 | 28 | /** 29 | * Returns the effective number of executor cores to use in the spark application 30 | * 31 | * @return the effective number of executor cores to use in the spark application 32 | */ 33 | int getEffectiveNumberOfCores(); 34 | } 35 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/data/model/TokenOwner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.model; 21 | 22 | /** 23 | * Token owner owns a token 24 | */ 25 | public interface TokenOwner 26 | { 27 | /** 28 | * @return the token it owns 29 | */ 30 | String token(); 31 | } 32 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/exception/ConsistencyNotSatisfiedException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.exception; 21 | 22 | /** 23 | * Consistency cannot be satisfied by the bulk operations, i.e. read or write 24 | */ 25 | public class ConsistencyNotSatisfiedException extends AnalyticsException 26 | { 27 | private static final long serialVersionUID = 992947698403422384L; 28 | 29 | public ConsistencyNotSatisfiedException(String message) 30 | { 31 | super(message); 32 | } 33 | 34 | public ConsistencyNotSatisfiedException(String message, Throwable cause) 35 | { 36 | super(message, cause); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/exception/S3ApiCallException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.exception; 21 | 22 | /** 23 | * Exception due to AWS S3 Api call failure 24 | */ 25 | public class S3ApiCallException extends AnalyticsException 26 | { 27 | private static final long serialVersionUID = -2763574614716368281L; 28 | 29 | public S3ApiCallException(String message, Throwable cause) 30 | { 31 | super(message, cause); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/exception/SidecarApiCallException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.exception; 21 | 22 | /** 23 | * Exception due to Cassandra Sidecar Api call failure 24 | */ 25 | public class SidecarApiCallException extends AnalyticsException 26 | { 27 | private static final long serialVersionUID = 3304206898661966469L; 28 | 29 | public SidecarApiCallException(String message) 30 | { 31 | super(message); 32 | } 33 | 34 | public SidecarApiCallException(String message, Throwable cause) 35 | { 36 | super(message, cause); 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/exception/UnsupportedAnalyticsOperationException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.exception; 21 | 22 | /** 23 | * Thrown to indicate that the requested operation is not supported by Cassandra Analytics 24 | */ 25 | public class UnsupportedAnalyticsOperationException extends AnalyticsException 26 | { 27 | private static final long serialVersionUID = 2032833657307034708L; 28 | 29 | public UnsupportedAnalyticsOperationException(String message) 30 | { 31 | super(message); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/reader/EmptyStreamScanner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.reader; 21 | 22 | public class EmptyStreamScanner implements StreamScanner 23 | { 24 | public static final EmptyStreamScanner INSTANCE = new EmptyStreamScanner(); 25 | 26 | @Override 27 | public RowData data() 28 | { 29 | return null; 30 | } 31 | 32 | @Override 33 | public boolean next() 34 | { 35 | return false; 36 | } 37 | 38 | @Override 39 | public void advanceToNextColumn() 40 | { 41 | } 42 | 43 | @Override 44 | public boolean hasMoreColumns() 45 | { 46 | return false; 47 | } 48 | 49 | @Override 50 | public void close() 51 | { 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/reader/IndexConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.reader; 21 | 22 | import java.util.function.Consumer; 23 | 24 | public interface IndexConsumer extends Consumer 25 | { 26 | void onFailure(Throwable t); 27 | 28 | void onFinished(long runtimeNanos); 29 | } 30 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/reader/common/ChunkCorruptException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.reader.common; 21 | 22 | import java.io.IOException; 23 | 24 | public class ChunkCorruptException extends IOException 25 | { 26 | public ChunkCorruptException(String message) 27 | { 28 | super(message); 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/reader/common/IIndexReader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.reader.common; 21 | 22 | import org.apache.cassandra.spark.reader.SparkSSTableReader; 23 | 24 | public interface IIndexReader extends SparkSSTableReader 25 | { 26 | } 27 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/sparksql/Cell.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.sparksql; 21 | 22 | public class Cell 23 | { 24 | public final Object[] values; 25 | public final int position; 26 | public final boolean isNewRow; 27 | public final long timestamp; 28 | 29 | Cell(Object[] values, int position, boolean isNewRow, long timestamp) 30 | { 31 | this.values = values; 32 | this.position = position; 33 | this.isNewRow = isNewRow; 34 | this.timestamp = timestamp; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/sparksql/NoMatchFoundException.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.sparksql; 21 | 22 | public class NoMatchFoundException extends Exception 23 | { 24 | public NoMatchFoundException() 25 | { 26 | } 27 | 28 | public NoMatchFoundException(String message) 29 | { 30 | super(message); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/TimeUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.utils; 21 | 22 | import java.util.concurrent.TimeUnit; 23 | 24 | public class TimeUtils 25 | { 26 | 27 | private TimeUtils() 28 | { 29 | 30 | } 31 | public static long nowMicros() 32 | { 33 | return TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis()); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/spark/utils/streaming/CassandraFile.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.utils.streaming; 21 | 22 | /** 23 | * High-level interface representing a type of Cassandra file, such as SSTable or CommitLog. 24 | */ 25 | public interface CassandraFile 26 | { 27 | } 28 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/main/java/org/apache/cassandra/util/CompressionUtil.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.util; 21 | 22 | import java.io.IOException; 23 | import java.nio.ByteBuffer; 24 | 25 | public interface CompressionUtil 26 | { 27 | ByteBuffer compress(byte[] ar) throws IOException; 28 | 29 | ByteBuffer compress(ByteBuffer input) throws IOException; 30 | 31 | ByteBuffer uncompress(byte[] bytes) throws IOException; 32 | 33 | ByteBuffer uncompress(ByteBuffer input) throws IOException; 34 | } 35 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/test/java/org/apache/cassandra/bridge/type/InternalDurationTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.bridge.type; 21 | 22 | import org.junit.jupiter.api.Test; 23 | 24 | import static org.junit.jupiter.api.Assertions.assertEquals; 25 | 26 | public class InternalDurationTest 27 | { 28 | @Test 29 | public void testDurationToString() 30 | { 31 | InternalDuration duration = new InternalDuration(1, 2, 123456789123L); 32 | assertEquals("mo1d2ns123456789123", duration.toString()); 33 | 34 | duration = new InternalDuration(0, -3, 987L); 35 | assertEquals("-mo0d3ns987", duration.toString()); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/test/resources/secrets/fakecerts/client-keystore-password: -------------------------------------------------------------------------------- 1 | cassandra-analytics 2 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/test/resources/secrets/fakecerts/client-keystore.p12: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-common/src/test/resources/secrets/fakecerts/client-keystore.p12 -------------------------------------------------------------------------------- /cassandra-analytics-common/src/test/resources/secrets/fakecerts/client-truststore-password: -------------------------------------------------------------------------------- 1 | cassandra-analytics 2 | -------------------------------------------------------------------------------- /cassandra-analytics-common/src/test/resources/secrets/fakecerts/client-truststore.jks: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-common/src/test/resources/secrets/fakecerts/client-truststore.jks -------------------------------------------------------------------------------- /cassandra-analytics-core-example/src/main/resources/keystore-private.p12: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core-example/src/main/resources/keystore-private.p12 -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CancelJobEvent.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.bulkwriter; 21 | 22 | /** 23 | * A simple data structure to describe an event that leads to job cancellation. 24 | * It contains the reason of cancellation and optionally the cause 25 | */ 26 | public class CancelJobEvent 27 | { 28 | public final Throwable exception; 29 | public final String reason; 30 | 31 | public CancelJobEvent(String reason) 32 | { 33 | this.reason = reason; 34 | this.exception = null; 35 | } 36 | 37 | public CancelJobEvent(String reason, Throwable throwable) 38 | { 39 | this.reason = reason; 40 | this.exception = throwable; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/DigestAlgorithmSupplier.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.bulkwriter; 21 | 22 | import java.io.Serializable; 23 | import java.util.function.Supplier; 24 | 25 | import org.apache.cassandra.spark.utils.DigestAlgorithm; 26 | 27 | /** 28 | * An interface that defines a {@link DigestAlgorithm} for a concrete digest type 29 | */ 30 | public interface DigestAlgorithmSupplier extends Supplier, Serializable 31 | { 32 | } 33 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SchemaInfo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.bulkwriter; 21 | 22 | import java.io.Serializable; 23 | import java.util.Set; 24 | 25 | import org.jetbrains.annotations.NotNull; 26 | 27 | public interface SchemaInfo extends Serializable 28 | { 29 | TableSchema getTableSchema(); 30 | 31 | @NotNull 32 | Set getUserDefinedTypeStatements(); 33 | } 34 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/TableInfoProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.bulkwriter; 21 | 22 | import java.util.List; 23 | 24 | import org.apache.cassandra.spark.common.schema.ColumnType; 25 | import org.apache.cassandra.spark.data.CqlField; 26 | 27 | public interface TableInfoProvider 28 | { 29 | CqlField.CqlType getColumnType(String columnName); 30 | 31 | List> getPartitionKeyTypes(); 32 | 33 | boolean columnExists(String columnName); 34 | 35 | List getPartitionKeyColumnNames(); 36 | 37 | String getCreateStatement(); 38 | 39 | List getPrimaryKeyColumnNames(); 40 | 41 | String getName(); 42 | 43 | String getKeyspaceName(); 44 | 45 | boolean hasSecondaryIndex(); 46 | 47 | List getColumnNames(); 48 | } 49 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/WriteMode.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.bulkwriter; 21 | 22 | public enum WriteMode 23 | { 24 | INSERT, 25 | DELETE_PARTITION 26 | } 27 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/WriterOption.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.bulkwriter; 21 | 22 | public interface WriterOption 23 | { 24 | String name(); 25 | } 26 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/Digest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.common; 21 | 22 | /** 23 | * Interface that represents the computed digest 24 | */ 25 | public interface Digest 26 | { 27 | /** 28 | * @return the string representation of the digest 29 | */ 30 | String value(); 31 | 32 | /** 33 | * @return the digest translated to Sidecar digest 34 | */ 35 | o.a.c.sidecar.client.shaded.common.request.data.Digest toSidecarDigest(); 36 | } 37 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/BooleanType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.common.schema; 21 | 22 | import java.nio.ByteBuffer; 23 | 24 | public class BooleanType implements ColumnType 25 | { 26 | private Boolean getBoolean(ByteBuffer bytes) 27 | { 28 | byte value = bytes.get(bytes.position()); 29 | return value != 0; 30 | } 31 | 32 | @Override 33 | public Boolean parseColumn(ByteBuffer buffer, int length) 34 | { 35 | byte[] value = new byte[length]; 36 | buffer.get(value, 0, length); 37 | return getBoolean(ByteBuffer.wrap(value)); 38 | } 39 | 40 | @Override 41 | public ByteBuffer serialize(Boolean value) 42 | { 43 | return ByteBuffer.allocate(1).put(0, (byte) (value ? 1 : 0)); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/DoubleType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.common.schema; 21 | 22 | import java.nio.ByteBuffer; 23 | 24 | public class DoubleType implements ColumnType 25 | { 26 | private Double getDouble(ByteBuffer bytes) 27 | { 28 | return bytes.getDouble(bytes.position()); 29 | } 30 | 31 | @Override 32 | public Double parseColumn(ByteBuffer buffer, int length) 33 | { 34 | byte[] value = new byte[length]; 35 | buffer.get(value, 0, length); 36 | return getDouble(ByteBuffer.wrap(value)); 37 | } 38 | 39 | @Override 40 | public ByteBuffer serialize(Double value) 41 | { 42 | return ByteBuffer.allocate(8).putDouble(0, value); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/IntegerType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.common.schema; 21 | 22 | import java.nio.ByteBuffer; 23 | 24 | public class IntegerType implements ColumnType 25 | { 26 | 27 | public static final int TYPE_SIZE = Integer.SIZE / Byte.SIZE; 28 | 29 | @Override 30 | public Integer parseColumn(ByteBuffer buffer, int length) 31 | { 32 | assert length == TYPE_SIZE; 33 | return buffer.getInt(); 34 | } 35 | 36 | @Override 37 | public ByteBuffer serialize(Integer value) 38 | { 39 | return ByteBuffer.allocate(TYPE_SIZE).putInt(0, value); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/LongType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.common.schema; 21 | 22 | import java.nio.ByteBuffer; 23 | 24 | public class LongType implements ColumnType 25 | { 26 | public static final int TYPE_SIZE = Long.SIZE / Byte.SIZE; 27 | 28 | @Override 29 | public Long parseColumn(ByteBuffer buffer, int length) 30 | { 31 | assert length == TYPE_SIZE; 32 | return buffer.getLong(); 33 | } 34 | 35 | @Override 36 | public ByteBuffer serialize(Long value) 37 | { 38 | return ByteBuffer.allocate(TYPE_SIZE).putLong(0, value); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/SetType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.common.schema; 21 | 22 | import java.nio.ByteBuffer; 23 | import java.util.List; 24 | 25 | public class SetType extends CollectionType 26 | { 27 | public final ColumnType elementType; 28 | 29 | public SetType(ColumnType elementType) 30 | { 31 | this.elementType = elementType; 32 | } 33 | 34 | @Override 35 | public T parseCollectionColumn(ByteBuffer colNameSuffix, ByteBuffer colValue) 36 | { 37 | return ColumnUtil.getField(colNameSuffix, elementType); 38 | } 39 | 40 | @Override 41 | public List finaliseCollection(List entryList) 42 | { 43 | return entryList; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/StringType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.common.schema; 21 | 22 | import java.nio.ByteBuffer; 23 | import java.nio.charset.StandardCharsets; 24 | 25 | public class StringType implements ColumnType 26 | { 27 | @Override 28 | public String parseColumn(ByteBuffer buffer, int length) 29 | { 30 | byte[] value = new byte[length]; 31 | buffer.get(value, 0, length); 32 | return new String(value, StandardCharsets.UTF_8); 33 | } 34 | 35 | @Override 36 | public ByteBuffer serialize(String value) 37 | { 38 | return ByteBuffer.wrap(value.getBytes(StandardCharsets.UTF_8)); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/StringUuidType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.common.schema; 21 | 22 | import java.nio.ByteBuffer; 23 | import java.util.UUID; 24 | 25 | public class StringUuidType implements ColumnType 26 | { 27 | @Override 28 | public String parseColumn(ByteBuffer buffer, int length) 29 | { 30 | return ColumnTypes.UUID.parseColumn(buffer, length).toString(); 31 | } 32 | 33 | @Override 34 | public ByteBuffer serialize(String valueStr) 35 | { 36 | UUID value = UUID.fromString(valueStr); 37 | return ColumnTypes.UUID.serialize(value); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/DigestAlgorithm.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.utils; 21 | 22 | import java.io.IOException; 23 | import java.nio.file.Path; 24 | 25 | import org.apache.cassandra.spark.common.Digest; 26 | 27 | /** 28 | * Interface that computes a {@link Digest} 29 | */ 30 | public interface DigestAlgorithm 31 | { 32 | /** 33 | * Calculates the {@link Digest} for the given file in the {@code path}. 34 | * 35 | * @param path the path of the file 36 | * @return the calculated digest for the given {@code path} 37 | * @throws IOException when an error occurs while reading the file or calculating the digest 38 | */ 39 | Digest calculateFileDigest(Path path) throws IOException; 40 | } 41 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/validation/BulkWriterKeyStoreValidation.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.validation; 21 | 22 | import org.apache.cassandra.spark.bulkwriter.BulkSparkConf; 23 | import org.jetbrains.annotations.NotNull; 24 | 25 | public class BulkWriterKeyStoreValidation extends KeyStoreValidation 26 | { 27 | public BulkWriterKeyStoreValidation(@NotNull BulkSparkConf configuration) 28 | { 29 | super( 30 | configuration.hasKeystoreAndKeystorePassword(), 31 | configuration.getKeyStoreTypeOrDefault(), 32 | configuration.getKeyStorePassword() == null ? null : configuration.getKeyStorePassword().toCharArray(), 33 | configuration::getKeyStore 34 | ); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/validation/BulkWriterTrustStoreValidation.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.validation; 21 | 22 | import org.apache.cassandra.spark.bulkwriter.BulkSparkConf; 23 | import org.jetbrains.annotations.NotNull; 24 | 25 | public class BulkWriterTrustStoreValidation extends TrustStoreValidation 26 | { 27 | public BulkWriterTrustStoreValidation(@NotNull BulkSparkConf configuration) 28 | { 29 | super( 30 | configuration.hasTruststoreAndTruststorePassword(), 31 | configuration.getTrustStoreTypeOrDefault(), 32 | configuration.getTrustStorePasswordOrDefault().toCharArray(), 33 | configuration::getTrustStore 34 | ); 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/validation/StartupValidatable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.validation; 21 | 22 | /** 23 | * An interface for a class that requires and can perform startup validation using {@link StartupValidator} 24 | */ 25 | public interface StartupValidatable 26 | { 27 | /** 28 | * Performs startup validation using {@link StartupValidator} with currently registered {@link StartupValidation}s, 29 | * throws a {@link RuntimeException} if any violations are found, 30 | * needs to be invoked once per execution before any actual work is started 31 | */ 32 | void startupValidate(); 33 | } 34 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister: -------------------------------------------------------------------------------- 1 | org.apache.cassandra.spark.sparksql.CassandraDataSource 2 | org.apache.cassandra.spark.sparksql.CassandraDataSink 3 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraInputPartition.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.sparksql; 21 | 22 | import org.apache.spark.sql.connector.read.InputPartition; 23 | 24 | class CassandraInputPartition implements InputPartition 25 | { 26 | private final int partitionId; 27 | 28 | CassandraInputPartition(int partitionId) 29 | { 30 | this.partitionId = partitionId; 31 | } 32 | 33 | public int getPartitionId() 34 | { 35 | return partitionId; 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/LocalDataSource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.sparksql; 21 | 22 | import org.apache.cassandra.spark.data.DataLayer; 23 | import org.apache.cassandra.spark.data.LocalDataLayer; 24 | import org.apache.spark.sql.util.CaseInsensitiveStringMap; 25 | import org.jetbrains.annotations.NotNull; 26 | 27 | @SuppressWarnings("unused") 28 | public class LocalDataSource extends CassandraTableProvider 29 | { 30 | @Override 31 | @NotNull 32 | public String shortName() 33 | { 34 | return "localsstabledatasource"; 35 | } 36 | 37 | @Override 38 | @NotNull 39 | public DataLayer getDataLayer(@NotNull CaseInsensitiveStringMap options) 40 | { 41 | return LocalDataLayer.from(options); 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/LocalPartitionSizeSource.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.sparksql; 21 | 22 | import org.apache.cassandra.spark.data.DataLayer; 23 | import org.apache.cassandra.spark.data.LocalDataLayer; 24 | import org.apache.spark.sql.util.CaseInsensitiveStringMap; 25 | 26 | public class LocalPartitionSizeSource extends PartitionSizeTableProvider 27 | { 28 | @Override 29 | public String shortName() 30 | { 31 | return "localpartitionsizesource"; 32 | } 33 | 34 | @Override 35 | public DataLayer getDataLayer(CaseInsensitiveStringMap options) 36 | { 37 | return LocalDataLayer.from(options); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/java/org/apache/cassandra/clients/SidecarNativeLibrariesTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.clients; 21 | 22 | import org.junit.jupiter.api.DisplayName; 23 | import org.junit.jupiter.api.Test; 24 | 25 | import o.a.c.sidecar.client.shaded.io.vertx.core.net.OpenSSLEngineOptions; 26 | 27 | import static org.junit.jupiter.api.Assertions.assertTrue; 28 | 29 | /** 30 | * Unit tests to ensure that Sidecar client native libraries are loaded correctly 31 | */ 32 | public class SidecarNativeLibrariesTest 33 | { 34 | @DisplayName("Ensures that the shading is correct for the vertx-client-shaded project") 35 | @Test 36 | void openSslIsAvailable() 37 | { 38 | assertTrue(OpenSSLEngineOptions.isAvailable()); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/TestRunnable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark; 21 | 22 | import java.io.IOException; 23 | import java.nio.file.Path; 24 | 25 | import org.apache.cassandra.bridge.CassandraBridge; 26 | import org.apache.cassandra.spark.data.partitioner.Partitioner; 27 | 28 | public interface TestRunnable 29 | { 30 | void run(Partitioner partitioner, Path directory, CassandraBridge bridge) throws IOException; 31 | } 32 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/NonValidatingTestSortedSSTableWriter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.bulkwriter; 21 | 22 | import java.nio.file.Path; 23 | 24 | import org.apache.cassandra.spark.utils.DigestAlgorithm; 25 | import org.jetbrains.annotations.NotNull; 26 | 27 | public class NonValidatingTestSortedSSTableWriter extends SortedSSTableWriter 28 | { 29 | public NonValidatingTestSortedSSTableWriter(MockTableWriter tableWriter, Path path, DigestAlgorithm digestAlgorithm, int partitionId) 30 | { 31 | super(tableWriter, path, digestAlgorithm, partitionId); 32 | } 33 | 34 | @Override 35 | public void validateSSTables(@NotNull BulkWriterContext writerContext) 36 | { 37 | // Skip validation for these tests 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/token/FailureHandlerTextContext.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.bulkwriter.token; 21 | 22 | import org.apache.cassandra.spark.bulkwriter.ClusterInfo; 23 | import org.apache.cassandra.spark.bulkwriter.JobInfo; 24 | import org.apache.cassandra.spark.bulkwriter.RingInstance; 25 | 26 | class FailureHandlerTextContext 27 | { 28 | TokenRangeMapping topology; 29 | JobInfo jobInfo; 30 | ClusterInfo clusterInfo; 31 | 32 | FailureHandlerTextContext(TokenRangeMapping topology, JobInfo jobInfo, ClusterInfo clusterInfo) 33 | { 34 | this.topology = topology; 35 | this.jobInfo = jobInfo; 36 | this.clusterInfo = clusterInfo; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/DefaultSizingTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data; 21 | 22 | import org.junit.jupiter.api.Test; 23 | 24 | import static org.junit.jupiter.api.Assertions.assertEquals; 25 | 26 | /** 27 | * Unit tests for the {@link DefaultSizing} class 28 | */ 29 | public class DefaultSizingTest 30 | { 31 | @Test 32 | public void testDefaultSizing() 33 | { 34 | Sizing sizing = new DefaultSizing(10); 35 | assertEquals(10, sizing.getEffectiveNumberOfCores()); 36 | 37 | sizing = new DefaultSizing(100); 38 | assertEquals(100, sizing.getEffectiveNumberOfCores()); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-1-big-Data.db: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-1-big-Data.db -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-1-big-Filter.db: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-1-big-Filter.db -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-1-big-Index.db: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-1-big-Index.db -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-1-big-Statistics.db: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-1-big-Statistics.db -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-1-big-Summary.db: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-1-big-Summary.db -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-1-big-TOC.txt: -------------------------------------------------------------------------------- 1 | TOC.txt 2 | Data.db 3 | Statistics.db 4 | Summary.db 5 | Filter.db 6 | Digest.crc32 7 | Index.db 8 | CompressionInfo.db 9 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-2-big-Data.db: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-2-big-Data.db -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-2-big-Filter.db: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-2-big-Filter.db -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-2-big-Index.db: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-2-big-Index.db -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-2-big-Statistics.db: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-2-big-Statistics.db -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-2-big-Summary.db: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-2-big-Summary.db -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/data/ks/table1-ea3b3e6b-0d78-4913-89f2-15fcf98711d0/na-2-big-TOC.txt: -------------------------------------------------------------------------------- 1 | TOC.txt 2 | Data.db 3 | Statistics.db 4 | Summary.db 5 | Filter.db 6 | Digest.crc32 7 | Index.db 8 | CompressionInfo.db 9 | -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/validation/keystore-certificate.p12: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/validation/keystore-certificate.p12 -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/validation/keystore-empty.p12: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/validation/keystore-empty.p12 -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/validation/keystore-expired.p12: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/validation/keystore-expired.p12 -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/validation/keystore-malformed.p12: -------------------------------------------------------------------------------- 1 | qwerty -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/validation/keystore-private.p12: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/validation/keystore-private.p12 -------------------------------------------------------------------------------- /cassandra-analytics-core/src/test/resources/validation/keystore-secret.p12: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/cassandra-analytics-core/src/test/resources/validation/keystore-secret.p12 -------------------------------------------------------------------------------- /cassandra-analytics-integration-framework/src/main/java/org/apache/cassandra/testing/TestVersion.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.testing; 20 | 21 | /** 22 | * Works with {@link TestVersionSupplier} 23 | */ 24 | public class TestVersion 25 | { 26 | private final String version; 27 | 28 | public TestVersion(String version) 29 | { 30 | this.version = version; 31 | } 32 | 33 | public String version() 34 | { 35 | return version; 36 | } 37 | 38 | @Override 39 | public String toString() 40 | { 41 | return "TestVersion{" + 42 | "version='" + version + '\'' + 43 | '}'; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkWriteDownInstanceMultipleTokensTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.analytics; 20 | 21 | import org.apache.cassandra.testing.ClusterBuilderConfiguration; 22 | 23 | public class BulkWriteDownInstanceMultipleTokensTest extends BulkWriteDownInstanceTest 24 | { 25 | @Override 26 | protected ClusterBuilderConfiguration testClusterConfiguration() 27 | { 28 | return super.testClusterConfiguration() 29 | .tokenCount(4); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/BulkWriteDownSidecarMultipleTokensTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.analytics; 20 | 21 | import org.apache.cassandra.testing.ClusterBuilderConfiguration; 22 | 23 | public class BulkWriteDownSidecarMultipleTokensTest extends BulkWriteDownSidecarTest 24 | { 25 | @Override 26 | protected ClusterBuilderConfiguration testClusterConfiguration() 27 | { 28 | return super.testClusterConfiguration() 29 | .tokenCount(4); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/CassandraAnalyticsSimpleMultipleTokensTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.analytics; 20 | 21 | import org.apache.cassandra.testing.ClusterBuilderConfiguration; 22 | 23 | public class CassandraAnalyticsSimpleMultipleTokensTest extends CassandraAnalyticsSimpleTest 24 | { 25 | @Override 26 | protected ClusterBuilderConfiguration testClusterConfiguration() 27 | { 28 | return super.testClusterConfiguration() 29 | .tokenCount(4); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /cassandra-analytics-integration-tests/src/test/java/org/apache/cassandra/analytics/testcontainer/BulkWriteS3CompatModeSimpleMultipleTokensTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.cassandra.analytics.testcontainer; 20 | 21 | import org.apache.cassandra.testing.ClusterBuilderConfiguration; 22 | 23 | public class BulkWriteS3CompatModeSimpleMultipleTokensTest extends BulkWriteS3CompatModeSimpleTest 24 | { 25 | @Override 26 | protected ClusterBuilderConfiguration testClusterConfiguration() 27 | { 28 | return super.testClusterConfiguration() 29 | .tokenCount(4); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /cassandra-analytics-sidecar-client/src/main/java/org/apache/cassandra/spark/validation/StartupValidation.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.validation; 21 | 22 | /** 23 | * An interface that has to be implemented by all startup validations 24 | */ 25 | @FunctionalInterface 26 | public interface StartupValidation 27 | { 28 | void validate(); 29 | 30 | default Throwable perform() 31 | { 32 | try 33 | { 34 | validate(); 35 | } 36 | catch (Throwable throwable) 37 | { 38 | return throwable; 39 | } 40 | return null; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/DecimalFeatures.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types; 21 | 22 | import java.util.Comparator; 23 | 24 | import org.apache.spark.sql.types.Decimal; 25 | 26 | interface DecimalFeatures extends SparkType 27 | { 28 | Comparator SPARK_DECIMAL_COMPARATOR = Comparator.naturalOrder(); 29 | 30 | @Override 31 | default int compareTo(Object first, Object second) 32 | { 33 | return SPARK_DECIMAL_COMPARATOR.compare((Decimal) first, (Decimal) second); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/NotImplementedFeatures.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types; 21 | 22 | import org.apache.cassandra.bridge.BigNumberConfig; 23 | import org.apache.cassandra.spark.data.CqlField; 24 | import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; 25 | import org.apache.spark.sql.types.DataType; 26 | 27 | interface NotImplementedFeatures extends SparkType 28 | { 29 | @Override 30 | default DataType dataType(BigNumberConfig bigNumberConfig) 31 | { 32 | throw CqlField.notImplemented(getClass().getName()); 33 | } 34 | 35 | @Override 36 | default Object nativeSparkSqlRowValue(GenericInternalRow row, int position) 37 | { 38 | throw CqlField.notImplemented(getClass().getName()); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/SparkAscii.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types; 21 | 22 | public class SparkAscii implements StringFeatures 23 | { 24 | public static final SparkAscii INSTANCE = new SparkAscii(); 25 | 26 | private SparkAscii() 27 | { 28 | 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/SparkBigInt.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types; 21 | 22 | public class SparkBigInt implements LongFeatures 23 | { 24 | public static final SparkBigInt INSTANCE = new SparkBigInt(); 25 | 26 | private SparkBigInt() 27 | { 28 | 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/SparkBlob.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types; 21 | 22 | public class SparkBlob implements BinaryFeatures 23 | { 24 | public static final SparkBlob INSTANCE = new SparkBlob(); 25 | 26 | private SparkBlob() 27 | { 28 | 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/SparkCounter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types; 21 | 22 | public class SparkCounter implements NotImplementedFeatures 23 | { 24 | public static final SparkCounter INSTANCE = new SparkCounter(); 25 | 26 | private SparkCounter() 27 | { 28 | 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/SparkInt.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types; 21 | 22 | public class SparkInt implements IntFeatures 23 | { 24 | public static final SparkInt INSTANCE = new SparkInt(); 25 | 26 | private SparkInt() 27 | { 28 | 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/SparkText.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types; 21 | 22 | public class SparkText implements StringFeatures 23 | { 24 | public static final SparkText INSTANCE = new SparkText(); 25 | 26 | private SparkText() 27 | { 28 | 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/SparkTime.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types; 21 | 22 | public class SparkTime implements LongFeatures 23 | { 24 | public static final SparkTime INSTANCE = new SparkTime(); 25 | 26 | private SparkTime() 27 | { 28 | 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/SparkTimeUUID.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types; 21 | 22 | public class SparkTimeUUID implements UUIDFeatures 23 | { 24 | public static final SparkTimeUUID INSTANCE = new SparkTimeUUID(); 25 | 26 | private SparkTimeUUID() 27 | { 28 | 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/SparkUUID.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types; 21 | 22 | public class SparkUUID implements UUIDFeatures 23 | { 24 | public static final SparkUUID INSTANCE = new SparkUUID(); 25 | 26 | private SparkUUID() 27 | { 28 | 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/SparkVarChar.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types; 21 | 22 | public class SparkVarChar implements StringFeatures 23 | { 24 | public static final SparkVarChar INSTANCE = new SparkVarChar(); 25 | 26 | private SparkVarChar() 27 | { 28 | 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/UUIDFeatures.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types; 21 | 22 | import org.apache.cassandra.spark.data.CqlField; 23 | 24 | interface UUIDFeatures extends StringFeatures 25 | { 26 | @Override 27 | default Object toTestRowType(Object value) 28 | { 29 | return java.util.UUID.fromString(value.toString()); 30 | } 31 | 32 | @Override 33 | default int compareTo(Object first, Object second) 34 | { 35 | return CqlField.UUID_COMPARATOR.compare(first.toString(), second.toString()); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/complex/MapFeatures.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types.complex; 21 | 22 | import org.apache.cassandra.spark.data.converter.types.SparkType; 23 | 24 | interface MapFeatures extends SparkType, CollectionFeatures 25 | { 26 | default SparkType keyType() 27 | { 28 | return sparkType(); 29 | } 30 | 31 | default SparkType valueType() 32 | { 33 | return sparkType(1); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/data/converter/types/complex/SparkSet.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.converter.types.complex; 21 | 22 | import java.util.stream.Collector; 23 | import java.util.stream.Collectors; 24 | 25 | import org.apache.cassandra.spark.data.CqlField; 26 | import org.apache.cassandra.spark.data.converter.SparkSqlTypeConverter; 27 | 28 | public class SparkSet extends SparkList 29 | { 30 | public SparkSet(SparkSqlTypeConverter converter, CqlField.CqlSet type) 31 | { 32 | super(converter, type); 33 | } 34 | 35 | @Override 36 | public Collector collector() 37 | { 38 | return Collectors.toSet(); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /cassandra-analytics-spark-converter/src/main/java/org/apache/cassandra/spark/utils/TemporaryDirectory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.utils; 21 | 22 | import java.io.IOException; 23 | import java.nio.file.Files; 24 | import java.nio.file.Path; 25 | import java.util.UUID; 26 | 27 | import org.apache.commons.io.FileUtils; 28 | 29 | public class TemporaryDirectory implements AutoCloseable 30 | { 31 | private final Path directory; 32 | 33 | public TemporaryDirectory() throws IOException 34 | { 35 | directory = Files.createTempDirectory(UUID.randomUUID().toString()); 36 | } 37 | 38 | public Path path() 39 | { 40 | return directory; 41 | } 42 | 43 | @Override 44 | public void close() throws IOException 45 | { 46 | FileUtils.deleteDirectory(directory.toFile()); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /cassandra-avro-converter/build.gradle: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | plugins { 21 | id('java-library') 22 | id('maven-publish') 23 | } 24 | 25 | configurations { 26 | all*.exclude(group: 'org.slf4j', module: 'slf4j-log4j12') 27 | all*.exclude(group: 'log4j', module: 'log4j') 28 | } 29 | 30 | publishing { 31 | publications { 32 | maven(MavenPublication) { 33 | from components.java 34 | groupId project.group 35 | artifactId "${archivesBaseName}" 36 | version System.getenv("CODE_VERSION") ?: "${version}" 37 | } 38 | } 39 | } 40 | 41 | dependencies { 42 | api(project(':cassandra-bridge')) 43 | compileOnly project(":cassandra-analytics-common") 44 | compileOnly project(":cassandra-bridge") 45 | compileOnly "org.apache.avro:avro:${avroVersion}" 46 | } 47 | -------------------------------------------------------------------------------- /cassandra-bridge/src/main/java/org/apache/cassandra/bridge/BloomFilter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.bridge; 21 | 22 | import java.nio.ByteBuffer; 23 | import java.util.function.Predicate; 24 | 25 | /** 26 | * Version independent interface to front bloom filter. 27 | */ 28 | public interface BloomFilter extends Predicate 29 | { 30 | /** 31 | * @param partitionKey serialzied partition key. 32 | * @return true if SSTable might contain a given partition key, might return false-positives but never false-negatives. 33 | */ 34 | default boolean mightContain(ByteBuffer partitionKey) 35 | { 36 | return test(partitionKey); 37 | } 38 | 39 | default boolean doesNotContain(ByteBuffer partitionKey) 40 | { 41 | return !mightContain(partitionKey); 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /cassandra-bridge/src/main/java/org/apache/cassandra/bridge/Tokenizer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.bridge; 21 | 22 | import java.math.BigInteger; 23 | import java.nio.ByteBuffer; 24 | 25 | /** 26 | * Interface that converts partition key to a BigInteger token 27 | */ 28 | public interface Tokenizer 29 | { 30 | BigInteger toToken(ByteBuffer partitionKey); 31 | } 32 | -------------------------------------------------------------------------------- /cassandra-bridge/src/testFixtures/java/org/apache/cassandra/spark/CommonTestUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark; 21 | 22 | import org.apache.cassandra.bridge.CassandraBridge; 23 | import org.apache.cassandra.spark.data.CqlField; 24 | import org.quicktheories.core.Gen; 25 | 26 | import static org.quicktheories.generators.SourceDSL.arbitrary; 27 | 28 | // CHECKSTYLE IGNORE: default constructor is required as this util class is extended elsewhere 29 | public class CommonTestUtils 30 | { 31 | CommonTestUtils() 32 | { 33 | 34 | } 35 | 36 | public static Gen cql3Type(CassandraBridge bridge) 37 | { 38 | return arbitrary().pick(bridge.supportedTypes()); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /cassandra-four-zero-avro-converter/build.gradle: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | plugins { 21 | id('java-library') 22 | } 23 | 24 | configurations { 25 | all*.exclude(group: 'org.slf4j', module: 'slf4j-log4j12') 26 | all*.exclude(group: 'log4j', module: 'log4j') 27 | } 28 | 29 | dependencies { 30 | compileOnly project(':cassandra-bridge') 31 | compileOnly project(":cassandra-analytics-cdc") 32 | compileOnly project(":cassandra-analytics-common") 33 | compileOnly project(":cassandra-four-zero-types") 34 | compileOnly project(":cassandra-avro-converter") 35 | compileOnly(project(path: ':cassandra-four-zero', configuration: 'shadow')) 36 | compileOnly "org.apache.avro:avro:${avroVersion}" 37 | } 38 | 39 | jar { 40 | archiveFileName = "four-zero-avro.jar" 41 | } -------------------------------------------------------------------------------- /cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/cdc/FourZeroMutation.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.cdc; 21 | 22 | import org.apache.cassandra.cdc.api.Mutation; 23 | 24 | public class FourZeroMutation implements Mutation 25 | { 26 | public final org.apache.cassandra.db.Mutation mutation; 27 | 28 | private FourZeroMutation(org.apache.cassandra.db.Mutation mutation) 29 | { 30 | this.mutation = mutation; 31 | } 32 | 33 | public static FourZeroMutation wrap(org.apache.cassandra.db.Mutation mutation) 34 | { 35 | return new FourZeroMutation(mutation); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /cassandra-four-zero-bridge/src/main/java/org/apache/cassandra/spark/reader/Scannable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.reader; 21 | 22 | import org.apache.cassandra.io.sstable.ISSTableScanner; 23 | 24 | public interface Scannable 25 | { 26 | ISSTableScanner scanner(); 27 | } 28 | -------------------------------------------------------------------------------- /cassandra-four-zero-types/src/main/java/org/apache/cassandra/spark/data/types/Ascii.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.types; 21 | 22 | import org.apache.cassandra.cql3.functions.types.DataType; 23 | import org.apache.cassandra.db.marshal.AbstractType; 24 | import org.apache.cassandra.db.marshal.AsciiType; 25 | 26 | public class Ascii extends StringBased 27 | { 28 | public static final Ascii INSTANCE = new Ascii(); 29 | 30 | @Override 31 | public String name() 32 | { 33 | return "ascii"; 34 | } 35 | 36 | @Override 37 | public AbstractType dataType() 38 | { 39 | return AsciiType.instance; 40 | } 41 | 42 | @Override 43 | public DataType driverDataType(boolean isFrozen) 44 | { 45 | return DataType.ascii(); 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /cassandra-four-zero-types/src/main/java/org/apache/cassandra/spark/data/types/BinaryBased.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.types; 21 | 22 | import org.apache.cassandra.spark.data.NativeType; 23 | 24 | public abstract class BinaryBased extends NativeType 25 | { 26 | 27 | } 28 | -------------------------------------------------------------------------------- /cassandra-four-zero-types/src/main/java/org/apache/cassandra/spark/data/types/Counter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.types; 21 | 22 | import org.apache.cassandra.spark.data.NativeType; 23 | 24 | public class Counter extends NativeType 25 | { 26 | public static final Counter INSTANCE = new Counter(); 27 | 28 | @Override 29 | public String name() 30 | { 31 | return "counter"; 32 | } 33 | 34 | @Override 35 | public boolean isSupported() 36 | { 37 | return false; 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /cassandra-four-zero-types/src/main/java/org/apache/cassandra/spark/data/types/LongBased.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.types; 21 | 22 | import org.apache.cassandra.spark.data.NativeType; 23 | import org.apache.cassandra.spark.utils.RandomUtils; 24 | 25 | public abstract class LongBased extends NativeType 26 | { 27 | @Override 28 | public Object randomValue(int minCollectionSize) 29 | { 30 | return (long) RandomUtils.randomPositiveInt(5_000_000); // Keep within bound to avoid overflows 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /cassandra-four-zero-types/src/main/java/org/apache/cassandra/spark/data/types/StringBased.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.types; 21 | 22 | import org.apache.cassandra.cql3.functions.types.SettableByIndexData; 23 | import org.apache.cassandra.spark.data.NativeType; 24 | import org.apache.cassandra.spark.utils.RandomUtils; 25 | 26 | public abstract class StringBased extends NativeType 27 | { 28 | @Override 29 | public Object randomValue(int minCollectionSize) 30 | { 31 | return RandomUtils.randomAlphanumeric(RandomUtils.randomPositiveInt(32)); 32 | } 33 | 34 | @Override 35 | protected void setInnerValueInternal(SettableByIndexData udtValue, int position, Object value) 36 | { 37 | udtValue.setString(position, (String) value); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /cassandra-four-zero-types/src/main/java/org/apache/cassandra/spark/data/types/Text.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.types; 21 | 22 | import org.apache.cassandra.cql3.functions.types.DataType; 23 | import org.apache.cassandra.db.marshal.AbstractType; 24 | import org.apache.cassandra.db.marshal.UTF8Type; 25 | 26 | public class Text extends StringBased 27 | { 28 | public static final Text INSTANCE = new Text(); 29 | 30 | @Override 31 | public String name() 32 | { 33 | return "text"; 34 | } 35 | 36 | @Override 37 | public AbstractType dataType() 38 | { 39 | return UTF8Type.instance; 40 | } 41 | 42 | @Override 43 | public DataType driverDataType(boolean isFrozen) 44 | { 45 | return DataType.text(); 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /cassandra-four-zero-types/src/main/java/org/apache/cassandra/spark/data/types/VarChar.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.data.types; 21 | 22 | import org.apache.cassandra.cql3.functions.types.DataType; 23 | import org.apache.cassandra.db.marshal.AbstractType; 24 | import org.apache.cassandra.db.marshal.UTF8Type; 25 | 26 | public class VarChar extends StringBased 27 | { 28 | public static final VarChar INSTANCE = new VarChar(); 29 | 30 | @Override 31 | public String name() 32 | { 33 | return "varchar"; 34 | } 35 | 36 | @Override 37 | public AbstractType dataType() 38 | { 39 | return UTF8Type.instance; 40 | } 41 | 42 | @Override 43 | public DataType driverDataType(boolean isFrozen) 44 | { 45 | return DataType.varchar(); 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /cassandra-four-zero-types/src/main/java/org/apache/cassandra/spark/reader/ListBuffer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.reader; 21 | 22 | public class ListBuffer extends ComplexTypeBuffer 23 | { 24 | ListBuffer(int cellCount) 25 | { 26 | super(cellCount, cellCount); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /cassandra-four-zero-types/src/main/java/org/apache/cassandra/spark/reader/MapBuffer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.reader; 21 | 22 | import org.apache.cassandra.db.rows.Cell; 23 | 24 | public class MapBuffer extends ComplexTypeBuffer 25 | { 26 | // MapBuffer requires 2x the number of buffers, one for the key and value. 27 | MapBuffer(int cellCount) 28 | { 29 | super(cellCount, cellCount * 2); 30 | } 31 | 32 | @Override 33 | public void addCell(Cell cell) 34 | { 35 | add(cell.path().get(0)); // Map - copy over key and value 36 | super.addCell(cell); 37 | } 38 | 39 | @Override 40 | protected int elements() 41 | { 42 | // divide 2 because we add key and value to the buffer, which makes it twice as big as the map entries. 43 | return super.elements() / 2; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /cassandra-four-zero-types/src/main/java/org/apache/cassandra/spark/reader/SetBuffer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.reader; 21 | 22 | import org.apache.cassandra.db.rows.Cell; 23 | 24 | public class SetBuffer extends ComplexTypeBuffer 25 | { 26 | SetBuffer(int cellCount) 27 | { 28 | super(cellCount, cellCount); 29 | } 30 | 31 | @Override 32 | public void addCell(Cell cell) 33 | { 34 | add(cell.path().get(0)); // Set - copy over key 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /cassandra-four-zero-types/src/main/java/org/apache/cassandra/spark/reader/UdtBuffer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | package org.apache.cassandra.spark.reader; 21 | 22 | public class UdtBuffer extends ComplexTypeBuffer 23 | { 24 | UdtBuffer(int cellCount) 25 | { 26 | super(cellCount, cellCount); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /cassandra-three-zero/TODO.md: -------------------------------------------------------------------------------- 1 | 20 | 21 | ### Note 22 | 23 | Cassandra Bridge implementation for Cassandra-all of version 3.0 is done and mostly working. 24 | 25 | It needs some adjustments in order to work with open-source Cassandra-all library (21 compilation errors). 26 | 27 | If there turns out to be public interest in having it available here, it will be open-sourced as a separate contribution. 28 | -------------------------------------------------------------------------------- /githooks/pre-push: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, 14 | # software distributed under the License is distributed on an 15 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | # KIND, either express or implied. See the License for the 17 | # specific language governing permissions and limitations 18 | # under the License. 19 | 20 | set -e 21 | 22 | PROJECT_ROOT="$(cd "$(dirname "${BASH_SOURCE[0]}")/.." && pwd)" 23 | 24 | checkstyle() { 25 | SCALA_VERSION=$1 26 | SPARK_VERSION=$2 27 | 28 | export SCALA_VERSION=$SCALA_VERSION 29 | export SPARK_VERSION=$SPARK_VERSION 30 | 31 | $PROJECT_ROOT/gradlew checkstyleMain checkstyleTest 32 | } 33 | 34 | echo "Running pre-push hook..." 35 | 36 | # scala 2.12 && spark 3 37 | checkstyle 2.12 3 38 | 39 | # scala 2.12 && spark 3 40 | checkstyle 2.13 3 41 | -------------------------------------------------------------------------------- /gradle/wrapper/gradle-wrapper.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/gradle/wrapper/gradle-wrapper.jar -------------------------------------------------------------------------------- /gradle/wrapper/gradle-wrapper.properties: -------------------------------------------------------------------------------- 1 | distributionBase=GRADLE_USER_HOME 2 | distributionPath=wrapper/dists 3 | distributionUrl=https\://services.gradle.org/distributions/gradle-8.14-bin.zip 4 | networkTimeout=10000 5 | validateDistributionUrl=true 6 | zipStoreBase=GRADLE_USER_HOME 7 | zipStorePath=wrapper/dists 8 | -------------------------------------------------------------------------------- /profiles/scala-2.12-spark-3-jdk-11.gradle: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | ext { 21 | jacksonVersion="2.14.2" 22 | jacksonScalaModuleVersion="2.14.2" 23 | scalaMajorVersion="2.12" 24 | scalaVersion="2.12.10" 25 | sparkGroupId="org.apache.spark" 26 | sparkMajorVersion="3" 27 | sparkVersion="3.2.2" 28 | } 29 | -------------------------------------------------------------------------------- /profiles/scala-2.13-spark-3-jdk-11.gradle: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, 13 | * software distributed under the License is distributed on an 14 | * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | * KIND, either express or implied. See the License for the 16 | * specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | 20 | ext { 21 | jacksonVersion="2.12.7" 22 | jacksonScalaModuleVersion="2.12.7" 23 | scalaMajorVersion="2.13" 24 | scalaVersion="2.13.10" 25 | sparkGroupId="org.apache.spark" 26 | sparkMajorVersion="3" 27 | sparkVersion="3.2.2" 28 | } 29 | -------------------------------------------------------------------------------- /scripts/.mvn/wrapper/maven-wrapper.jar: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-analytics/72d6a7d2b2caa2c62b67a95c5799e30edbabd1c9/scripts/.mvn/wrapper/maven-wrapper.jar -------------------------------------------------------------------------------- /scripts/.mvn/wrapper/maven-wrapper.properties: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | distributionUrl=https://repo.maven.apache.org/maven2/org/apache/maven/apache-maven/3.8.8/apache-maven-3.8.8-bin.zip 18 | wrapperUrl=https://repo.maven.apache.org/maven2/org/apache/maven/wrapper/maven-wrapper/3.2.0/maven-wrapper-3.2.0.jar 19 | -------------------------------------------------------------------------------- /scripts/build-dependencies.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | # 3 | # Licensed to the Apache Software Foundation (ASF) under one 4 | # or more contributor license agreements. See the NOTICE file 5 | # distributed with this work for additional information 6 | # regarding copyright ownership. The ASF licenses this file 7 | # to you under the Apache License, Version 2.0 (the 8 | # "License"); you may not use this file except in compliance 9 | # with the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | set -xe 21 | 22 | SCRIPT_DIR=$( dirname -- "$( readlink -f -- "$0"; )"; ) 23 | 24 | ${SCRIPT_DIR}/build-dtest-jars.sh 25 | --------------------------------------------------------------------------------