├── .asf.yaml ├── .github ├── ISSUE_TEMPLATE │ └── please-make-a-jira-ticket-or-ask-questions-on-the-mailing-list.md ├── pull_request_template.md └── workflows │ └── main.yml ├── .gitignore ├── CHANGES.txt ├── LICENSE.txt ├── NOTICE ├── README.md ├── build.sbt ├── connector └── src │ ├── it │ ├── resources │ │ ├── log4j.properties │ │ ├── log4j2.properties │ │ ├── logback.xml │ │ ├── metrics.properties │ │ └── test.conf │ └── scala │ │ ├── com │ │ └── datastax │ │ │ ├── bdp │ │ │ └── spark │ │ │ │ └── search │ │ │ │ ├── SearchAnalyticsIntegrationSpec.scala │ │ │ │ └── SearchSupport.scala │ │ │ └── spark │ │ │ └── connector │ │ │ ├── CassandraJavaUtilSpec.scala │ │ │ ├── SparkCassandraITFlatSpecBase.scala │ │ │ ├── cluster │ │ │ ├── Cluster.scala │ │ │ ├── ClusterHolder.scala │ │ │ └── Fixtures.scala │ │ │ ├── cql │ │ │ ├── CassandraAuthenticatedConnectorSpec.scala │ │ │ ├── CassandraConnectorSpec.scala │ │ │ ├── CassandraPartitionKeyWhereSpec.scala │ │ │ ├── CassandraSSLClientAuthConnectorSpec.scala │ │ │ ├── CassandraSSLConnectorSpec.scala │ │ │ ├── ContinuousPagingScannerSpec.scala │ │ │ ├── SchemaSpec.scala │ │ │ └── sai │ │ │ │ ├── IndexedKeySpec.scala │ │ │ │ ├── IndexedListSpec.scala │ │ │ │ ├── IndexedMapSpec.scala │ │ │ │ ├── IndexedNumericSpec.scala │ │ │ │ ├── IndexedSetSpec.scala │ │ │ │ ├── IndexedStaticSpec.scala │ │ │ │ ├── IndexedStringSpec.scala │ │ │ │ ├── SaiBaseSpec.scala │ │ │ │ └── SaiCollectionBaseSpec.scala │ │ │ ├── datasource │ │ │ ├── CassandraCatalogNamespaceSpec.scala │ │ │ ├── CassandraCatalogSpecBase.scala │ │ │ ├── CassandraCatalogTableReadSpec.scala │ │ │ ├── CassandraCatalogTableSpec.scala │ │ │ └── CassandraCatalogTableWriteSpec.scala │ │ │ ├── doc │ │ │ ├── DocCheck.scala │ │ │ └── DocExamples.scala │ │ │ ├── rdd │ │ │ ├── CassandraJavaPairRDDSpec.scala │ │ │ ├── CassandraJavaRDDSpec.scala │ │ │ ├── CassandraRDDMockSpec.scala │ │ │ ├── CassandraRDDSpec.scala │ │ │ ├── CassandraTableScanRDDSpec.scala │ │ │ ├── ConnectorMetricsSpec.scala │ │ │ ├── CustomTableScanMethodSpec.scala │ │ │ ├── DseGraphUnionRDDSpec.scala │ │ │ ├── InnerJoinSpec.scala │ │ │ ├── PartitionedCassandraRDDSpec.scala │ │ │ ├── RDDSpec.scala │ │ │ ├── ReplicaRepartitionedCassandraRDDSpec.scala │ │ │ ├── partitioner │ │ │ │ ├── CassandraPartitionGeneratorSpec.scala │ │ │ │ ├── DataSizeEstimatesSpec.scala │ │ │ │ └── TokenGeneratorSpec.scala │ │ │ ├── reader │ │ │ │ └── PrefetchingResultSetIteratorSpec.scala │ │ │ └── typeTests │ │ │ │ ├── AbstractDateTypeTest.scala │ │ │ │ ├── AbstractTypeTest.scala │ │ │ │ ├── AsciiTypeTest.scala │ │ │ │ ├── BigintTypeTest.scala │ │ │ │ ├── BlobTypeTest.scala │ │ │ │ ├── BooleanTypeTest.scala │ │ │ │ ├── CounterTypeTest.scala │ │ │ │ ├── DateTypeCETTest.scala │ │ │ │ ├── DateTypeCSTTest.scala │ │ │ │ ├── DateTypePSTTest.scala │ │ │ │ ├── DecimalTypeTest.scala │ │ │ │ ├── DoubleTypeTest.scala │ │ │ │ ├── FloatTypeTest.scala │ │ │ │ ├── InetTypeTest.scala │ │ │ │ ├── IntTypeTest.scala │ │ │ │ ├── SmallIntTypeTest.scala │ │ │ │ ├── TextTypeTest.scala │ │ │ │ ├── TimeTypeTest.scala │ │ │ │ ├── TimeUUIDTypeTest.scala │ │ │ │ ├── TimestampTypeTest.scala │ │ │ │ ├── TinyIntTypeTest.scala │ │ │ │ ├── UUIDTypeTest.scala │ │ │ │ ├── VarintTypeTest.scala │ │ │ │ └── VectorTypeTest.scala │ │ │ ├── sql │ │ │ ├── CassandraDataFrameDateBehaviors.scala │ │ │ ├── CassandraDataFrameDateCETSpec.scala │ │ │ ├── CassandraDataFrameDatePSTSpec.scala │ │ │ ├── CassandraDataFrameMetadataSpec.scala │ │ │ ├── CassandraDataFrameSelectUdtSpec.scala │ │ │ ├── CassandraDataFrameSpec.scala │ │ │ ├── CassandraDataSourceNoPushdownSpec.scala │ │ │ ├── CassandraDataSourceSpec.scala │ │ │ ├── CassandraSQLClusterLevelSpec.scala │ │ │ ├── CassandraSQLSpec.scala │ │ │ └── CassandraStreamingSinkSpec.scala │ │ │ ├── streaming │ │ │ └── RDDStreamingSpec.scala │ │ │ ├── types │ │ │ ├── DateRangeTypeSpec.scala │ │ │ ├── GeometricTypeSpec.scala │ │ │ ├── TupleTypeSpec.scala │ │ │ └── UserDefinedTypeSpec.scala │ │ │ ├── util │ │ │ ├── CatalystUtil.scala │ │ │ └── MultiThreadedSpec.scala │ │ │ └── writer │ │ │ ├── BoundStatementBuilderSpec.scala │ │ │ ├── GroupingBatchBuilderSpec.scala │ │ │ ├── TableWriterColumnNamesSpec.scala │ │ │ ├── TableWriterSpec.scala │ │ │ └── ThrottlingSpec.scala │ │ └── org │ │ └── apache │ │ └── spark │ │ ├── DseTestUtil.scala │ │ ├── metrics │ │ └── CassandraConnectorSourceSpec.scala │ │ └── sql │ │ ├── CassandraPrunedFilteredScanSpec.scala │ │ ├── CassandraSourceOptionsSpec.scala │ │ ├── cassandra │ │ └── execution │ │ │ ├── CassandraDirectJoinSpec.scala │ │ │ └── JoinThrottlingSpec.scala │ │ └── datastax │ │ └── test │ │ ├── empty │ │ └── EmptyStreamSource.scala │ │ └── monotonic │ │ └── FakeStreamSource.scala │ ├── main │ ├── java │ │ └── com │ │ │ └── datastax │ │ │ ├── driver │ │ │ └── core │ │ │ │ └── MetadataHook.java │ │ │ └── spark │ │ │ └── connector │ │ │ └── japi │ │ │ ├── CassandraJavaUtil.java │ │ │ ├── CassandraStreamingJavaUtil.java │ │ │ ├── DStreamJavaFunctions.java │ │ │ ├── PairRDDJavaFunctions.java │ │ │ ├── RDDAndDStreamCommonJavaFunctions.java │ │ │ ├── RDDJavaFunctions.java │ │ │ ├── SparkContextJavaFunctions.java │ │ │ ├── StreamingContextJavaFunctions.java │ │ │ └── rdd │ │ │ ├── CassandraJavaPairRDD.java │ │ │ ├── CassandraJavaRDD.java │ │ │ ├── CassandraJoinJavaRDD.java │ │ │ └── CassandraTableScanJavaRDD.java │ ├── main.iml │ ├── scala-2.12 │ │ └── com │ │ │ └── datastax │ │ │ └── spark │ │ │ └── connector │ │ │ ├── japi │ │ │ └── GenericJavaRowReaderFactory.java │ │ │ └── util │ │ │ └── RuntimeUtil.scala │ ├── scala-2.13 │ │ └── com │ │ │ └── datastax │ │ │ └── spark │ │ │ └── connector │ │ │ ├── japi │ │ │ └── GenericJavaRowReaderFactory.java │ │ │ └── util │ │ │ └── RuntimeUtil.scala │ └── scala │ │ ├── com │ │ └── datastax │ │ │ ├── bdp │ │ │ ├── spark │ │ │ │ └── ContinuousPagingScanner.scala │ │ │ └── util │ │ │ │ ├── MiscUtil.scala │ │ │ │ └── ScalaJavaUtil.scala │ │ │ └── spark │ │ │ └── connector │ │ │ ├── BatchSize.scala │ │ │ ├── CassandraSparkExtensions.scala │ │ │ ├── CassandraTableScanRDDFunctions.scala │ │ │ ├── ColumnSelector.scala │ │ │ ├── DatasetFunctions.scala │ │ │ ├── DocUtil.scala │ │ │ ├── PairRDDFunctions.scala │ │ │ ├── RDDFunctions.scala │ │ │ ├── SparkContextFunctions.scala │ │ │ ├── cql │ │ │ ├── AuthConf.scala │ │ │ ├── CassandraConnectionFactory.scala │ │ │ ├── CassandraConnector.scala │ │ │ ├── CassandraConnectorConf.scala │ │ │ ├── QueryUtils.scala │ │ │ ├── RefCountMap.scala │ │ │ ├── RefCountedCache.scala │ │ │ ├── Scanner.scala │ │ │ ├── SessionProxy.scala │ │ │ └── package.scala │ │ │ ├── datasource │ │ │ ├── CassandraCatalog.scala │ │ │ ├── CassandraInJoinReaderFactory.scala │ │ │ ├── CassandraScanBuilder.scala │ │ │ ├── CassandraScanPartitionReaderFactory.scala │ │ │ ├── CassandraSourceUtil.scala │ │ │ ├── CassandraTable.scala │ │ │ ├── CassandraWriteBuilder.scala │ │ │ ├── CasssandraDriverDataWriterFactory.scala │ │ │ ├── InternalRowWriterFactory.scala │ │ │ ├── JoinHelper.scala │ │ │ ├── ScanHelper.scala │ │ │ ├── UnsafeRowReaderFactory.scala │ │ │ └── UnsafeRowWriterFactory.scala │ │ │ ├── mapper │ │ │ └── DataFrameColumnMapper.scala │ │ │ ├── package.scala │ │ │ ├── rdd │ │ │ ├── AbstractCassandraJoin.scala │ │ │ ├── CassandraCoGroupedRDD.scala │ │ │ ├── CassandraJoinRDD.scala │ │ │ ├── CassandraLeftJoinRDD.scala │ │ │ ├── CassandraLimit.scala │ │ │ ├── CassandraMergeJoinRDD.scala │ │ │ ├── CassandraRDD.scala │ │ │ ├── CassandraTableRowReaderProvider.scala │ │ │ ├── CassandraTableScanRDD.scala │ │ │ ├── ClusteringOrder.scala │ │ │ ├── CqlWhereClause.scala │ │ │ ├── DseGraphUnionedRDD.scala │ │ │ ├── EmptyCassandraRDD.scala │ │ │ ├── ReadConf.scala │ │ │ ├── SpannedByKeyRDD.scala │ │ │ ├── SpannedRDD.scala │ │ │ ├── ValidRDDType.scala │ │ │ ├── package.scala │ │ │ ├── partitioner │ │ │ │ ├── BucketingRangeIndex.scala │ │ │ │ ├── CassandraPartition.scala │ │ │ │ ├── CassandraPartitionGenerator.scala │ │ │ │ ├── CassandraPartitionedRDD.scala │ │ │ │ ├── CassandraPartitioner.scala │ │ │ │ ├── DataSizeEstimates.scala │ │ │ │ ├── Murmur3PartitionerTokenRangeSplitter.scala │ │ │ │ ├── NodeAddresses.scala │ │ │ │ ├── RandomPartitionerTokenRangeSplitter.scala │ │ │ │ ├── ReplicaPartitioner.scala │ │ │ │ ├── SplitSizeEstimator.scala │ │ │ │ ├── TokenGenerator.scala │ │ │ │ ├── TokenRangeClusterer.scala │ │ │ │ ├── TokenRangeSplitter.scala │ │ │ │ ├── dht │ │ │ │ │ ├── Token.scala │ │ │ │ │ ├── TokenFactory.scala │ │ │ │ │ └── TokenRange.scala │ │ │ │ └── package.scala │ │ │ └── reader │ │ │ │ ├── ClassBasedRowReader.scala │ │ │ │ ├── FunctionBasedRowReader.scala │ │ │ │ ├── KeyValueRowReader.scala │ │ │ │ ├── PrefetchingResultSetIterator.scala │ │ │ │ ├── RowReader.scala │ │ │ │ ├── RowReaderFactory.scala │ │ │ │ ├── ValueRowReader.scala │ │ │ │ └── package.scala │ │ │ ├── streaming │ │ │ ├── CassandraStreamingRDD.scala │ │ │ ├── DStreamFunctions.scala │ │ │ ├── StreamingContextFunctions.scala │ │ │ └── package.scala │ │ │ ├── types │ │ │ └── ColumnTypeConf.scala │ │ │ ├── util │ │ │ ├── BufferedIterator2.scala │ │ │ ├── ClassLoaderCheck.scala │ │ │ ├── CodecRegistryUtil.scala │ │ │ ├── ConfigCheck.scala │ │ │ ├── ConfigParameter.scala │ │ │ ├── CountingIterator.scala │ │ │ ├── CqlWhereParser.scala │ │ │ ├── DataFrameOption.scala │ │ │ ├── JavaApiHelper.scala │ │ │ ├── MagicalTypeTricks.scala │ │ │ ├── MergeJoinIterator.scala │ │ │ ├── MultiMergeJoinIterator.scala │ │ │ ├── PatitionKeyTools.scala │ │ │ ├── PriorityHashMap.scala │ │ │ ├── RefBuilder.scala │ │ │ ├── SerialShutdownHooks.scala │ │ │ ├── SpanningIterator.scala │ │ │ ├── Threads.scala │ │ │ └── package.scala │ │ │ └── writer │ │ │ ├── AsyncExecutor.scala │ │ │ ├── Batch.scala │ │ │ ├── BatchGroupingKey.scala │ │ │ ├── BatchStatementBuilder.scala │ │ │ ├── BoundStatementBuilder.scala │ │ │ ├── CassandraRowWriter.scala │ │ │ ├── DefaultRowWriter.scala │ │ │ ├── GroupingBatchBuilder.scala │ │ │ ├── NullKeyColumnException.scala │ │ │ ├── ObjectSizeEstimator.scala │ │ │ ├── QueryExecutor.scala │ │ │ ├── RateLimiter.scala │ │ │ ├── ReplicaLocator.scala │ │ │ ├── RichStatement.scala │ │ │ ├── RowWriter.scala │ │ │ ├── RowWriterFactory.scala │ │ │ ├── SqlRowWriter.scala │ │ │ ├── TableWriter.scala │ │ │ ├── WritableToCassandra.scala │ │ │ ├── WriteConf.scala │ │ │ ├── WriteOption.scala │ │ │ └── package.scala │ │ └── org │ │ └── apache │ │ └── spark │ │ ├── metrics │ │ ├── CassandraConnectorSource.scala │ │ ├── CassandraSink.scala │ │ ├── InputMetricsUpdater.scala │ │ ├── MetricsUpdater.scala │ │ └── OutputMetricsUpdater.scala │ │ └── sql │ │ └── cassandra │ │ ├── BasicCassandraPredicatePushDown.scala │ │ ├── CassandraMetadataFunctions.scala │ │ ├── CassandraPredicateRules.scala │ │ ├── CassandraSQLRow.scala │ │ ├── CassandraSourceRelation.scala │ │ ├── DataTypeConverter.scala │ │ ├── DefaultSource.scala │ │ ├── DsePredicateRules.scala │ │ ├── InClausePredicateRules.scala │ │ ├── PredicateOps.scala │ │ ├── SolrPredicateRules.scala │ │ ├── TimeUUIDPredicateRules.scala │ │ ├── execution │ │ ├── CassandraDirectJoinExec.scala │ │ └── CassandraDirectJoinStrategy.scala │ │ ├── package-info.java │ │ └── package.scala │ └── test │ ├── java │ └── com │ │ └── datastax │ │ └── spark │ │ └── connector │ │ ├── CassandraJavaUtilTest.java │ │ ├── CassandraStreamingJavaUtilTest.java │ │ ├── SampleJavaBean.java │ │ ├── SampleJavaBeanSubClass.java │ │ ├── SampleJavaBeanWithMultipleCtors.java │ │ ├── SampleJavaBeanWithTransientFields.java │ │ ├── SampleJavaBeanWithoutNoArgsCtor.java │ │ ├── SampleWeirdJavaBean.java │ │ ├── SampleWithDeeplyNestedJavaBean.java │ │ ├── SampleWithNestedJavaBean.java │ │ ├── japi │ │ ├── CassandraRowTest.java │ │ ├── SparkContextJavaFunctionsTest.java │ │ └── rdd │ │ │ ├── CassandraJavaPairRDDTest.java │ │ │ ├── CassandraJavaRDDTest.java │ │ │ └── CassandraJoinJavaRDDTest.java │ │ └── mapper │ │ ├── JavaTestBean.java │ │ └── JavaTestUDTBean.java │ ├── resources │ ├── log4j.properties │ └── logback.xml │ └── scala │ ├── com │ └── datastax │ │ ├── bdp │ │ └── spark │ │ │ ├── DseAuthConfFactorySpec.scala │ │ │ └── DseByosAuthConfFactorySpec.scala │ │ ├── driver │ │ └── core │ │ │ └── RowMock.scala │ │ └── spark │ │ └── connector │ │ ├── ColumnSelectorSpec.scala │ │ ├── cql │ │ ├── CassandraConnectorConfSpec.scala │ │ └── DefaultConnectionFactoryTest.scala │ │ ├── embedded │ │ ├── Assertions.scala │ │ ├── Embedded.scala │ │ ├── SparkRepl.scala │ │ ├── SparkTemplate.scala │ │ ├── UserDefinedProperty.scala │ │ └── package.scala │ │ ├── rdd │ │ ├── CassandraRDDMock.scala │ │ ├── CqlWhereClauseSpec.scala │ │ ├── partitioner │ │ │ ├── BucketingRangeIndexSpec.scala │ │ │ ├── Murmur3PartitionerTokenRangeSplitterSpec.scala │ │ │ ├── RandomPartitionerTokenRangeSplitterSpec.scala │ │ │ ├── SplitterBehaviors.scala │ │ │ ├── TokenRangeClustererTest.scala │ │ │ └── dht │ │ │ │ ├── Murmur3TokenFactorySpec.scala │ │ │ │ ├── RandomPartitionerTokenFactorySpec.scala │ │ │ │ └── TokenRangeSpec.scala │ │ └── reader │ │ │ └── ClassBasedRowReaderTest.scala │ │ ├── samples.scala │ │ ├── testkit │ │ └── package.scala │ │ ├── util │ │ ├── AnyObjectFactoryTest.scala │ │ ├── BufferedIterator2Spec.scala │ │ ├── ConfigCheckSpec.scala │ │ ├── CqlWhereParserTest.scala │ │ ├── DeprecationSpec.scala │ │ ├── MergeJoinIteratorSpec.scala │ │ ├── PriorityHashMapSpec.scala │ │ └── SpanningIteratorSpec.scala │ │ └── writer │ │ ├── AsyncExecutorTest.scala │ │ ├── DefaultRowWriterTest.scala │ │ ├── RateLimiterSpec.scala │ │ ├── WriteConfTest.scala │ │ └── WriteOptionTest.scala │ └── org │ └── apache │ └── spark │ ├── metrics │ ├── InputMetricsUpdaterSpec.scala │ └── OutputMetricsUpdaterSpec.scala │ └── sql │ └── cassandra │ ├── ConsolidateSettingsSpec.scala │ ├── DsePredicateRulesSpec.scala │ ├── InClausePredicateRulesSpec.scala │ ├── PredicatePushDownSpec.scala │ └── SolrPredicateRulesSpec.scala ├── doc ├── 0_quick_start.md ├── 10_embedded.md ├── 11_metrics.md ├── 12_building_and_artifacts.md ├── 13_1_setup_spark_shell.md ├── 13_spark_shell.md ├── 14_data_frames.md ├── 15_python.md ├── 16_partitioning.md ├── 17_submitting.md ├── 1_connecting.md ├── 2_loading.md ├── 3_selection.md ├── 4_mapper.md ├── 5_saving.md ├── 6_advanced_mapper.md ├── 7_java_api.md ├── 8_streaming.md ├── 9_demos.md ├── FAQ.md ├── data_source_v1.md ├── developers.md └── reference.md ├── driver └── src │ ├── main │ ├── scala-2.12 │ │ └── types │ │ │ └── CanBuildFrom.scala │ ├── scala-2.13 │ │ └── com │ │ │ └── datastax │ │ │ └── spark │ │ │ └── connector │ │ │ └── types │ │ │ └── CanBuildFrom.scala │ └── scala │ │ └── com │ │ └── datastax │ │ └── spark │ │ └── connector │ │ ├── CassandraRow.scala │ │ ├── ColumnRef.scala │ │ ├── GettableByIndexData.scala │ │ ├── GettableData.scala │ │ ├── ScalaGettableByIndexData.scala │ │ ├── ScalaGettableData.scala │ │ ├── TableRef.scala │ │ ├── TupleValue.scala │ │ ├── UDTValue.scala │ │ ├── cql │ │ ├── LocalNodeFirstLoadBalancingPolicy.scala │ │ ├── MultipleRetryPolicy.scala │ │ ├── MultiplexingSchemaListener.scala │ │ └── Schema.scala │ │ ├── japi │ │ ├── CassandraRow.scala │ │ ├── JavaGettableByIndexData.scala │ │ ├── JavaGettableData.scala │ │ ├── TupleValue.scala │ │ └── UDTValue.scala │ │ ├── mapper │ │ ├── ColumnMap.scala │ │ ├── ColumnMapper.scala │ │ ├── ColumnMapperConvention.scala │ │ ├── DefaultColumnMapper.scala │ │ ├── GettableDataToMappedTypeConverter.scala │ │ ├── JavaBeanColumnMapper.scala │ │ ├── MappedToGettableDataConverter.scala │ │ ├── PropertyExtractor.scala │ │ ├── ReflectionColumnMapper.scala │ │ ├── TupleColumnMapper.scala │ │ └── package.scala │ │ ├── types │ │ ├── CassandraOption.scala │ │ ├── CollectionColumnType.scala │ │ ├── ColumnType.scala │ │ ├── PrimitiveColumnType.scala │ │ ├── TimestampFormatter.scala │ │ ├── TimestampParser.scala │ │ ├── TupleType.scala │ │ ├── TypeAdapters.scala │ │ ├── TypeConverter.scala │ │ ├── UserDefinedType.scala │ │ ├── VectorType.scala │ │ └── package.scala │ │ └── util │ │ ├── AnyObjectFactory.scala │ │ ├── ByteBufferUtil.scala │ │ ├── DriverUtil.scala │ │ ├── Logging.scala │ │ ├── NameTools.scala │ │ ├── Quote.scala │ │ ├── Reflect.scala │ │ ├── ReflectionUtil.scala │ │ └── Symbols.scala │ └── test │ ├── java │ └── com │ │ └── datastax │ │ └── spark │ │ └── connector │ │ └── mapper │ │ └── ColumnMapperTestUDTBean.java │ └── scala │ └── com │ └── datastax │ └── spark │ └── connector │ ├── CassandraRowTest.scala │ ├── ColumnRefSpec.scala │ ├── cql │ ├── LocalNodeFirstLoadBalancingPolicySpec.scala │ ├── MultipleRetryPolicySpec.scala │ ├── MultiplexingSchemaListenerTest.scala │ └── TableDefSpec.scala │ ├── mapper │ ├── DefaultColumnMapperTest.scala │ ├── GettableDataToMappedTypeConverterSpec.scala │ ├── JavaBeanColumnMapperTest.scala │ ├── MappedToGettableDataConverterSpec.scala │ ├── PropertyExtractorTest.scala │ └── TupleColumnMapperTest.scala │ ├── types │ ├── CanBuildFromTest.scala │ ├── CollectionColumnTypeSpec.scala │ ├── ColumnTypeSpec.scala │ ├── TimestampParserSpec.scala │ ├── TypeConverterTest.scala │ └── TypeSerializationTest.scala │ └── util │ └── ReflectionUtilSpec.scala ├── generateDocs.sh ├── project ├── Dependencies.scala ├── Publishing.scala ├── Testing.scala ├── Versions.scala ├── build.properties ├── plugins.sbt └── updates.sbt ├── rootdoc.txt ├── sbt └── sbt ├── scalastyle-config.xml ├── test-support └── src │ └── main │ ├── resources │ ├── client.crt │ ├── client.key │ ├── client.keystore │ ├── client.truststore │ ├── server.keystore │ ├── server.truststore │ └── server_localhost.keystore │ └── scala │ └── com │ └── datastax │ └── spark │ └── connector │ └── ccm │ ├── CcmBridge.scala │ ├── CcmConfig.scala │ ├── ClusterMode.scala │ └── mode │ ├── ClusterModeExecutor.scala │ ├── DebugModeExecutor.scala │ ├── DeveloperModeExecutor.scala │ ├── ExistingModeExecutor.scala │ └── StandardModeExecutor.scala └── testing └── scripts └── mac-localhost-aliases.sh /.asf.yaml: -------------------------------------------------------------------------------- 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 | 18 | notifications: 19 | commits: commits@cassandra.apache.org 20 | issues: commits@cassandra.apache.org 21 | pullrequests: pr@cassandra.apache.org 22 | jira_options: link worklog 23 | 24 | github: 25 | description: "Apache Spark to Apache Cassandra connector" 26 | homepage: https://apache.github.io/cassandra-spark-connector/ 27 | enabled_merge_buttons: 28 | squash: false 29 | merge: false 30 | rebase: true 31 | features: 32 | wiki: false 33 | issues: false 34 | projects: false 35 | discussions: false 36 | autolink_jira: 37 | - CASSANDRA 38 | - CASSANALYTICS 39 | protected_branches: 40 | trunk: 41 | required_linear_history: true 42 | -------------------------------------------------------------------------------- /.github/ISSUE_TEMPLATE/please-make-a-jira-ticket-or-ask-questions-on-the-mailing-list.md: -------------------------------------------------------------------------------- 1 | --- 2 | name: Please Make a Jira Ticket or Ask Questions on the Mailing List 3 | about: How to get help and report Bugs 4 | title: You shouldn't be making this issue :) 5 | labels: '' 6 | assignees: '' 7 | 8 | --- 9 | 10 | For issue tracking we use [JIRA](https://issues.apache.org/jira/projects/CASSANALYTICS) 11 | Community QA at [Apache Cassandra Community](https://cassandra.apache.org/_/community.html#discussions) 12 | -------------------------------------------------------------------------------- /.github/pull_request_template.md: -------------------------------------------------------------------------------- 1 | # Description 2 | 3 | ## How did the Spark Cassandra Connector Work or Not Work Before this Patch 4 | 5 | Describe the problem, or state of the project that this patch fixes. Explain 6 | why this is a problem if this isn't obvious. 7 | 8 | Example: 9 | "When I read from tables with 3 INTS I get a ThreeIntException(). This is a problem because I often want to read from a table with three integers." 10 | 11 | ## General Design of the patch 12 | 13 | How the fix is accomplished, were new parameters or classes added? Why did you 14 | pursue this particular fix? 15 | 16 | Example: "I removed the incorrect assertion which would throw the ThreeIntException. This exception was incorrectly added and the assertion is not actually needed." 17 | 18 | Fixes: [Put JIRA Reference HERE](https://issues.apache.org/jira/projects/CASSANALYTICS) 19 | 20 | # How Has This Been Tested? 21 | 22 | Almost all changes and especially bug fixes will require a test to be added to either the integration or Unit Tests. Any tests added will be automatically run on travis when the pull request is pushed to github. Be sure to run suites locally as well. 23 | 24 | # Checklist: 25 | 26 | - [ ] I have a ticket in the [JIRA](https://issues.apache.org/jira/projects/CASSANALYTICS) 27 | - [ ] I have performed a self-review of my own code 28 | - [ ] Locally all tests pass (make sure tests fail without your patch) 29 | -------------------------------------------------------------------------------- /.github/workflows/main.yml: -------------------------------------------------------------------------------- 1 | name: CI 2 | 3 | on: 4 | # Triggers the workflow on push or pull request events but only for the trunk branch 5 | push: 6 | pull_request: 7 | 8 | workflow_dispatch: 9 | 10 | jobs: 11 | build: 12 | runs-on: ubuntu-latest 13 | strategy: 14 | fail-fast: false 15 | matrix: 16 | scala: [2.12.19, 2.13.13] 17 | db-version: [3.11.19, 4.0.17, 4.1.8, 5.0.4, dse-6.8.44] 18 | 19 | steps: 20 | - uses: actions/checkout@v4 21 | 22 | - name: Install ccm via pip 23 | # if cassandra-ccm's trunk breaks this CI, please file a report, 24 | # and temporarily switch this to @cassandra-test or @ where sha is the last known working ccm commit 25 | run: pip install git+https://github.com/apache/cassandra-ccm.git@trunk 26 | 27 | - name: Setup Java 28 | uses: actions/setup-java@v4 29 | with: 30 | distribution: "temurin" 31 | java-version: | # order is important, the last one is the default which will be used by SBT 32 | 11 33 | 8 34 | 35 | - name: sbt tests 36 | env: 37 | TEST_PARALLEL_TASKS: 1 38 | CCM_CASSANDRA_VERSION: ${{ matrix.db-version }} 39 | PUBLISH_VERSION: test 40 | JAVA8_HOME: ${{ env.JAVA_HOME_8_X64 }} 41 | JAVA11_HOME: ${{ env.JAVA_HOME_11_X64 }} 42 | run: sbt/sbt ++${{ matrix.scala }} test it:test 43 | 44 | - name: Publish Test Report 45 | uses: mikepenz/action-junit-report@v4 46 | if: always() 47 | with: 48 | report_paths: '**/target/test-reports/*.xml' 49 | annotate_only: true 50 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | *.class 2 | *.log 3 | .DS_Store 4 | # sbt specific 5 | .cache/ 6 | .history/ 7 | .lib/ 8 | dist/* 9 | target/ 10 | lib_managed/ 11 | src_managed/ 12 | project/boot/ 13 | project/plugins/project/ 14 | sbt/sbt-launch*.jar 15 | cassandra-server/* 16 | metastore_db 17 | 18 | # Scala-IDE specific 19 | .scala_dependencies 20 | .worksheet 21 | .idea 22 | .idea_modules 23 | *.ipr 24 | *.iws 25 | *.iml 26 | 27 | checkpoint 28 | -------------------------------------------------------------------------------- /connector/src/it/resources/log4j.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, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | 17 | # for production, you should probably set pattern to %c instead of %l. 18 | # (%l is slower.) 19 | 20 | # output messages into a rolling log file as well as stdout 21 | log4j.rootLogger=WARN,stdout 22 | 23 | # stdout 24 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 25 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 26 | log4j.appender.stdout.layout.ConversionPattern=%5p %d{HH:mm:ss,SSS} [T%X{TEST_GROUP_NO}] %C (%F:%L) - %m%n 27 | 28 | # Avoid "no host ID found" when starting a fresh node 29 | log4j.logger.org.apache.cassandra.db.SystemKeyspace=ERROR 30 | 31 | # Avoid "address already in use" when starting multiple local Spark masters 32 | log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR 33 | log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR 34 | 35 | # Suppress some warnings 36 | log4j.logger.com.datastax.driver.core.NettyUtil=ERROR 37 | log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR 38 | log4j.logger.org.apache.cassandra.utils.CLibrary=ERROR 39 | log4j.logger.org.apache.cassandra.service.StartupChecks=ERROR 40 | log4j.logger.org.spark-project.jetty.server.Server=ERROR 41 | log4j.logger.org.eclipse.jetty.server.Server=ERROR 42 | 43 | #See CCM Bridge INFO 44 | log4j.logger.com.datastax.spark.connector.ccm=INFO 45 | -------------------------------------------------------------------------------- /connector/src/it/resources/log4j2.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | 18 | # Set everything to be logged to the console 19 | rootLogger.level = warn 20 | rootLogger.appenderRef.stdout.ref = console 21 | 22 | appender.console.type = Console 23 | appender.console.name = console 24 | appender.console.target = SYSTEM_OUT 25 | appender.console.layout.type = PatternLayout 26 | appender.console.layout.pattern = %5p %d{HH:mm:ss,SSS} [T%X{TEST_GROUP_NO}] %C (%F:%L) - %m%n 27 | 28 | logger.ccm.name = com.datastax.spark.connector.ccm 29 | logger.ccm.level = info 30 | -------------------------------------------------------------------------------- /connector/src/it/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | %d{yyyy-MM-dd HH:mm:ss} [%thread] %-5level %logger{36} - %msg%n 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | -------------------------------------------------------------------------------- /connector/src/it/resources/metrics.properties: -------------------------------------------------------------------------------- 1 | #*.sink.csv.class=org.apache.spark.metrics.sink.CsvSink 2 | # 3 | ## Polling period for CsvSink 4 | #*.sink.csv.period=1 5 | # 6 | #*.sink.csv.unit=seconds 7 | # 8 | ## Polling directory for CsvSink 9 | #*.sink.csv.directory=/tmp/spark/sink 10 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/cql/CassandraSSLClientAuthConnectorSpec.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.cql 20 | 21 | import com.datastax.spark.connector.SparkCassandraITFlatSpecBase 22 | import com.datastax.spark.connector.cluster.SSLCluster 23 | 24 | class CassandraSSLClientAuthConnectorSpec extends SparkCassandraITFlatSpecBase with SSLCluster { 25 | 26 | override lazy val conn = CassandraConnector(defaultConf) 27 | 28 | "A CassandraConnector" should "be able to use a secure connection when using native protocol" in { 29 | conn.withSessionDo { session => 30 | assert(session !== null) 31 | assert(session.isClosed === false) 32 | } 33 | } 34 | 35 | } 36 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/cql/CassandraSSLConnectorSpec.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.cql 20 | 21 | import com.datastax.spark.connector.SparkCassandraITFlatSpecBase 22 | import com.datastax.spark.connector.cluster.SSLCluster 23 | 24 | class CassandraSSLConnectorSpec extends SparkCassandraITFlatSpecBase with SSLCluster { 25 | 26 | override lazy val conn = CassandraConnector(defaultConf) 27 | 28 | "A CassandraConnector" should "be able to use a secure connection when using native protocol" in { 29 | conn.withSessionDo { session => 30 | assert(session !== null) 31 | assert(session.isClosed === false) 32 | } 33 | } 34 | 35 | } 36 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/CassandraRDDMockSpec.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd 20 | 21 | import com.datastax.spark.connector.cluster.DefaultCluster 22 | import com.datastax.spark.connector.{CassandraRow, SparkCassandraITFlatSpecBase} 23 | import com.datastax.spark.connector.cql.CassandraConnector 24 | 25 | class CassandraRDDMockSpec extends SparkCassandraITFlatSpecBase with DefaultCluster { 26 | 27 | override lazy val conn = CassandraConnector(defaultConf) 28 | 29 | "A CassandraRDDMock" should "behave like a CassandraRDD without needing Cassandra" in { 30 | val columns = Seq("key", "value") 31 | //Create a fake CassandraRDD[CassandraRow] 32 | val rdd = sc 33 | .parallelize(1 to 10) 34 | .map(num => CassandraRow.fromMap(columns.zip(Seq(num, num)).toMap)) 35 | 36 | val fakeCassandraRDD: CassandraRDD[CassandraRow] = new CassandraRDDMock(rdd) 37 | 38 | fakeCassandraRDD.cassandraCount() should be (10) 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/AsciiTypeTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import com.datastax.oss.driver.api.core.cql.Row 22 | import com.datastax.spark.connector.cluster.DefaultCluster 23 | 24 | class AsciiTypeTest extends AbstractTypeTest[String, String] with DefaultCluster { 25 | override val typeName = "ascii" 26 | 27 | override val typeData: Seq[String] = Seq("row1", "row2", "row3", "row4", "row5") 28 | override val addData: Seq[String] = Seq("row6", "row7", "row8", "row9", "row10") 29 | 30 | override def getDriverColumn(row: Row, colName: String): String = { 31 | row.getString(colName) 32 | } 33 | 34 | } 35 | 36 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/BigintTypeTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import java.lang.Long 22 | 23 | import com.datastax.oss.driver.api.core.cql.Row 24 | import com.datastax.spark.connector.cluster.DefaultCluster 25 | 26 | class BigintTypeTest extends AbstractTypeTest[Long, Long] with DefaultCluster { 27 | override val typeName = "bigint" 28 | 29 | override val typeData: Seq[Long] = Seq(new Long(1000000L), new Long(2000000L), new Long(3000000L), new Long(4000000L), new Long(5000000L)) 30 | override val addData: Seq[Long] = Seq(new Long(6000000000L), new Long(70000000L), new Long(80000000L), new Long(9000000L), new Long(10000000L)) 31 | 32 | override def getDriverColumn(row: Row, colName: String): Long = { 33 | row.getLong(colName) 34 | } 35 | } 36 | 37 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/BooleanTypeTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import java.lang.Boolean 22 | 23 | import com.datastax.oss.driver.api.core.cql.Row 24 | import com.datastax.spark.connector.cluster.DefaultCluster 25 | 26 | class BooleanTypeTest extends AbstractTypeTest[Boolean, Boolean] with DefaultCluster { 27 | override val typeName = "boolean" 28 | 29 | override val typeData: Seq[Boolean] = Seq(new Boolean(true)) 30 | override val addData: Seq[Boolean] = Seq(new Boolean(false)) 31 | 32 | override def getDriverColumn(row: Row, colName: String): Boolean = { 33 | row.getBoolean(colName) 34 | } 35 | 36 | } 37 | 38 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/DateTypeCETTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import java.util.TimeZone 22 | 23 | import com.datastax.spark.connector.cluster.CETCluster 24 | 25 | class DateTypeCETTest extends DateTypeTest(TimeZone.getTimeZone("CET")) with CETCluster { 26 | } 27 | 28 | class SqlDateTypeCETTest extends SqlDateTypeTest(TimeZone.getTimeZone("CET")) with CETCluster { 29 | } 30 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/DateTypeCSTTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import java.util.TimeZone 22 | 23 | import com.datastax.spark.connector.cluster.{CETCluster, CSTCluster} 24 | 25 | class DateTypeCSTTest extends DateTypeTest(TimeZone.getTimeZone("CST")) with CSTCluster { 26 | } 27 | 28 | class SqlDateTypeCSTTest extends SqlDateTypeTest(TimeZone.getTimeZone("CST")) with CSTCluster { 29 | } 30 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/DateTypePSTTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import java.util.TimeZone 22 | 23 | import com.datastax.spark.connector.cluster.PSTCluster 24 | 25 | class DateTypePSTTest extends DateTypeTest(TimeZone.getTimeZone("PST")) with PSTCluster { 26 | } 27 | 28 | class SqlDateTypePSTTest extends SqlDateTypeTest(TimeZone.getTimeZone("PST")) with PSTCluster { 29 | } 30 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/DecimalTypeTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import com.datastax.oss.driver.api.core.cql.Row 22 | import com.datastax.spark.connector.cluster.DefaultCluster 23 | 24 | 25 | class DecimalTypeTest extends AbstractTypeTest[BigDecimal, java.math.BigDecimal] with DefaultCluster { 26 | 27 | implicit def toBigDecimal(str: String) = BigDecimal(str) 28 | 29 | override def convertToDriverInsertable(testValue: BigDecimal): java.math.BigDecimal = testValue.bigDecimal 30 | 31 | override val typeName = "decimal" 32 | 33 | override val typeData: Seq[BigDecimal] = Seq("100.1", "200.2", "301.1") 34 | override val addData: Seq[BigDecimal] = Seq("600.6", "700.7", "721.444") 35 | 36 | override def getDriverColumn(row: Row, colName: String): BigDecimal = { 37 | BigDecimal(row.getBigDecimal(colName)) 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/DoubleTypeTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import java.lang.Double 22 | 23 | import com.datastax.oss.driver.api.core.cql.Row 24 | import com.datastax.spark.connector.cluster.DefaultCluster 25 | 26 | class DoubleTypeTest extends AbstractTypeTest[Double, Double] with DefaultCluster { 27 | override val typeName = "double" 28 | 29 | override val typeData: Seq[Double] = Seq(new Double(100.1), new Double(200.2),new Double(300.3), new Double(400.4), new Double(500.5)) 30 | override val addData: Seq[Double] = Seq(new Double(600.6), new Double(700.7), new Double(800.8), new Double(900.9), new Double(1000.12)) 31 | 32 | override def getDriverColumn(row: Row, colName: String): Double = { 33 | row.getDouble(colName) 34 | } 35 | } 36 | 37 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/FloatTypeTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import java.lang.Float 22 | 23 | import com.datastax.oss.driver.api.core.cql.Row 24 | import com.datastax.spark.connector.cluster.DefaultCluster 25 | 26 | class FloatTypeTest extends AbstractTypeTest[Float, Float] with DefaultCluster { 27 | override val typeName = "float" 28 | 29 | override val typeData: Seq[Float] = Seq(new Float(100.1), new Float(200.2),new Float(300.3), new Float(400.4), new Float(500.5)) 30 | override val addData: Seq[Float] = Seq(new Float(600.6), new Float(700.7), new Float(800.8), new Float(900.9), new Float(1000.12)) 31 | 32 | override def getDriverColumn(row: Row, colName: String): Float = { 33 | row.getFloat(colName) 34 | } 35 | 36 | } 37 | 38 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/IntTypeTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import com.datastax.oss.driver.api.core.cql.Row 22 | import com.datastax.spark.connector.cluster.DefaultCluster 23 | 24 | class IntTypeTest extends AbstractTypeTest[Integer, Integer] with DefaultCluster { 25 | override val typeName = "int" 26 | 27 | override val typeData: Seq[Integer] = Seq(new Integer(1), new Integer(2), new Integer(3), new Integer(4), new Integer(5)) 28 | override val addData: Seq[Integer] = Seq(new Integer(6), new Integer(7), new Integer(8), new Integer(9), new Integer(10)) 29 | 30 | override def getDriverColumn(row: Row, colName: String): Integer = { 31 | row.getInt(colName) 32 | } 33 | 34 | } 35 | 36 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/SmallIntTypeTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import com.datastax.oss.driver.api.core.DefaultProtocolVersion 22 | import com.datastax.oss.driver.api.core.cql.Row 23 | import com.datastax.spark.connector.cluster.DefaultCluster 24 | 25 | class SmallIntTypeTest extends AbstractTypeTest[Short, java.lang.Short] with DefaultCluster { 26 | override val minPV = DefaultProtocolVersion.V4 27 | override protected val typeName: String = "smallint" 28 | 29 | override protected val typeData: Seq[Short] = (1 to 10).map(_.toShort) 30 | override protected val addData: Seq[Short] = (11 to 20).map(_.toShort) 31 | 32 | override def getDriverColumn(row: Row, colName: String): Short = row.getShort(colName) 33 | 34 | } 35 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/TextTypeTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import com.datastax.oss.driver.api.core.cql.Row 22 | import com.datastax.spark.connector.cluster.DefaultCluster 23 | 24 | class TextTypeTest extends AbstractTypeTest[String, String] with DefaultCluster { 25 | override val typeName = "text" 26 | 27 | override val typeData: Seq[String] = Seq("काचं शक्नोम्यत्तुम् । नोपहिनस्ति माम् ॥", "⠊⠀⠉⠁⠝⠀⠑⠁⠞⠀⠛⠇⠁⠎⠎⠀⠁⠝⠙⠀⠊⠞⠀⠙⠕⠑⠎⠝⠞⠀⠓⠥⠗⠞⠀⠍⠑", "אני יכול לאכול זכוכית וזה לא מזיק לי.", " நான் கண்ணாடி சாப்பிடுவேன், அதனால் எனக்கு ஒரு கேடும் வராது.", " ᠪᠢ ᠰᠢᠯᠢ ᠢᠳᠡᠶᠦ ᠴᠢᠳᠠᠨᠠ ᠂ ᠨᠠᠳᠤᠷ ᠬᠣᠤᠷᠠᠳᠠᠢ ᠪᠢᠰᠢ ") 28 | override val addData: Seq[String] = Seq(" ᚛᚛ᚉᚑᚅᚔᚉᚉᚔᚋ ᚔᚈᚔ ᚍᚂᚐᚅᚑ ᚅᚔᚋᚌᚓᚅᚐ᚜", "I kaun Gloos essen, es tuat ma ned weh.", " Meg tudom enni az üveget, nem lesz tőle bajom", "Можам да јадам стакло, а не ме штета.", "Կրնամ ապակի ուտել և ինծի անհանգիստ չըներ։") 29 | 30 | override def getDriverColumn(row: Row, colName: String): String = { 31 | row.getString(colName) 32 | } 33 | } 34 | 35 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/TimeTypeTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import java.time.LocalTime 22 | 23 | import com.datastax.spark.connector._ 24 | 25 | import com.datastax.oss.driver.api.core.DefaultProtocolVersion 26 | import com.datastax.oss.driver.api.core.cql.Row 27 | import com.datastax.spark.connector.cluster.DefaultCluster 28 | 29 | class TimeTypeTest extends AbstractTypeTest[LocalTime, LocalTime] with DefaultCluster { 30 | 31 | override val minPV = DefaultProtocolVersion.V4 32 | 33 | override def getDriverColumn(row: Row, colName: String): LocalTime = row.getLocalTime(colName) 34 | 35 | override protected val typeName: String = "time" 36 | 37 | override protected val typeData: Seq[LocalTime] = (1L to 5L).map(LocalTime.ofNanoOfDay) 38 | override protected val addData: Seq[LocalTime] = (6L to 10L).map(LocalTime.ofNanoOfDay) 39 | 40 | "Time Types" should "be writable as dates" in skipIfProtocolVersionLT(minPV) { 41 | val times = (100 to 500 by 100).map(LocalTime.ofNanoOfDay(_)) 42 | sc.parallelize(times.map(x => (x, x, x, x))).saveToCassandra(keyspaceName, typeNormalTable) 43 | val results = sc.cassandraTable[(LocalTime, LocalTime, LocalTime, LocalTime)](keyspaceName, typeNormalTable).collect 44 | checkNormalRowConsistency(times, results) 45 | } 46 | 47 | } 48 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/TimeUUIDTypeTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import java.util.UUID 22 | 23 | import com.datastax.oss.driver.api.core.cql.Row 24 | import com.datastax.spark.connector.cluster.DefaultCluster 25 | 26 | class TimeUUIDTypeTest extends AbstractTypeTest[UUID, UUID] with DefaultCluster { 27 | 28 | override val typeName = "timeuuid" 29 | 30 | override val typeData: Seq[UUID] = Seq(UUID.fromString("61129590-FBE4-11E3-A3AC-0800200C9A66"), UUID.fromString("61129591-FBE4-11E3-A3AC-0800200C9A66"), 31 | UUID.fromString("61129592-FBE4-11E3-A3AC-0800200C9A66"), UUID.fromString("61129593-FBE4-11E3-A3AC-0800200C9A66"), UUID.fromString("61129594-FBE4-11E3-A3AC-0800200C9A66")) 32 | override val addData: Seq[UUID] = Seq(UUID.fromString("204FF380-FBE5-11E3-A3AC-0800200C9A66"), UUID.fromString("204FF381-FBE5-11E3-A3AC-0800200C9A66"), UUID.fromString("204FF382-FBE5-11E3-A3AC-0800200C9A66"), UUID.fromString("204FF383-FBE5-11E3-A3AC-0800200C9A66"), UUID.fromString("204FF384-FBE5-11E3-A3AC-0800200C9A66")) 33 | 34 | override def getDriverColumn(row: Row, colName: String): UUID = { 35 | row.getUuid(colName) 36 | } 37 | } 38 | 39 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/TimestampTypeTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import java.text.SimpleDateFormat 22 | import java.time.Instant 23 | import java.util.Date 24 | 25 | import com.datastax.oss.driver.api.core.cql.Row 26 | import com.datastax.spark.connector.cluster.DefaultCluster 27 | 28 | class TimestampTypeTest extends AbstractTypeTest[Instant, Instant] with DefaultCluster { 29 | override val typeName = "timestamp" 30 | val sdf = new SimpleDateFormat("dd/MM/yyyy") 31 | 32 | override val typeData: Seq[Instant] = Seq( 33 | sdf.parse("03/08/1985"), 34 | sdf.parse("03/08/1986"), 35 | sdf.parse("03/08/1987"), 36 | sdf.parse("03/08/1988"), 37 | sdf.parse("03/08/1989")).map(_.toInstant) 38 | override val addData: Seq[Instant] = Seq( 39 | sdf.parse("03/08/1990"), 40 | sdf.parse("03/08/1991"), 41 | sdf.parse("03/08/1992"), 42 | sdf.parse("03/08/1993"), 43 | sdf.parse("03/08/1994")).map(_.toInstant) 44 | 45 | override def getDriverColumn(row: Row, colName: String): Instant = { 46 | row.getInstant(colName) 47 | } 48 | 49 | } 50 | 51 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/rdd/typeTests/TinyIntTypeTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.typeTests 20 | 21 | import com.datastax.oss.driver.api.core.DefaultProtocolVersion 22 | import com.datastax.oss.driver.api.core.cql.Row 23 | import com.datastax.spark.connector.cluster.DefaultCluster 24 | 25 | class TinyIntTypeTest extends AbstractTypeTest[Int, java.lang.Byte] with DefaultCluster { 26 | override val minPV = DefaultProtocolVersion.V4 27 | override protected val typeName: String = "tinyint" 28 | 29 | override protected val typeData: Seq[Int] =Seq(1, 2, 3, 4, 5) 30 | override protected val addData: Seq[Int] = Seq(6, 7, 8, 9, 10) 31 | 32 | override def getDriverColumn(row: Row, colName: String): Int = { 33 | row.getByte(colName).toInt 34 | } 35 | 36 | override def convertToDriverInsertable(testValue: Int): java.lang.Byte = testValue.toByte 37 | 38 | 39 | } 40 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/sql/CassandraDataFrameDateCETSpec.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.sql 20 | 21 | import java.util.TimeZone 22 | 23 | import com.datastax.spark.connector.cluster.CETCluster 24 | import org.scalatest.FlatSpec 25 | 26 | /** 27 | * This should be executed in separate JVM, as Catalyst caches default time zone 28 | */ 29 | class CassandraDataFrameDateCETSpec extends FlatSpec with CassandraDataFrameDateBehaviors with CETCluster { 30 | 31 | val centralEuropeanTimeZone = TimeZone.getTimeZone("CET") 32 | 33 | "A DataFrame in CET timezone" should behave like dataFrame(centralEuropeanTimeZone) 34 | } 35 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/sql/CassandraDataFrameDatePSTSpec.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.sql 20 | 21 | import java.util.TimeZone 22 | 23 | import com.datastax.spark.connector.cluster.PSTCluster 24 | import org.scalatest.FlatSpec 25 | 26 | /** 27 | * This should be executed in separate JVM, as Catalyst caches default time zone 28 | */ 29 | class CassandraDataFrameDatePSTSpec extends FlatSpec with CassandraDataFrameDateBehaviors with PSTCluster { 30 | 31 | val pacificTimeZone = TimeZone.getTimeZone("PST") 32 | 33 | "A DataFrame in PST timezone" should behave like dataFrame(pacificTimeZone) 34 | } 35 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/sql/CassandraDataSourceNoPushdownSpec.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.sql 20 | 21 | class CassandraDataSourceNoPushdownSpec extends CassandraDataSourceSpec { 22 | override def pushDown: Boolean = false 23 | } 24 | -------------------------------------------------------------------------------- /connector/src/it/scala/com/datastax/spark/connector/util/CatalystUtil.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.util 20 | 21 | import com.datastax.spark.connector.datasource.CassandraScan 22 | import org.apache.spark.sql.execution._ 23 | import org.apache.spark.sql.execution.datasources.v2.BatchScanExec 24 | 25 | object CatalystUtil { 26 | 27 | def findCassandraScan(sparkPlan: SparkPlan): Option[CassandraScan] = { 28 | sparkPlan.collectFirst{ case BatchScanExec(_, scan: CassandraScan, _, _, _, _) => scan} 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /connector/src/it/scala/org/apache/spark/DseTestUtil.scala: -------------------------------------------------------------------------------- 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.spark 20 | 21 | import org.apache.spark.rdd.UnionPartition 22 | 23 | object DseTestUtil { 24 | 25 | //UnionPartition is private[spark], expose it's parentPartition method 26 | def getParentPartition[T](unionPartition: Partition): Partition = { 27 | unionPartition match { case unionPartition: UnionPartition[T @unchecked] => unionPartition.parentPartition} 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /connector/src/main/java/com/datastax/spark/connector/japi/StreamingContextJavaFunctions.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 com.datastax.spark.connector.japi; 20 | 21 | import org.apache.spark.streaming.StreamingContext; 22 | 23 | /** 24 | * Java API wrapper over {@link org.apache.spark.streaming.StreamingContext} to provide Spark Cassandra Connector 25 | * functionality. 26 | * 27 | *

To obtain an instance of this wrapper, use one of the factory methods in {@link 28 | * com.datastax.spark.connector.japi.CassandraJavaUtil} class.

29 | */ 30 | @SuppressWarnings("UnusedDeclaration") 31 | public class StreamingContextJavaFunctions extends SparkContextJavaFunctions { 32 | public final StreamingContext ssc; 33 | 34 | StreamingContextJavaFunctions(StreamingContext ssc) { 35 | super(ssc.sparkContext()); 36 | this.ssc = ssc; 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /connector/src/main/main.iml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | -------------------------------------------------------------------------------- /connector/src/main/scala-2.12/com/datastax/spark/connector/util/RuntimeUtil.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.util 20 | 21 | import org.apache.spark.repl.SparkILoop 22 | import scala.tools.nsc.Settings 23 | import java.io.{BufferedReader, PrintWriter} 24 | import scala.collection.parallel.ParIterable 25 | 26 | class Scala213SparkILoop(in: BufferedReader, out: PrintWriter) extends SparkILoop(in, out) { 27 | 28 | def run(interpreterSettings: Settings): Boolean = { 29 | super.process(interpreterSettings) 30 | } 31 | } 32 | 33 | 34 | object RuntimeUtil { 35 | 36 | def toParallelIterable[A](iterable: Iterable[A]): ParIterable[A] = { 37 | iterable.par 38 | } 39 | 40 | def createSparkILoop(in: BufferedReader, out: PrintWriter): Scala213SparkILoop = { 41 | new Scala213SparkILoop(in, out) 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /connector/src/main/scala-2.13/com/datastax/spark/connector/util/RuntimeUtil.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.util 20 | 21 | import org.apache.spark.repl.SparkILoop 22 | 23 | import java.io.{BufferedReader, PrintWriter} 24 | import scala.collection.parallel.ParIterable 25 | 26 | 27 | object RuntimeUtil { 28 | 29 | def toParallelIterable[A](iterable: Iterable[A]): ParIterable[A] = { 30 | import scala.collection.parallel.CollectionConverters._ 31 | iterable.par 32 | } 33 | 34 | def createSparkILoop(in: BufferedReader, out: PrintWriter): SparkILoop = { 35 | new SparkILoop(in, out) 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/bdp/util/MiscUtil.scala: -------------------------------------------------------------------------------- 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 | 20 | package com.datastax.bdp.util 21 | 22 | import scala.reflect.runtime.universe 23 | 24 | object MiscUtil { 25 | def objectOrClassName(o: AnyRef): String = { 26 | val mirror = universe.runtimeMirror(o.getClass.getClassLoader) 27 | mirror.reflect(o).symbol.asClass.fullName 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/BatchSize.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | import com.datastax.spark.connector.writer.WriteConf 22 | 23 | sealed trait BatchSize 24 | 25 | case class RowsInBatch(batchSize: Int) extends BatchSize 26 | case class BytesInBatch(batchSize: Int) extends BatchSize 27 | 28 | object BatchSize { 29 | @deprecated("Use com.datastax.spark.connector.FixedBatchSize instead of a number", "1.1") 30 | implicit def intToFixedBatchSize(batchSize: Int): RowsInBatch = RowsInBatch(batchSize) 31 | 32 | val Automatic = BytesInBatch(WriteConf.BatchSizeBytesParam.default) 33 | } 34 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/CassandraSparkExtensions.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | import org.apache.spark.sql.{SparkSessionExtensions, catalyst} 22 | import org.apache.spark.sql.cassandra.execution.CassandraDirectJoinStrategy 23 | import org.apache.spark.sql.cassandra.{CassandraMetaDataRule, CassandraMetadataFunction} 24 | import org.apache.spark.sql.catalyst.FunctionIdentifier 25 | import com.datastax.spark.connector.util.Logging 26 | import org.apache.spark.sql.catalyst.expressions.Expression 27 | 28 | class CassandraSparkExtensions extends (SparkSessionExtensions => Unit) with Logging { 29 | override def apply(extensions: SparkSessionExtensions): Unit = { 30 | extensions.injectPlannerStrategy(CassandraDirectJoinStrategy.apply) 31 | extensions.injectResolutionRule(session => CassandraMetaDataRule) 32 | extensions.injectFunction(CassandraMetadataFunction.cassandraTTLFunctionDescriptor) 33 | extensions.injectFunction(CassandraMetadataFunction.cassandraWriteTimeFunctionDescriptor) 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/DocUtil.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | import java.nio.file.{FileSystems, Files, Paths} 22 | 23 | import com.datastax.spark.connector.util.{ConfigCheck, RefBuilder} 24 | 25 | object DocUtil { 26 | 27 | def main(args: Array[String]) { 28 | 29 | val DefaultReferenceFile = Paths.get("..").resolve("doc").resolve("reference.md") 30 | 31 | println("Generating Reference Documentation for Spark Cassandra Conenctor") 32 | println(s"Found ${ConfigCheck.validStaticProperties.size} Parameters") 33 | 34 | val markdown = RefBuilder.getMarkDown() 35 | 36 | println(s"Generating Reference Documentation for Spark Cassandra Conenctor to ${DefaultReferenceFile.toAbsolutePath}") 37 | 38 | Files.write(DefaultReferenceFile, markdown.getBytes) 39 | 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/PairRDDFunctions.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | import com.datastax.spark.connector.rdd.SpannedByKeyRDD 22 | import org.apache.spark.rdd.RDD 23 | 24 | class PairRDDFunctions[K, V](rdd: RDD[(K, V)]) extends Serializable { 25 | 26 | /** 27 | * Groups items with the same key, assuming the items with the same key are next to each other 28 | * in the collection. It does not perform shuffle, therefore it is much faster than using 29 | * much more universal Spark RDD `groupByKey`. For this method to be useful with Cassandra tables, 30 | * the key must represent a prefix of the primary key, containing at least the partition key of the 31 | * Cassandra table. */ 32 | def spanByKey: RDD[(K, Seq[V])] = 33 | new SpannedByKeyRDD[K, V](rdd) 34 | 35 | } 36 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/cql/QueryUtils.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.cql 20 | 21 | import java.nio.ByteBuffer 22 | 23 | import com.datastax.oss.driver.api.core.cql.BoundStatement 24 | import com.datastax.spark.connector.writer.NullKeyColumnException 25 | 26 | import scala.jdk.CollectionConverters._ 27 | 28 | object QueryUtils { 29 | /** 30 | * If a bound statement has all partition key components bound it will 31 | * return a routing key, but if all components are not bound it returns 32 | * null. When this is the case we want to let the user know which columns 33 | * were not correctly bound 34 | * @param bs a statement completely bound with all parameters 35 | * @return The routing key 36 | */ 37 | def getRoutingKeyOrError(bs: BoundStatement): ByteBuffer = { 38 | val routingKey = bs.getRoutingKey 39 | if (routingKey == null) throw new NullKeyColumnException(nullPartitionKeyValues(bs)) 40 | routingKey 41 | } 42 | 43 | private def nullPartitionKeyValues(bs: BoundStatement) = { 44 | val pkIndicies = bs.getPreparedStatement.getPartitionKeyIndices 45 | val boundValues = bs.getValues 46 | pkIndicies.asScala 47 | .filter(bs.isNull(_)) 48 | .map(bs.getPreparedStatement.getVariableDefinitions.get(_)) 49 | .map(_.getName) 50 | .mkString(", ") 51 | } 52 | 53 | 54 | } 55 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/cql/package.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | import com.datastax.oss.driver.api.core.cql.Row 22 | 23 | 24 | /** Contains a [[cql.CassandraConnector]] object which is used to connect 25 | * to a Cassandra cluster and to send CQL statements to it. `CassandraConnector` 26 | * provides a Scala-idiomatic way of working with `Session` object 27 | * and takes care of connection pooling and proper resource disposal.*/ 28 | package object cql { 29 | 30 | def getRowBinarySize(row: Row): Int = { 31 | var size = 0 32 | for (i <- 0 until row.getColumnDefinitions.size() if !row.isNull(i)) 33 | size += row.getBytesUnsafe(i).remaining() 34 | size 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/rdd/CassandraLimit.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd 20 | 21 | sealed trait CassandraLimit 22 | 23 | case class CassandraPartitionLimit(rowsNumber: Long) extends CassandraLimit { 24 | require(rowsNumber > 0, s"$rowsNumber <= 0. Per Partition Limits must be greater than 0") 25 | } 26 | case class SparkPartitionLimit(rowsNumber: Long) extends CassandraLimit { 27 | require(rowsNumber > 0, s"$rowsNumber <= 0. Limits must be greater than 0") 28 | } 29 | 30 | object CassandraLimit { 31 | 32 | def limitToClause 33 | (limit: Option[CassandraLimit]): String = limit match { 34 | case Some(SparkPartitionLimit(rowsNumber)) => s"LIMIT $rowsNumber" 35 | case Some(CassandraPartitionLimit(rowsNumber)) => s"PER PARTITION LIMIT $rowsNumber" 36 | case None => "" 37 | } 38 | 39 | def limitForIterator(limit: Option[CassandraLimit]): Option[Long] = limit.collect { 40 | case SparkPartitionLimit(rowsNumber) => rowsNumber 41 | } 42 | } 43 | 44 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/rdd/ClusteringOrder.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd 20 | 21 | import com.datastax.spark.connector.cql.TableDef 22 | 23 | sealed trait ClusteringOrder extends Serializable { 24 | private[connector] def toCql(tableDef: TableDef): String 25 | } 26 | 27 | object ClusteringOrder { 28 | private[connector] def cqlClause(tableDef: TableDef, order: String) = 29 | tableDef.clusteringColumns.headOption.map(cc => s"""ORDER BY "${cc.columnName}" $order""") 30 | .getOrElse(throw new IllegalArgumentException("Order by can be specified only if there are some clustering columns")) 31 | 32 | case object Ascending extends ClusteringOrder { 33 | override private[connector] def toCql(tableDef: TableDef): String = cqlClause(tableDef, "ASC") 34 | } 35 | 36 | case object Descending extends ClusteringOrder { 37 | override private[connector] def toCql(tableDef: TableDef): String = cqlClause(tableDef, "DESC") 38 | } 39 | 40 | } 41 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/rdd/SpannedByKeyRDD.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd 20 | 21 | import org.apache.spark.{TaskContext, Partition} 22 | import org.apache.spark.annotation.DeveloperApi 23 | import org.apache.spark.rdd.RDD 24 | 25 | import com.datastax.spark.connector.util.SpanningIterator 26 | 27 | /** 28 | * Similar to [[SpannedRDD]] but, instead of extracting the key by the given function, 29 | * it groups binary tuples by the first element of each tuple. 30 | */ 31 | private[connector] class SpannedByKeyRDD[K, V](parent: RDD[(K, V)]) extends RDD[(K, Seq[V])](parent) { 32 | 33 | override protected def getPartitions = parent.partitions 34 | 35 | @DeveloperApi 36 | override def compute(split: Partition, context: TaskContext) = { 37 | val parentIterator = parent.iterator(split, context) 38 | def keyFunction(item: (K, V)) = item._1 39 | def extractValues(group: (K, Seq[(K, V)])) = (group._1, group._2.map(_._2)) 40 | new SpanningIterator(parentIterator, keyFunction).map(extractValues) 41 | } 42 | 43 | } 44 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/rdd/SpannedRDD.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd 20 | 21 | import org.apache.spark.{TaskContext, Partition} 22 | import org.apache.spark.annotation.DeveloperApi 23 | import org.apache.spark.rdd.RDD 24 | 25 | import com.datastax.spark.connector.util.SpanningIterator 26 | 27 | /** 28 | * Groups items with the same key, assuming items with the same key are next to each other in 29 | * the parent collection. Contrary to Spark GroupedRDD, it does not perform shuffle, therefore it 30 | * is much faster. A key for each item is obtained by calling a given function. 31 | * 32 | * This RDD is very useful for grouping data coming out from Cassandra, because they are already 33 | * coming in order of partitioning key i.e. it is not possible for two rows 34 | * with the same partition key to be in different Spark partitions. 35 | * 36 | * @param parent parent RDD 37 | * @tparam K type of keys 38 | * @tparam T type of elements to be grouped together 39 | */ 40 | private[connector] class SpannedRDD[K, T](parent: RDD[T], f: T => K) 41 | extends RDD[(K, Iterable[T])](parent) { 42 | 43 | override protected def getPartitions = parent.partitions 44 | 45 | @DeveloperApi 46 | override def compute(split: Partition, context: TaskContext) = 47 | new SpanningIterator(parent.iterator(split, context), f) 48 | 49 | } 50 | 51 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/rdd/ValidRDDType.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd 20 | 21 | import java.io.{Serializable => JavaSerializable} 22 | 23 | import com.datastax.spark.connector.types.TypeConverter 24 | 25 | import scala.annotation.implicitNotFound 26 | 27 | @implicitNotFound("Not a valid RDD type. There should exists either a type converter for the type or the type should implement Serializable") 28 | trait ValidRDDType[T] 29 | 30 | object ValidRDDType { 31 | implicit def withTypeConverterAsValidRDDType[T](implicit tc: TypeConverter[T]): ValidRDDType[T] = null 32 | 33 | implicit def javaSerializableAsValidRDDType[T <: JavaSerializable]: ValidRDDType[T] = null 34 | } 35 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/rdd/package.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | 22 | /** Contains [[com.datastax.spark.connector.rdd.CassandraTableScanRDD]] class that is the main entry point for 23 | * analyzing Cassandra data from Spark. */ 24 | package object rdd { 25 | 26 | } 27 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/Murmur3PartitionerTokenRangeSplitter.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.partitioner 20 | 21 | import com.datastax.spark.connector.rdd.partitioner.dht.LongToken 22 | 23 | /** Fast token range splitter assuming that data are spread out evenly in the whole range. */ 24 | private[partitioner] class Murmur3PartitionerTokenRangeSplitter 25 | extends TokenRangeSplitter[Long, LongToken] { 26 | 27 | private type TokenRange = com.datastax.spark.connector.rdd.partitioner.dht.TokenRange[Long, LongToken] 28 | 29 | override def split(tokenRange: TokenRange, splitSize: Int): Seq[TokenRange] = { 30 | val rangeSize = tokenRange.rangeSize 31 | val splitPointsCount = if (rangeSize < splitSize) rangeSize.toInt else splitSize 32 | val splitPoints = (0 until splitPointsCount).map({ i => 33 | new LongToken(tokenRange.start.value + (rangeSize * i / splitPointsCount).toLong) 34 | }) :+ tokenRange.end 35 | 36 | for (Seq(left, right) <- splitPoints.sliding(2).toSeq) yield 37 | new TokenRange(left, right, tokenRange.replicas, tokenRange.tokenFactory) 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/RandomPartitionerTokenRangeSplitter.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.partitioner 20 | 21 | import com.datastax.spark.connector.rdd.partitioner.dht.BigIntToken 22 | 23 | /** Fast token range splitter assuming that data are spread out evenly in the whole range. */ 24 | private[partitioner] class RandomPartitionerTokenRangeSplitter 25 | extends TokenRangeSplitter[BigInt, BigIntToken] { 26 | 27 | private type TokenRange = com.datastax.spark.connector.rdd.partitioner.dht.TokenRange[BigInt, BigIntToken] 28 | 29 | private def wrapWithMax(max: BigInt)(token: BigInt): BigInt = { 30 | if (token <= max) token else token - max 31 | } 32 | 33 | override def split(tokenRange: TokenRange, splitCount: Int): Seq[TokenRange] = { 34 | val rangeSize = tokenRange.rangeSize 35 | val wrap = wrapWithMax(tokenRange.tokenFactory.maxToken.value)(_) 36 | 37 | val splitPointsCount = if (rangeSize < splitCount) rangeSize.toInt else splitCount 38 | val splitPoints = (0 until splitPointsCount).map({ i => 39 | val nextToken: BigInt = tokenRange.start.value + (rangeSize * i / splitPointsCount) 40 | new BigIntToken(wrap(nextToken)) 41 | }) :+ tokenRange.end 42 | 43 | for (Seq(left, right) <- splitPoints.sliding(2).toSeq) yield 44 | new TokenRange(left, right, tokenRange.replicas, tokenRange.tokenFactory) 45 | } 46 | } -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/rdd/partitioner/package.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd 20 | 21 | /** Provides components for partitioning a Cassandra table into smaller parts of appropriate size. 22 | * Each partition can be processed locally on at least one cluster node. */ 23 | package object partitioner { 24 | 25 | } 26 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/rdd/reader/RowReader.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.reader 20 | 21 | import com.datastax.oss.driver.api.core.cql.Row 22 | import com.datastax.spark.connector.{CassandraRowMetadata, ColumnRef} 23 | 24 | /** Transforms a Cassandra Java driver `Row` into high-level row representation, e.g. a tuple 25 | * or a user-defined case class object. The target type `T` must be serializable. */ 26 | trait RowReader[T] extends Serializable { 27 | 28 | /** Reads column values from low-level `Row` and turns them into higher level representation. 29 | * 30 | * @param row row fetched from Cassandra 31 | * @param rowMetaData column names and codec available in the `row` */ 32 | def read(row: Row, rowMetaData: CassandraRowMetadata): T 33 | 34 | /** List of columns this `RowReader` is going to read. 35 | * Useful to avoid fetching the columns that are not needed. */ 36 | def neededColumns: Option[Seq[ColumnRef]] 37 | 38 | } 39 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/rdd/reader/package.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd 20 | 21 | import com.datastax.spark.connector.CassandraRow 22 | 23 | /** Provides components for reading data rows from Cassandra and converting them to objects of desired type. 24 | * Additionally provides a generic [[CassandraRow CassandraRow]] class which can represent any row.*/ 25 | package object reader { 26 | 27 | } 28 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/streaming/StreamingContextFunctions.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.streaming 20 | 21 | import com.datastax.spark.connector.cql.CassandraConnector 22 | import com.datastax.spark.connector.rdd.{ReadConf, ValidRDDType} 23 | import org.apache.spark.streaming.StreamingContext 24 | import com.datastax.spark.connector.SparkContextFunctions 25 | import com.datastax.spark.connector.rdd.reader.RowReaderFactory 26 | 27 | /** Provides Cassandra-specific methods on `org.apache.spark.streaming.StreamingContext`. 28 | * @param ssc the Spark Streaming context 29 | */ 30 | class StreamingContextFunctions (ssc: StreamingContext) extends SparkContextFunctions(ssc.sparkContext) { 31 | import scala.reflect.ClassTag 32 | 33 | override def cassandraTable[T](keyspace: String, table: String)( 34 | implicit 35 | connector: CassandraConnector = CassandraConnector(ssc.sparkContext), 36 | readConf: ReadConf = ReadConf.fromSparkConf(sc.getConf), 37 | ct: ClassTag[T], 38 | rrf: RowReaderFactory[T], 39 | ev: ValidRDDType[T]): CassandraStreamingRDD[T] = { 40 | 41 | new CassandraStreamingRDD[T](ssc, connector, keyspace, table, readConf = readConf) 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/streaming/package.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | import org.apache.spark.streaming.StreamingContext 22 | import org.apache.spark.streaming.dstream.DStream 23 | 24 | import scala.reflect.ClassTag 25 | 26 | package object streaming { 27 | 28 | implicit def toStreamingContextFunctions(ssc: StreamingContext): SparkContextFunctions = 29 | new StreamingContextFunctions(ssc) 30 | 31 | implicit def toDStreamFunctions[T: ClassTag](ds: DStream[T]): DStreamFunctions[T] = 32 | new DStreamFunctions[T](ds) 33 | 34 | } 35 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/types/ColumnTypeConf.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.types 20 | 21 | import com.datastax.spark.connector.util._ 22 | 23 | case class ColumnTypeConf(customFromDriver: Option[String]) 24 | 25 | object ColumnTypeConf { 26 | 27 | val ReferenceSection = "Custom Cassandra Type Parameters (Expert Use Only)" 28 | 29 | val deprecatedCustomDriverTypeParam = DeprecatedConfigParameter( 30 | "spark.cassandra.dev.customFromDriver", 31 | None, 32 | deprecatedSince = "Analytics Connector 1.0", 33 | rational = "The ability to load new driver type converters at runtime has been removed" 34 | ) 35 | 36 | } -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/util/ClassLoaderCheck.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.util 20 | 21 | 22 | /** 23 | * Do not remove. 24 | * This is a temporary marker class that is loaded by name during application shutdown. 25 | * See SPARKC-620 for details. 26 | */ 27 | object ClassLoaderCheck {} 28 | 29 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/util/CodecRegistryUtil.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.util 20 | 21 | import com.datastax.oss.driver.api.core.`type`.DataType 22 | import com.datastax.oss.driver.api.core.`type`.codec.TypeCodec 23 | import com.datastax.oss.driver.api.core.`type`.codec.registry.CodecRegistry 24 | 25 | 26 | object CodecRegistryUtil { 27 | def codecFor(registry: CodecRegistry, cqlType: DataType, value: AnyRef) : TypeCodec[AnyRef] = { 28 | if(value==null) registry.codecFor(cqlType) 29 | else registry.codecFor(cqlType, value) 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/util/CountingIterator.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.util 20 | 21 | /** Counts elements fetched form the underlying iterator. Limit causes iterator to terminate early */ 22 | class CountingIterator[T](iterator: Iterator[T], limit: Option[Long] = None) extends Iterator[T] { 23 | private var _count = 0 24 | 25 | /** Returns the number of successful invocations of `next` */ 26 | def count = _count 27 | 28 | def hasNext = limit match { 29 | case Some(l) => _count < l && iterator.hasNext 30 | case _ => iterator.hasNext 31 | } 32 | 33 | def next() = { 34 | val item = iterator.next() 35 | _count += 1 36 | item 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/util/DataFrameOption.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.util 20 | 21 | trait DataFrameOption { 22 | val sqlOptionName: String 23 | 24 | def sqlOption(value: Any): Map[String, String] = { 25 | require(value != null) 26 | Map(sqlOptionName -> value.toString) 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/util/SpanningIterator.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.util 20 | 21 | import scala.collection.mutable.ArrayBuffer 22 | 23 | /** An iterator that groups items having the same value of the given function (key). 24 | * To be included in the same group, items with the same key must be next to each other 25 | * in the original collection. 26 | * 27 | * `SpanningIterator` buffers internally one group at a time and the wrapped iterator 28 | * is consumed in a lazy way. 29 | * 30 | * Example: 31 | * {{{ 32 | * val collection = Seq(1 -> "a", 1 -> "b", 1 -> "c", 2 -> "d", 2 -> "e") 33 | * val iterator = new SpanningIterator(collection.iterator, (x: (Int, String)) => x._1) 34 | * val result = iterator.toSeq // Seq(1 -> Seq("a", "b", "c"), 2 -> Seq("d", "e")) 35 | * }}} 36 | */ 37 | class SpanningIterator[K, T](iterator: Iterator[T], f: T => K) extends Iterator[(K, Seq[T])] { 38 | 39 | private[this] val items = new BufferedIterator2(iterator) 40 | 41 | override def hasNext = items.hasNext 42 | 43 | override def next(): (K, Seq[T]) = { 44 | val key = f(items.head) 45 | val buffer = new ArrayBuffer[T] 46 | items.appendWhile(r => f(r) == key, buffer) 47 | (key, buffer.toSeq) 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/util/Threads.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.util 20 | 21 | import java.util.concurrent.{Executors, LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit} 22 | 23 | import com.google.common.util.concurrent.ThreadFactoryBuilder 24 | 25 | import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService} 26 | 27 | object Threads extends Logging { 28 | 29 | implicit val BlockingIOExecutionContext: ExecutionContextExecutorService = { 30 | val threadFactory = new ThreadFactoryBuilder() 31 | .setDaemon(true) 32 | .setNameFormat("spark-cassandra-connector-io" + "%d") 33 | .setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler { 34 | override def uncaughtException(t: Thread, e: Throwable): Unit = { 35 | logWarning(s"Unhandled exception in thread ${t.getName}.", e) 36 | } 37 | }) 38 | .build 39 | ExecutionContext.fromExecutorService(Executors.newCachedThreadPool(threadFactory)) 40 | } 41 | } 42 | 43 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/util/package.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | import com.datastax.dse.driver.api.core.auth.ProxyAuthentication 22 | import com.datastax.oss.driver.api.core.cql.Statement 23 | import com.datastax.spark.connector.cql.{CassandraConnector, Schema, TableDef} 24 | 25 | /** Useful stuff that didn't fit elsewhere. */ 26 | package object util { 27 | 28 | def maybeExecutingAs[StatementT <: Statement[StatementT]](stmt: StatementT, proxyUser: Option[String]): StatementT = { 29 | proxyUser match { 30 | case Some(user) => 31 | ProxyAuthentication.executeAs(user, stmt) 32 | case _ => 33 | stmt 34 | } 35 | } 36 | 37 | def schemaFromCassandra( 38 | connector: CassandraConnector, 39 | keyspaceName: Option[String] = None, 40 | tableName: Option[String] = None): Schema = { 41 | connector.withSessionDo(Schema.fromCassandra(_, keyspaceName, tableName)) 42 | } 43 | 44 | def tableFromCassandra( 45 | connector: CassandraConnector, 46 | keyspaceName: String, 47 | tableName: String): TableDef = { 48 | connector.withSessionDo(Schema.tableFromCassandra(_, keyspaceName, tableName)) 49 | } 50 | 51 | } 52 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/writer/BatchGroupingKey.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.writer 20 | 21 | sealed trait BatchGroupingKey 22 | 23 | object BatchGroupingKey { 24 | 25 | /** Any row can be added to any batch. This works the same as previous batching implementation. */ 26 | case object None extends BatchGroupingKey 27 | 28 | /** Each batch is associated with a set of replicas. If a set of replicas for the inserted row is 29 | * the same as it is for a batch, the row can be added to the batch. */ 30 | case object ReplicaSet extends BatchGroupingKey 31 | 32 | /** Each batch is associated with a partition key. If the partition key of the inserted row is the 33 | * same as it is for a batch, the row can be added to the batch. */ 34 | case object Partition extends BatchGroupingKey 35 | 36 | def apply(name: String): BatchGroupingKey = name.toLowerCase match { 37 | case "none" => None 38 | case "replica_set" => ReplicaSet 39 | case "partition" => Partition 40 | case _ => throw new IllegalArgumentException(s"Invalid batch level: $name") 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/writer/BatchStatementBuilder.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.writer 20 | 21 | import com.datastax.oss.driver.api.core.ConsistencyLevel 22 | import com.datastax.oss.driver.api.core.cql.BatchType 23 | import com.datastax.spark.connector.util.Logging 24 | 25 | private[connector] class BatchStatementBuilder( 26 | val batchType: BatchType, 27 | val consistencyLevel: ConsistencyLevel) extends Logging { 28 | 29 | /** Converts a sequence of statements into a batch if its size is greater than 1. 30 | * Sets the routing key and consistency level. */ 31 | def maybeCreateBatch(stmts: Seq[RichBoundStatementWrapper]): RichStatement = { 32 | require(stmts.nonEmpty, "Statements list cannot be empty") 33 | val stmt = stmts.head 34 | 35 | if (stmts.size == 1) { 36 | stmt.setConsistencyLevel(consistencyLevel) 37 | } else { 38 | new RichBatchStatementWrapper(batchType, consistencyLevel, stmts) 39 | } 40 | } 41 | 42 | } 43 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/writer/CassandraRowWriter.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.writer 20 | 21 | import com.datastax.spark.connector.{ColumnRef, CassandraRow} 22 | import com.datastax.spark.connector.cql.TableDef 23 | 24 | /** A [[RowWriter]] that can write [[CassandraRow]] objects.*/ 25 | class CassandraRowWriter(table: TableDef, selectedColumns: IndexedSeq[ColumnRef]) extends RowWriter[CassandraRow] { 26 | 27 | override val columnNames = selectedColumns.map(_.columnName) 28 | 29 | private val columns = columnNames.map(table.columnByName).toIndexedSeq 30 | private val converters = columns.map(_.columnType.converterToCassandra) 31 | 32 | override def readColumnValues(data: CassandraRow, buffer: Array[Any]) = { 33 | for ((c, i) <- columnNames.zipWithIndex) { 34 | val value = data.getRaw(c) 35 | val convertedValue = converters(i).convert(value) 36 | buffer(i) = convertedValue 37 | } 38 | } 39 | } 40 | 41 | 42 | object CassandraRowWriter { 43 | 44 | object Factory extends RowWriterFactory[CassandraRow] { 45 | override def rowWriter(table: TableDef, selectedColumns: IndexedSeq[ColumnRef]) = 46 | new CassandraRowWriter(table, selectedColumns) 47 | } 48 | 49 | } 50 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/writer/DefaultRowWriter.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.writer 20 | 21 | import scala.reflect.runtime.universe._ 22 | import scala.collection.Seq 23 | import com.datastax.spark.connector.ColumnRef 24 | import com.datastax.spark.connector.cql.TableDef 25 | import com.datastax.spark.connector.mapper.{ColumnMapper, MappedToGettableDataConverter} 26 | 27 | /** A `RowWriter` suitable for saving objects mappable by a [[com.datastax.spark.connector.mapper.ColumnMapper ColumnMapper]]. 28 | * Can save case class objects, java beans and tuples. */ 29 | class DefaultRowWriter[T : TypeTag : ColumnMapper]( 30 | table: TableDef, 31 | selectedColumns: IndexedSeq[ColumnRef]) 32 | extends RowWriter[T] { 33 | 34 | private val converter = MappedToGettableDataConverter[T](table, selectedColumns) 35 | override val columnNames = selectedColumns.map(_.columnName) 36 | 37 | override def readColumnValues(data: T, buffer: Array[Any]) = { 38 | val row = converter.convert(data) 39 | for (i <- columnNames.indices) 40 | buffer(i) = row.getRaw(i) 41 | } 42 | } 43 | 44 | object DefaultRowWriter { 45 | 46 | def factory[T : ColumnMapper : TypeTag] = new RowWriterFactory[T] { 47 | override def rowWriter(tableDef: TableDef, selectedColumns: IndexedSeq[ColumnRef]) = { 48 | new DefaultRowWriter[T](tableDef, selectedColumns) 49 | } 50 | } 51 | } 52 | 53 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/writer/NullKeyColumnException.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.writer 20 | 21 | case class NullKeyColumnException(columnName: String) 22 | extends NullPointerException(s"Invalid null value for key column $columnName") 23 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/writer/QueryExecutor.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.writer 20 | 21 | import com.datastax.oss.driver.api.core.CqlSession 22 | import com.datastax.oss.driver.api.core.cql.AsyncResultSet 23 | import com.datastax.spark.connector.writer.AsyncExecutor.Handler 24 | 25 | class QueryExecutor( 26 | session: CqlSession, 27 | maxConcurrentQueries: Int, 28 | successHandler: Option[Handler[RichStatement]], 29 | failureHandler: Option[Handler[RichStatement]]) 30 | 31 | extends AsyncExecutor[RichStatement, AsyncResultSet]( 32 | stmt => session.executeAsync(stmt.stmt), 33 | maxConcurrentQueries, 34 | successHandler, 35 | failureHandler) 36 | 37 | object QueryExecutor { 38 | 39 | /** 40 | * Builds a query executor whose max requests per connection is limited to the MaxRequests per Connection 41 | */ 42 | def apply( 43 | session: CqlSession, 44 | maxConcurrentQueries: Int, 45 | successHandler: Option[Handler[RichStatement]], 46 | failureHandler: Option[Handler[RichStatement]]): QueryExecutor = { 47 | 48 | new QueryExecutor(session, maxConcurrentQueries, successHandler, failureHandler) 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/writer/RowWriter.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.writer 20 | 21 | 22 | /** `RowWriter` knows how to extract column names and values from custom row objects 23 | * and how to convert them to values that can be written to Cassandra. 24 | * `RowWriter` is required to apply any user-defined data type conversion. */ 25 | trait RowWriter[T] extends Serializable { 26 | 27 | /** List of columns this `RowWriter` is going to write. 28 | * Used to construct appropriate INSERT or UPDATE statement. */ 29 | def columnNames: Seq[String] 30 | 31 | /** Extracts column values from `data` object and writes them into the given buffer 32 | * in the same order as they are listed in the columnNames sequence. */ 33 | def readColumnValues(data: T, buffer: Array[Any]) 34 | 35 | } 36 | -------------------------------------------------------------------------------- /connector/src/main/scala/com/datastax/spark/connector/writer/package.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | /** Contains components for writing RDDs to Cassandra */ 22 | package object writer { 23 | 24 | } 25 | -------------------------------------------------------------------------------- /connector/src/main/scala/org/apache/spark/sql/cassandra/CassandraPredicateRules.scala: -------------------------------------------------------------------------------- 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.spark.sql.cassandra 20 | 21 | import com.datastax.spark.connector.cql.TableDef 22 | import org.apache.spark.sql.sources.Filter 23 | import org.apache.spark.SparkConf 24 | 25 | case class AnalyzedPredicates( 26 | handledByCassandra: Set[Filter], 27 | handledBySpark: Set[Filter] ){ 28 | override def toString(): String = { 29 | s"""C* Filters: [${handledByCassandra.mkString(", ")}] 30 | |Spark Filters [${handledBySpark.mkString(", ")}]""".stripMargin 31 | } 32 | } 33 | 34 | trait CassandraPredicateRules{ 35 | def apply(predicates: AnalyzedPredicates, tableDef: TableDef, conf: SparkConf): AnalyzedPredicates 36 | } 37 | -------------------------------------------------------------------------------- /connector/src/main/scala/org/apache/spark/sql/cassandra/package-info.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.spark.sql.cassandra; -------------------------------------------------------------------------------- /connector/src/test/java/com/datastax/spark/connector/SampleJavaBean.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 com.datastax.spark.connector; 20 | 21 | import java.io.Serializable; 22 | 23 | /** 24 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because 25 | * Scala adds some additional accessors and mutators. 26 | */ 27 | public class SampleJavaBean implements Serializable { 28 | private Integer key; 29 | private String value; 30 | 31 | public static SampleJavaBean newInstance(Integer key, String value) { 32 | SampleJavaBean bean = new SampleJavaBean(); 33 | bean.setKey(key); 34 | bean.setValue(value); 35 | return bean; 36 | } 37 | 38 | public Integer getKey() { 39 | return key; 40 | } 41 | 42 | public void setKey(Integer key) { 43 | this.key = key; 44 | } 45 | 46 | public String getValue() { 47 | return value; 48 | } 49 | 50 | public void setValue(String value) { 51 | this.value = value; 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /connector/src/test/java/com/datastax/spark/connector/SampleJavaBeanSubClass.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 com.datastax.spark.connector; 20 | 21 | /** 22 | * This is a sample JavaBean style class/subclass. In order to test JavaAPI correctly, we cannot 23 | * implement this in Scala because Scala adds some additional accessors and mutators. 24 | */ 25 | public class SampleJavaBeanSubClass extends SampleJavaBean 26 | { 27 | private String subClassField; 28 | 29 | public static SampleJavaBeanSubClass newInstance(Integer key, String value, String subClassField) { 30 | SampleJavaBeanSubClass bean = new SampleJavaBeanSubClass(); 31 | bean.setKey(key); 32 | bean.setValue(value); 33 | bean.setSubClassField(subClassField); 34 | return bean; 35 | } 36 | 37 | public String getSubClassField() 38 | { 39 | return subClassField; 40 | } 41 | 42 | public void setSubClassField(String subClassField) 43 | { 44 | this.subClassField = subClassField; 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /connector/src/test/java/com/datastax/spark/connector/SampleJavaBeanWithMultipleCtors.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 com.datastax.spark.connector; 20 | 21 | import java.io.Serializable; 22 | 23 | /** 24 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because 25 | * Scala adds some additional accessors and mutators. 26 | */ 27 | public class SampleJavaBeanWithMultipleCtors implements Serializable { 28 | private Integer key; 29 | private String value; 30 | 31 | public SampleJavaBeanWithMultipleCtors(Integer key) { 32 | this.key = key; 33 | } 34 | 35 | public SampleJavaBeanWithMultipleCtors() { 36 | } 37 | 38 | public SampleJavaBeanWithMultipleCtors(Integer key, String value) { 39 | this.key = key; 40 | this.value = value; 41 | } 42 | 43 | public Integer getKey() { 44 | return key; 45 | } 46 | 47 | public void setKey(Integer key) { 48 | this.key = key; 49 | } 50 | 51 | public String getValue() { 52 | return value; 53 | } 54 | 55 | public void setValue(String value) { 56 | this.value = value; 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /connector/src/test/java/com/datastax/spark/connector/SampleJavaBeanWithoutNoArgsCtor.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 com.datastax.spark.connector; 20 | 21 | import java.io.Serializable; 22 | 23 | /** 24 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because 25 | * Scala adds some additional accessors and mutators. 26 | */ 27 | public class SampleJavaBeanWithoutNoArgsCtor implements Serializable { 28 | private Integer key; 29 | private String value; 30 | 31 | private SampleJavaBeanWithoutNoArgsCtor(Integer key, String value) { 32 | this.key = key; 33 | this.value = value; 34 | } 35 | 36 | public Integer getKey() { 37 | return key; 38 | } 39 | 40 | public void setKey(Integer key) { 41 | this.key = key; 42 | } 43 | 44 | public String getValue() { 45 | return value; 46 | } 47 | 48 | public void setValue(String value) { 49 | this.value = value; 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /connector/src/test/java/com/datastax/spark/connector/SampleWeirdJavaBean.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 com.datastax.spark.connector; 20 | 21 | import java.io.Serializable; 22 | 23 | /** 24 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because 25 | * Scala adds some additional accessors and mutators. 26 | */ 27 | public class SampleWeirdJavaBean implements Serializable { 28 | private Integer devil; 29 | private String cat; 30 | 31 | public static SampleWeirdJavaBean newInstance(Integer key, String value) { 32 | SampleWeirdJavaBean bean = new SampleWeirdJavaBean(); 33 | bean.setDevil(key); 34 | bean.setCat(value); 35 | return bean; 36 | } 37 | 38 | public Integer getDevil() { 39 | return devil; 40 | } 41 | 42 | public void setDevil(Integer devil) { 43 | this.devil = devil; 44 | } 45 | 46 | public String getCat() { 47 | return cat; 48 | } 49 | 50 | public void setCat(String cat) { 51 | this.cat = cat; 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /connector/src/test/java/com/datastax/spark/connector/SampleWithDeeplyNestedJavaBean.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 com.datastax.spark.connector; 20 | 21 | import java.io.Serializable; 22 | 23 | /** 24 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because 25 | * Scala adds some additional accessors and mutators. 26 | */ 27 | public class SampleWithDeeplyNestedJavaBean implements Serializable { 28 | public class IntermediateClass implements Serializable { 29 | public class InnerClass implements Serializable { 30 | private Integer key; 31 | private String value; 32 | 33 | public InnerClass(Integer key) { 34 | this.key = key; 35 | } 36 | 37 | public InnerClass() { 38 | } 39 | 40 | public InnerClass(Integer key, String value) { 41 | this.key = key; 42 | this.value = value; 43 | } 44 | 45 | public Integer getKey() { 46 | return key; 47 | } 48 | 49 | public void setKey(Integer key) { 50 | this.key = key; 51 | } 52 | 53 | public String getValue() { 54 | return value; 55 | } 56 | 57 | public void setValue(String value) { 58 | this.value = value; 59 | } 60 | } 61 | } 62 | } 63 | -------------------------------------------------------------------------------- /connector/src/test/java/com/datastax/spark/connector/SampleWithNestedJavaBean.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 com.datastax.spark.connector; 20 | 21 | import java.io.Serializable; 22 | 23 | /** 24 | * This is a sample JavaBean style class. In order to test JavaAPI correctly, we cannot implement this in Scala because 25 | * Scala adds some additional accessors and mutators. 26 | */ 27 | public class SampleWithNestedJavaBean implements Serializable { 28 | public class InnerClass implements Serializable { 29 | private Integer key; 30 | private String value; 31 | 32 | public InnerClass(Integer key) { 33 | this.key = key; 34 | } 35 | 36 | public InnerClass() { 37 | } 38 | 39 | public InnerClass(Integer key, String value) { 40 | this.key = key; 41 | this.value = value; 42 | } 43 | 44 | public Integer getKey() { 45 | return key; 46 | } 47 | 48 | public void setKey(Integer key) { 49 | this.key = key; 50 | } 51 | 52 | public String getValue() { 53 | return value; 54 | } 55 | 56 | public void setValue(String value) { 57 | this.value = value; 58 | } 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /connector/src/test/java/com/datastax/spark/connector/japi/SparkContextJavaFunctionsTest.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 com.datastax.spark.connector.japi; 20 | 21 | import com.datastax.oss.driver.api.core.DefaultConsistencyLevel; 22 | import com.datastax.spark.connector.rdd.ReadConf; 23 | import org.apache.spark.SparkConf; 24 | import org.apache.spark.SparkContext; 25 | import org.junit.Test; 26 | import org.mockito.Mockito; 27 | 28 | import static org.junit.Assert.assertEquals; 29 | import static org.mockito.Mockito.when; 30 | 31 | public class SparkContextJavaFunctionsTest 32 | { 33 | @Test 34 | public void testReadConfPopulating() { 35 | SparkConf conf = new SparkConf(); 36 | conf.set("spark.cassandra.input.fetch.sizeInRows", "1234"); 37 | conf.set("spark.cassandra.input.split.sizeInMB", "4321"); 38 | conf.set("spark.cassandra.input.consistency.level", "THREE"); 39 | 40 | SparkContext sc = Mockito.mock(SparkContext.class); 41 | when(sc.getConf()).thenReturn(conf); 42 | 43 | ReadConf readConf = CassandraJavaUtil.javaFunctions(sc).cassandraTable("a", "b").rdd().readConf(); 44 | 45 | assertEquals(readConf.fetchSizeInRows(), 1234); 46 | assertEquals(readConf.splitSizeInMB(), 4321); 47 | assertEquals(readConf.consistencyLevel(), DefaultConsistencyLevel.THREE); 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /connector/src/test/java/com/datastax/spark/connector/japi/rdd/CassandraJoinJavaRDDTest.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 com.datastax.spark.connector.japi.rdd; 20 | 21 | import static com.datastax.spark.connector.japi.CassandraJavaUtil.someColumns; 22 | import static org.hamcrest.CoreMatchers.is; 23 | import static org.hamcrest.MatcherAssert.assertThat; 24 | import org.junit.Test; 25 | import static org.mockito.Mockito.mock; 26 | import static org.mockito.Mockito.when; 27 | 28 | import com.datastax.spark.connector.rdd.CassandraJoinRDD; 29 | 30 | @SuppressWarnings({"unchecked", "RedundantTypeArguments"}) 31 | public class CassandraJoinJavaRDDTest { 32 | 33 | @Test 34 | public void testOn() { 35 | CassandraJoinRDD rdd = mock(CassandraJoinRDD.class); 36 | CassandraJoinRDD rdd2 = mock(CassandraJoinRDD.class); 37 | when(rdd.on(someColumns("a", "b"))).thenReturn(rdd2); 38 | CassandraJoinJavaRDD jrdd = new CassandraJoinJavaRDD<>(rdd, String.class, Integer.class); 39 | assertThat(jrdd.on(someColumns("a", "b")).rdd(), is(rdd2)); 40 | } 41 | 42 | } 43 | -------------------------------------------------------------------------------- /connector/src/test/java/com/datastax/spark/connector/mapper/JavaTestBean.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 com.datastax.spark.connector.mapper; 20 | 21 | import com.datastax.oss.driver.api.mapper.annotations.CqlName; 22 | import com.datastax.oss.driver.api.mapper.annotations.Entity; 23 | 24 | import java.io.Serializable; 25 | 26 | /** 27 | * This is a Java Bean style class with Java Driver style annotations built in. 28 | * This class also contains nested UDTs with its own mappings 29 | */ 30 | @Entity 31 | public class JavaTestBean implements Serializable { 32 | 33 | @CqlName(value = "cassandra_property_1") 34 | public Integer property1; 35 | @CqlName(value = "cassandra_camel_case_property") 36 | public Integer camelCaseProperty; 37 | public JavaTestUDTBean nested; 38 | 39 | public int getProperty1() { 40 | return property1; 41 | } 42 | 43 | public void setProperty1(int property1) { 44 | this.property1 = property1; 45 | } 46 | 47 | public int getCamelCaseProperty() { 48 | return camelCaseProperty; 49 | } 50 | 51 | public void setCamelCaseProperty(int camelCaseProperty) { 52 | this.camelCaseProperty = camelCaseProperty; 53 | } 54 | 55 | public JavaTestUDTBean getNested() { 56 | return nested; 57 | } 58 | 59 | public void setNested(JavaTestUDTBean nested) { 60 | this.nested = nested; 61 | } 62 | 63 | } 64 | -------------------------------------------------------------------------------- /connector/src/test/java/com/datastax/spark/connector/mapper/JavaTestUDTBean.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 com.datastax.spark.connector.mapper; 20 | 21 | import com.datastax.oss.driver.api.mapper.annotations.CqlName; 22 | 23 | import java.io.Serializable; 24 | 25 | /** 26 | * This Java POJO represents an UDT in Cassandra 27 | * 28 | * Created by soumya on 9/15/16. 29 | */ 30 | public class JavaTestUDTBean implements Serializable { 31 | public Integer field; 32 | @CqlName(value = "cassandra_another_field") 33 | public Integer anotherField; 34 | @CqlName(value = "cassandra_yet_another_field") 35 | public Integer completelyUnrelatedField; 36 | 37 | public Integer getField() { 38 | return field; 39 | } 40 | 41 | public void setField(Integer field) { 42 | this.field = field; 43 | } 44 | 45 | public Integer getAnotherField() { 46 | return anotherField; 47 | } 48 | 49 | public void setAnotherField(Integer anotherField) { 50 | this.anotherField = anotherField; 51 | } 52 | 53 | public Integer getCompletelyUnrelatedField() { 54 | return completelyUnrelatedField; 55 | } 56 | 57 | public void setCompletelyUnrelatedField(Integer completelyUnrelatedField) { 58 | this.completelyUnrelatedField = completelyUnrelatedField; 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /connector/src/test/resources/log4j.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, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | 17 | # for production, you should probably set pattern to %c instead of %l. 18 | # (%l is slower.) 19 | 20 | # output messages into a rolling log file as well as stdout 21 | log4j.rootLogger=WARN,stdout 22 | 23 | # stdout 24 | log4j.appender.stdout=org.apache.log4j.ConsoleAppender 25 | log4j.appender.stdout.layout=org.apache.log4j.PatternLayout 26 | log4j.appender.stdout.layout.ConversionPattern=%5p %d{HH:mm:ss,SSS} %C (%F:%L) - %m%n 27 | 28 | # Avoid "no host ID found" when starting a fresh node 29 | log4j.logger.org.apache.cassandra.db.SystemKeyspace=ERROR 30 | 31 | # Avoid "address already in use" when starting multiple local Spark masters 32 | log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR 33 | log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR 34 | 35 | # Suppress some warnings 36 | log4j.logger.com.datastax.driver.core.NettyUtil=ERROR 37 | log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR 38 | log4j.logger.org.apache.cassandra.utils.CLibrary=ERROR 39 | log4j.logger.org.apache.cassandra.service.StartupChecks=ERROR 40 | log4j.logger.org.spark-project.jetty.server.Server=ERROR 41 | log4j.logger.org.eclipse.jetty.server.Server=ERROR 42 | -------------------------------------------------------------------------------- /connector/src/test/resources/logback.xml: -------------------------------------------------------------------------------- 1 | 2 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | %d{yyyy-MM-dd HH:mm:ss} [%thread] %-5level %logger{36} - %msg%n 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | -------------------------------------------------------------------------------- /connector/src/test/scala/com/datastax/bdp/spark/DseAuthConfFactorySpec.scala: -------------------------------------------------------------------------------- 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 com.datastax.bdp.spark 20 | /** 21 | TODO: 22 | import org.apache.commons.lang3.SerializationUtils 23 | import org.apache.spark.SparkConf 24 | import org.scalatest.{FlatSpec, Matchers} 25 | 26 | import com.datastax.bdp.config.{ClientConfigurationFactory, YamlClientConfiguration} 27 | import com.datastax.bdp.spark.DseAuthConfFactory.DsePasswordAuthConf 28 | import com.datastax.bdp.test.ng.{DataGenerator, DseScalaTestBase, ToString, YamlProvider} 29 | 30 | class DseAuthConfFactorySpec extends FlatSpec with Matchers with DseScalaTestBase { 31 | 32 | beforeClass { 33 | YamlProvider.provideDefaultYamls() 34 | YamlClientConfiguration.setAsClientConfigurationImpl() 35 | } 36 | 37 | it should "produce equivalent AuthConf instances for the same SparkConf" in { 38 | def genAuthConf = DseAuthConfFactory.authConf(new SparkConf()) 39 | 40 | genAuthConf shouldBe genAuthConf 41 | } 42 | 43 | it should "produce comparable DsePasswordAuthConf instances" in { 44 | val gen = new DataGenerator() 45 | val cases = gen.generate[DsePasswordAuthConf]() 46 | for (c <- cases) { 47 | withClue(s"Comparing ${ToString.toStringWithNames(c)} failed") { 48 | val duplicate = SerializationUtils.roundtrip(c) 49 | duplicate shouldBe c 50 | } 51 | } 52 | } 53 | } 54 | **/ 55 | -------------------------------------------------------------------------------- /connector/src/test/scala/com/datastax/spark/connector/cql/DefaultConnectionFactoryTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.cql 20 | 21 | import java.io.IOException 22 | 23 | import org.apache.spark.SparkEnv 24 | import org.mockito.Mockito 25 | import org.mockito.invocation.InvocationOnMock 26 | import org.mockito.stubbing.Answer 27 | import org.scalatest.FlatSpec 28 | import org.scalatestplus.mockito.MockitoSugar 29 | 30 | class DefaultConnectionFactoryTest extends FlatSpec with MockitoSugar { 31 | 32 | /** DefaultConnectionFactory relies on a non-null SparkEnv */ 33 | private def mockedSparkEnv[T](code: => T): T = { 34 | val original = SparkEnv.get 35 | val sparkEnv = Mockito.mock(classOf[SparkEnv], new Answer[Option[String]] { 36 | override def answer(invocation: InvocationOnMock): Option[String] = None 37 | }) 38 | SparkEnv.set(sparkEnv) 39 | try { 40 | code 41 | } finally { 42 | SparkEnv.set(original) 43 | } 44 | } 45 | 46 | it should "complain when a malformed URL is provided" in mockedSparkEnv { 47 | intercept[IOException] { 48 | DefaultConnectionFactory.maybeGetLocalFile("secure-bundle.zip") 49 | } 50 | } 51 | 52 | it should "complain when an URL with unrecognized scheme is provided" in mockedSparkEnv { 53 | intercept[IOException] { 54 | DefaultConnectionFactory.maybeGetLocalFile("hdfs:///secure-bundle.zip") 55 | } 56 | } 57 | 58 | } 59 | -------------------------------------------------------------------------------- /connector/src/test/scala/com/datastax/spark/connector/embedded/SparkTemplate.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.embedded 20 | 21 | import java.nio.file.Files 22 | 23 | import org.apache.log4j.{Level, Logger} 24 | import org.apache.spark.SparkConf 25 | 26 | object SparkTemplate { 27 | 28 | val DefaultParallelism = 2 29 | 30 | /** Default configuration for [[org.apache.spark.SparkContext SparkContext]]. */ 31 | private val _defaultConf = new SparkConf(true) 32 | .set("spark.cassandra.connection.keepAliveMS", "5000") 33 | .set("spark.cassandra.connection.timeoutMS", "30000") 34 | .set("spark.ui.showConsoleProgress", "false") 35 | .set("spark.ui.enabled", "false") 36 | .set("spark.cleaner.ttl", "3600") 37 | .set("spark.sql.extensions","com.datastax.spark.connector.CassandraSparkExtensions") 38 | .setMaster(sys.env.getOrElse("IT_TEST_SPARK_MASTER", s"local[$DefaultParallelism]")) 39 | .setAppName("Test") 40 | 41 | 42 | def defaultConf = _defaultConf.clone() 43 | 44 | def withoutLogging[T]( f: => T): T={ 45 | val level = Logger.getRootLogger.getLevel 46 | Logger.getRootLogger.setLevel(Level.OFF) 47 | val ret = f 48 | Logger.getRootLogger.setLevel(level) 49 | ret 50 | } 51 | 52 | } 53 | -------------------------------------------------------------------------------- /connector/src/test/scala/com/datastax/spark/connector/embedded/package.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | import scala.concurrent.duration.FiniteDuration 22 | 23 | package object embedded { 24 | 25 | /* Factor by which to scale timeouts during tests, e.g. to account for shared build system load. */ 26 | implicit class SparkTestDuration(val duration: FiniteDuration) extends AnyVal { 27 | def dilated: FiniteDuration = (duration * 1.0).asInstanceOf[FiniteDuration] 28 | } 29 | } 30 | -------------------------------------------------------------------------------- /connector/src/test/scala/com/datastax/spark/connector/rdd/CqlWhereClauseSpec.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd 20 | 21 | import org.scalatest.{FlatSpec, Matchers} 22 | 23 | class CqlWhereClauseSpec extends FlatSpec with Matchers { 24 | 25 | it should "produce a string for each predicate" in { 26 | val clause = CqlWhereClause(Seq("x < ?", "y = ?"), Seq(1, "aaa")) 27 | 28 | clause.toString shouldBe "[[x < ?, 1],[y = ?, aaa]]" 29 | } 30 | 31 | it should "produce empty predicate string for empty predicate list" in { 32 | val clause = CqlWhereClause(Seq(), Seq()) 33 | 34 | clause.toString shouldBe "[]" 35 | } 36 | 37 | it should "produce valid string for IN clause predicate" in { 38 | val clause = CqlWhereClause(Seq("x < ?", "z IN (?, ?)", "y IN (?, ?, ?)", "a = ?"), Seq(1, 2, 3, 4, 5, 6, 7)) 39 | 40 | clause.toString shouldBe "[[x < ?, 1],[z IN (?, ?), (2, 3)],[y IN (?, ?, ?), (4, 5, 6)],[a = ?, 7]]" 41 | } 42 | 43 | it should "complain when the number of values doesn't match the number of placeholders '?'" in { 44 | intercept[AssertionError] { 45 | CqlWhereClause(Seq("x < ?"), Seq()) 46 | } 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /connector/src/test/scala/com/datastax/spark/connector/rdd/reader/ClassBasedRowReaderTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.rdd.reader 20 | 21 | import com.datastax.spark.connector.cql.{RegularColumn, ColumnDef, PartitionKeyColumn, TableDef} 22 | import org.apache.commons.lang3.SerializationUtils 23 | import org.junit.Test 24 | 25 | import com.datastax.spark.connector.types.{BigIntType, IntType, VarCharType} 26 | 27 | case class TestClass(a: String, b: Int, c: Option[Long]) 28 | 29 | class ClassBasedRowReaderTest { 30 | 31 | private val a = ColumnDef("a", PartitionKeyColumn, VarCharType) 32 | private val b = ColumnDef("b", RegularColumn, IntType) 33 | private val c = ColumnDef("c", RegularColumn, BigIntType) 34 | private val table = TableDef("test", "table", Seq(a), Nil, Seq(b, c)) 35 | 36 | @Test 37 | def testSerialize() { 38 | val reader = new ClassBasedRowReader[TestClass](table, table.columnRefs) 39 | SerializationUtils.roundtrip(reader) 40 | } 41 | 42 | } 43 | -------------------------------------------------------------------------------- /connector/src/test/scala/com/datastax/spark/connector/samples.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector { 20 | 21 | case class SampleScalaCaseClass(key: Int, value: String) 22 | 23 | class SampleScalaClass(val key: Int, val value: String) extends Serializable 24 | 25 | class SampleScalaClassWithNoFields(key: Int, value: String) extends Serializable 26 | 27 | class SampleScalaClassWithMultipleCtors(var key: Int, var value: String) extends Serializable { 28 | def this(key: Int) = this(key, null) 29 | 30 | def this() = this(0, null) 31 | } 32 | 33 | class SampleWithNestedScalaCaseClass extends Serializable { 34 | 35 | case class InnerClass(key: Int, value: String) 36 | 37 | } 38 | 39 | class SampleWithDeeplyNestedScalaCaseClass extends Serializable { 40 | 41 | class IntermediateClass extends Serializable { 42 | 43 | case class InnerClass(key: Int, value: String) 44 | 45 | } 46 | 47 | } 48 | 49 | object SampleObject { 50 | 51 | case class ClassInObject(key: Int, value: String) 52 | 53 | } 54 | 55 | } -------------------------------------------------------------------------------- /connector/src/test/scala/com/datastax/spark/connector/testkit/package.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | import org.scalatest.{Matchers, WordSpecLike} 22 | 23 | package object testkit { 24 | 25 | /** Basic unit test abstraction. */ 26 | trait AbstractSpec extends WordSpecLike with Matchers 27 | 28 | val dataSeq = Seq ( 29 | Seq("1first", "1round", "1words"), 30 | Seq("2second", "2round", "2words"), 31 | Seq("3third", "3round", "3words"), 32 | Seq("4fourth", "4round", "4words") 33 | ) 34 | 35 | val data = dataSeq.head 36 | 37 | } 38 | -------------------------------------------------------------------------------- /doc/10_embedded.md: -------------------------------------------------------------------------------- 1 | # Documentation 2 | 3 | # Spark-Connector-Embedded No Longer Supported 4 | 5 | _Note that this artifact was removed 2.5 and newer. Spark Cassandra Connector integration 6 | tests rely on [ccm](https://github.com/riptano/ccm)._ 7 | 8 | Please see the `test-support` package for more info about our new CCM based integration testing framework 9 | 10 | ## Examples 11 | 12 | [Next - Performance Monitoring](11_metrics.md) 13 | -------------------------------------------------------------------------------- /doc/12_building_and_artifacts.md: -------------------------------------------------------------------------------- 1 | # Documentation 2 | 3 | ## Building 4 | 5 | ### Scala Versions 6 | You can choose to build, assemble and run both Spark and the Spark Cassandra Connector against Scala 2.11, 2.12 or 2.13. 7 | 8 | As of Spark Cassandra Connector 2.5.0 the build defaults to Scala 2.12. To build 2.11 artifacts, use `++2.11.12` switch 9 | in your sbt commands, e.g. `sbt/sbt ++2.11.12 package`. 10 | 11 | ### Building The Main Artifacts 12 | 13 | All artifacts are generated to the standard output directories based on the Scala binary version you use. 14 | 15 | In the root directory run: 16 | 17 | sbt/sbt package 18 | 19 | The library package jars will be generated to: 20 | 21 | spark-cassandra-connector/connector/target/scala-{binary.version}/ 22 | spark-cassandra-connector/driver/target/scala-{binary.version}/ 23 | 24 | The command generates also The Assembly Jar discussed below. 25 | 26 | ### Building The Assembly Jar 27 | 28 | The Assembly Jar is built by the `sbt/sbt package` command mentioned above and by the dedicated 29 | command `sbt/sbt assembly`. 30 | 31 | In the root directory run: 32 | 33 | sbt/sbt assembly 34 | 35 | A fat jar with `assembly` suffix will be generated to: 36 | 37 | spark-cassandra-connector/connector/target/scala-{binary.version}/ 38 | 39 | The jar contains the Spark Cassandra Connector and its dependencies. Some of the dependencies are shaded to avoid 40 | classpath conflicts. 41 | It is recommended to use the main artifact when possible. 42 | 43 | [Next - The Spark Shell](13_spark_shell.md) 44 | -------------------------------------------------------------------------------- /doc/13_1_setup_spark_shell.md: -------------------------------------------------------------------------------- 1 | # Documentation 2 | 3 | ## Setting up Cassandra 4 | 5 | To install and set up open-source Cassandra, refer to the instructions provided by the 6 | [Apache Cassandra project](https://cassandra.apache.org/doc/latest/). 7 | 8 | ## Setting up spark 9 | 10 | ### Download Spark 11 | 12 | Download a pre-built Spark from https://spark.apache.org/downloads.html 13 | Untar the tar.gz downloaded with 14 | 15 | tar -xvf spark-*-.tgz 16 | 17 | ### Start Spark in Stand Alone Mode (Optional) 18 | 19 | [Official Spark Instructions](https://spark.apache.org/docs/latest/spark-standalone.html) 20 | 21 | If you would like to run against a separate executor JVM then you need a running Spark Master and Worker. 22 | By default the spark-shell will run in local mode (driver/master/executor share a jvm.) 23 | 24 | Go to the newly created directory and start up Spark in stand-alone mode bound to localhost 25 | 26 | cd spark* 27 | ./sbin/start-all.sh 28 | 29 | At this point you should be able to access the Spark UI at localhost:8080. In the display you 30 | should see a single worker. At the top of this website you should see a URL set for the spark master. Save 31 | the master address (the entire spark://something:7077) if you would like to connect the shell to 32 | this stand alone spark master (use as sparkMasterAddress below). 33 | -------------------------------------------------------------------------------- /doc/9_demos.md: -------------------------------------------------------------------------------- 1 | # Demos 2 | 3 | Demos are not a part of Spark Cassandra Connector as of 2.0.0. 4 | 5 | Build examples are provided here 6 | [Spark Build Examples](https://github.com/DataStax-Examples/spark-build-examples) 7 | 8 | [Back to main page](../README.md) 9 | -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/TableRef.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | /** Store table name, keyspace name and option cluster name, keyspace is equivalent to database */ 22 | case class TableRef(table: String, keyspace: String, cluster: Option[String] = None) 23 | 24 | -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/TupleValue.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | import com.datastax.oss.driver.api.core.data.{TupleValue => DriverTupleValue} 22 | import com.datastax.spark.connector.types.NullableTypeConverter 23 | 24 | import scala.reflect.runtime.universe._ 25 | 26 | final case class TupleValue(values: Any*) extends ScalaGettableByIndexData with Product { 27 | override def columnValues = values.toIndexedSeq.map(_.asInstanceOf[AnyRef]) 28 | 29 | override def productArity: Int = columnValues.size 30 | 31 | override def productElement(n: Int): Any = getRaw(n) 32 | } 33 | 34 | object TupleValue { 35 | 36 | def fromJavaDriverTupleValue 37 | (value: DriverTupleValue) 38 | : TupleValue = { 39 | val values = 40 | for (i <- 0 until value.getType.getComponentTypes.size()) yield 41 | GettableData.get(value, i) 42 | new TupleValue(values: _*) 43 | } 44 | 45 | val TypeTag = typeTag[TupleValue] 46 | val Symbol = typeOf[TupleValue].asInstanceOf[TypeRef].sym 47 | 48 | implicit object TupleValueConverter extends NullableTypeConverter[TupleValue] { 49 | def targetTypeTag = TypeTag 50 | def convertPF = { 51 | case x: TupleValue => x 52 | } 53 | } 54 | } -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/japi/TupleValue.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.japi 20 | 21 | import com.datastax.spark.connector.types.{NullableTypeConverter, TypeConverter} 22 | import com.datastax.spark.connector.{TupleValue => ConnectorTupleValue} 23 | 24 | import scala.annotation.varargs 25 | import scala.reflect.runtime.universe._ 26 | 27 | final class TupleValue private (val columnValues: IndexedSeq[AnyRef]) 28 | extends JavaGettableByIndexData with Serializable 29 | 30 | 31 | object TupleValue { 32 | 33 | val TypeTag = typeTag[TupleValue] 34 | 35 | implicit object UDTValueConverter extends NullableTypeConverter[TupleValue] { 36 | def targetTypeTag = TypeTag 37 | 38 | def convertPF = { 39 | case x: TupleValue => x 40 | case x: ConnectorTupleValue => 41 | new TupleValue(x.columnValues) 42 | } 43 | } 44 | 45 | TypeConverter.registerConverter(UDTValueConverter) 46 | 47 | @varargs 48 | def newTuple(values: Object*): TupleValue = 49 | new TupleValue(values.toIndexedSeq) 50 | } 51 | 52 | -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/japi/UDTValue.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.japi 20 | 21 | import com.datastax.spark.connector.types.{NullableTypeConverter, TypeConverter} 22 | import com.datastax.spark.connector.{CassandraRowMetadata, UDTValue => ConnectorUDTValue} 23 | 24 | import scala.reflect.runtime.universe._ 25 | 26 | final class UDTValue(val metaData: CassandraRowMetadata, val columnValues: IndexedSeq[AnyRef]) 27 | extends JavaGettableData with Serializable 28 | 29 | object UDTValue { 30 | 31 | val UDTValueTypeTag = implicitly[TypeTag[UDTValue]] 32 | 33 | implicit object UDTValueConverter extends NullableTypeConverter[UDTValue] { 34 | def targetTypeTag = UDTValueTypeTag 35 | 36 | def convertPF = { 37 | case x: UDTValue => x 38 | case x: ConnectorUDTValue => 39 | new UDTValue(x.metaData, x.columnValues) 40 | } 41 | } 42 | 43 | TypeConverter.registerConverter(UDTValueConverter) 44 | 45 | } -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/mapper/ColumnMapperConvention.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.mapper 20 | 21 | import com.datastax.spark.connector.ColumnRef 22 | import org.apache.commons.lang3.StringUtils 23 | 24 | object ColumnMapperConvention { 25 | 26 | def camelCaseToUnderscore(str: String): String = 27 | StringUtils.splitByCharacterTypeCamelCase(str).mkString("_").replaceAll("_+", "_").toLowerCase 28 | 29 | def columnForProperty(propertyName: String, columnByName: Map[String, ColumnRef]): Option[ColumnRef] = { 30 | val underscoreName = camelCaseToUnderscore(propertyName) 31 | val candidateColumnNames = Seq(propertyName, underscoreName) 32 | candidateColumnNames.iterator 33 | .map(name => columnByName.get(name)) 34 | .find(_.isDefined) 35 | .flatten 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/mapper/PropertyExtractor.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.mapper 20 | 21 | import java.lang.reflect.Method 22 | 23 | import scala.util.Try 24 | 25 | /** Extracts values from fields of an object. */ 26 | class PropertyExtractor[T](val cls: Class[T], val propertyNames: Seq[String]) extends Serializable { 27 | 28 | private def getter(name: String) = 29 | cls.getMethod(name) 30 | 31 | @transient 32 | private lazy val methods: Array[Method] = 33 | propertyNames.map(getter).toArray 34 | 35 | @transient 36 | private lazy val methodByName = 37 | methods.map(m => (m.getName, m)).toMap 38 | 39 | def extract(obj: T): Array[AnyRef] = 40 | extract(obj, Array.ofDim(methods.length)) 41 | 42 | def extract(obj: T, target: Array[AnyRef]): Array[AnyRef] = { 43 | for (i <- methods.indices) 44 | target(i) = methods(i).invoke(obj) 45 | target 46 | } 47 | 48 | def extractProperty(obj: T, propertyName: String): AnyRef = { 49 | val m = methodByName(propertyName) 50 | m.invoke(obj) 51 | } 52 | } 53 | 54 | object PropertyExtractor { 55 | 56 | def availablePropertyNames(cls: Class[_], requestedPropertyNames: Seq[String]): Seq[String] = 57 | requestedPropertyNames.filter(name => Try(cls.getMethod(name)).isSuccess) 58 | 59 | } 60 | -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/mapper/package.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | /** Provides machinery for mapping Cassandra tables to user defined Scala classes or tuples. 22 | * The main class in this package is [[mapper.ColumnMapper]] responsible for matching Scala object's 23 | * properties with Cassandra column names.*/ 24 | package object mapper { 25 | 26 | } 27 | -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/types/TimestampFormatter.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.types 20 | 21 | import java.text.SimpleDateFormat 22 | import java.util.Date 23 | 24 | /** Formats timestamps and dates using CQL timestamp format `yyyy-MM-dd HH:mm:ssZ` */ 25 | object TimestampFormatter { 26 | 27 | private val TimestampPattern = "yyyy-MM-dd HH:mm:ssZ" 28 | 29 | def format(date: Date): String = 30 | new SimpleDateFormat(TimestampPattern).format(date) 31 | } 32 | -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/types/TypeAdapters.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.types 20 | 21 | /** 22 | * Type adapters that serve as a middle step in conversion from one type to another. 23 | * 24 | * Adapters are utilized by types with scheme ([[UserDefinedType]]], [[TupleType]]) to convert an instance of 25 | * an type to corresponding adapter and than to final value of the given type. 26 | */ 27 | private[spark] object TypeAdapters { 28 | /** 29 | * Adapter for multi-values types that my be returned as a sequence. 30 | * 31 | * It is used to extend conversion capabilities offered by Tuple type. 32 | */ 33 | trait ValuesSeqAdapter { 34 | def toSeq(): Seq[Any] 35 | } 36 | 37 | /** 38 | * Adapter for multi-value types that may return values by name. 39 | * 40 | * It is used to extend conversion capabilities offered by UDT type. 41 | */ 42 | trait ValueByNameAdapter { 43 | def getByName(name: String): Any 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/types/VectorType.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.types 20 | 21 | import scala.language.existentials 22 | import scala.reflect.runtime.universe._ 23 | 24 | case class VectorType[T](elemType: ColumnType[T], dimension: Int) extends ColumnType[Seq[T]] { 25 | 26 | override def isCollection: Boolean = false 27 | 28 | @transient 29 | lazy val scalaTypeTag = { 30 | implicit val elemTypeTag = elemType.scalaTypeTag 31 | implicitly[TypeTag[Seq[T]]] 32 | } 33 | 34 | def cqlTypeName = s"vector<${elemType.cqlTypeName}, ${dimension}>" 35 | 36 | override def converterToCassandra: TypeConverter[_ <: AnyRef] = 37 | new TypeConverter.OptionToNullConverter(TypeConverter.seqConverter(elemType.converterToCassandra)) 38 | } 39 | -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/types/package.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector 20 | 21 | import scala.util.Try 22 | 23 | /** Offers type conversion magic, so you can receive Cassandra column values in a form you like the most. 24 | * Simply specify the type you want to use on the Scala side, and the column value will be converted automatically. 25 | * Works also with complex objects like collections. */ 26 | package object types { 27 | 28 | /** Makes sure the sequence does not contain any lazy transformations. 29 | * This guarantees that if T is Serializable, the collection is Serializable. */ 30 | def unlazify[T](seq: IndexedSeq[T]): IndexedSeq[T] = IndexedSeq(seq: _*) 31 | 32 | } 33 | -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/util/ByteBufferUtil.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.util 20 | 21 | import java.nio.ByteBuffer 22 | 23 | object ByteBufferUtil { 24 | 25 | /** Copies the remaining bytes of the buffer into the given array, starting from offset zero. 26 | * The array must have capacity to store all of the remaining bytes of the buffer. 27 | * The buffer's position remains untouched. */ 28 | def copyBuffer(src: ByteBuffer, dest: Array[Byte]): Array[Byte] = { 29 | if (src.hasArray) { 30 | val length: Int = src.remaining 31 | val offset: Int = src.arrayOffset + src.position() 32 | System.arraycopy(src.array, offset, dest, 0, length) 33 | } else { 34 | src.duplicate.get(dest) 35 | } 36 | dest 37 | } 38 | 39 | /** Converts a byte buffer into an array. 40 | * The buffer's position remains untouched. */ 41 | def toArray(buffer: ByteBuffer): Array[Byte] = { 42 | if (buffer.hasArray && 43 | buffer.arrayOffset + buffer.position() == 0 && 44 | buffer.remaining == buffer.array.length) { 45 | buffer.array 46 | } else { 47 | val dest = new Array[Byte](buffer.remaining) 48 | copyBuffer(buffer, dest) 49 | } 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/util/DriverUtil.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.util 20 | 21 | import java.net.{InetAddress, InetSocketAddress} 22 | import java.util.Optional 23 | 24 | import com.datastax.oss.driver.api.core.CqlIdentifier 25 | import com.datastax.oss.driver.api.core.metadata.Node 26 | 27 | object DriverUtil { 28 | 29 | //TODO use CqlIdentifier instead? Use implicit conversion to String? To internal string? 30 | def toName(id: CqlIdentifier): String = id.asInternal() 31 | 32 | def toOption[T](optional: Optional[T]): Option[T] = 33 | if (optional.isPresent) Some(optional.get()) else None 34 | 35 | def toAddress(node: Node): Option[InetSocketAddress] = { 36 | node.getEndPoint.resolve() match { 37 | case address: InetSocketAddress => if (address.isUnresolved) { 38 | Option(new InetSocketAddress(address.getHostString, address.getPort)) 39 | } else { 40 | Option(address) 41 | } 42 | case _ => toOption(node.getBroadcastAddress) 43 | } 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/util/Quote.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.util 20 | 21 | object Quote { 22 | 23 | def quote(name: String): String = "\"" + name + "\"" 24 | 25 | } 26 | -------------------------------------------------------------------------------- /driver/src/main/scala/com/datastax/spark/connector/util/Reflect.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.util 20 | 21 | import scala.reflect.runtime.universe._ 22 | 23 | private[connector] object Reflect { 24 | 25 | def constructor(tpe: Type): Symbol = tpe.decl(termNames.CONSTRUCTOR) 26 | 27 | def member(tpe: Type, name: String): Symbol = tpe.member(TermName(name)) 28 | 29 | def methodSymbol(tpe: Type): MethodSymbol = { 30 | val constructors = constructor(tpe).asTerm.alternatives.map(_.asMethod) 31 | val paramCount = constructors.map(_.paramLists.flatten.size).max 32 | constructors.filter(_.paramLists.flatten.size == paramCount) match { 33 | case List(onlyOne) => onlyOne 34 | case _ => throw new IllegalArgumentException( 35 | "Multiple constructors with the same number of parameters not allowed.") 36 | } 37 | } 38 | } 39 | 40 | -------------------------------------------------------------------------------- /driver/src/test/java/com/datastax/spark/connector/mapper/ColumnMapperTestUDTBean.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 com.datastax.spark.connector.mapper; 20 | 21 | import com.datastax.oss.driver.api.mapper.annotations.CqlName; 22 | 23 | import java.io.Serializable; 24 | 25 | public class ColumnMapperTestUDTBean implements Serializable { 26 | public Integer field; 27 | @CqlName(value = "cassandra_another_field") 28 | public Integer anotherField; 29 | @CqlName(value = "cassandra_yet_another_field") 30 | public Integer completelyUnrelatedField; 31 | 32 | public Integer getField() { 33 | return field; 34 | } 35 | 36 | public void setField(Integer field) { 37 | this.field = field; 38 | } 39 | 40 | public Integer getAnotherField() { 41 | return anotherField; 42 | } 43 | 44 | public void setAnotherField(Integer anotherField) { 45 | this.anotherField = anotherField; 46 | } 47 | 48 | public Integer getCompletelyUnrelatedField() { 49 | return completelyUnrelatedField; 50 | } 51 | 52 | public void setCompletelyUnrelatedField(Integer completelyUnrelatedField) { 53 | this.completelyUnrelatedField = completelyUnrelatedField; 54 | } 55 | } 56 | 57 | -------------------------------------------------------------------------------- /driver/src/test/scala/com/datastax/spark/connector/mapper/PropertyExtractorTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.mapper 20 | 21 | import org.junit.Assert._ 22 | import org.junit.Test 23 | 24 | class PropertyExtractorTest { 25 | 26 | class TestClass(val field1: String, val field2: Int) 27 | 28 | @Test 29 | def testSimpleExtraction() { 30 | val testObject = new TestClass("a", 1) 31 | val propertyExtractor = new PropertyExtractor(classOf[TestClass], Seq("field1", "field2")) 32 | val result = propertyExtractor.extract(testObject) 33 | assertEquals(2, result.length) 34 | assertEquals("a", result(0)) 35 | assertEquals(1, result(1)) 36 | } 37 | 38 | @Test 39 | def testAvailableProperties() { 40 | val triedProperties = Seq("field1", "foo", "bar") 41 | val availableProperties = PropertyExtractor.availablePropertyNames(classOf[TestClass], triedProperties) 42 | assertEquals(Seq("field1"), availableProperties) 43 | } 44 | 45 | @Test(expected = classOf[NoSuchMethodException]) 46 | def testWrongPropertyName() { 47 | val testObject = new TestClass("a", 1) 48 | val propertyExtractor = new PropertyExtractor(classOf[TestClass], Seq("foo")) 49 | propertyExtractor.extract(testObject) 50 | } 51 | 52 | } 53 | -------------------------------------------------------------------------------- /driver/src/test/scala/com/datastax/spark/connector/types/CanBuildFromTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.types 20 | 21 | import org.apache.commons.lang3.SerializationUtils 22 | import org.junit.Assert._ 23 | import org.junit.Test 24 | 25 | class CanBuildFromTest { 26 | 27 | @Test 28 | def testBuild() { 29 | val bf = CanBuildFrom.setCanBuildFrom[Int] 30 | val builder = bf.apply() 31 | builder += 1 32 | builder += 2 33 | builder += 3 34 | assertEquals(Set(1,2,3), builder.result()) 35 | } 36 | 37 | @Test 38 | def testSerializeAndBuild() { 39 | val bf = CanBuildFrom.setCanBuildFrom[Int] 40 | val bf2 = SerializationUtils.roundtrip(bf) 41 | val builder = bf2.apply() 42 | builder += 1 43 | builder += 2 44 | builder += 3 45 | assertEquals(Set(1,2,3), builder.result()) 46 | } 47 | 48 | @Test 49 | def testSerializeAndBuildWithOrdering() { 50 | val bf = CanBuildFrom.treeSetCanBuildFrom[Int] 51 | val bf2 = SerializationUtils.roundtrip(bf) 52 | val builder = bf2.apply() 53 | builder += 1 54 | builder += 2 55 | builder += 3 56 | assertEquals(Set(1,2,3), builder.result()) 57 | } 58 | 59 | 60 | } 61 | -------------------------------------------------------------------------------- /driver/src/test/scala/com/datastax/spark/connector/types/CollectionColumnTypeSpec.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.types 20 | 21 | import org.scalatest.{FlatSpec, Matchers} 22 | 23 | class CollectionColumnTypeSpec extends FlatSpec with Matchers { 24 | 25 | val udt = UserDefinedType( 26 | name = "address", 27 | columns = IndexedSeq( 28 | UDTFieldDef("street", VarCharType), 29 | UDTFieldDef("number", IntType) 30 | ) 31 | ) 32 | 33 | "ListType" should "mark nested UDT types as frozen" in { 34 | ListType(udt).cqlTypeName shouldBe "list>" 35 | } 36 | 37 | it should "not mark non UDT types as frozen" in { 38 | ListType(IntType).cqlTypeName shouldBe "list" 39 | } 40 | 41 | "SetType" should "mark nested UDT types as frozen" in { 42 | SetType(udt).cqlTypeName shouldBe "set>" 43 | } 44 | 45 | it should "not mark non UDT types as frozen" in { 46 | SetType(IntType).cqlTypeName shouldBe "set" 47 | } 48 | 49 | "MapType" should "mark key UDT types as frozen" in { 50 | MapType(udt, IntType).cqlTypeName shouldBe "map, int>" 51 | } 52 | 53 | it should "mark value UDT types as frozen" in { 54 | MapType(IntType, udt).cqlTypeName shouldBe "map>" 55 | } 56 | 57 | } 58 | -------------------------------------------------------------------------------- /driver/src/test/scala/com/datastax/spark/connector/types/TimestampParserSpec.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.types 20 | 21 | import org.scalatest.FlatSpec 22 | 23 | class TimestampParserSpec extends FlatSpec { 24 | 25 | it should "parse fast all supported date[time[zone]] formats" in { 26 | /* look in [[DateTimeFormatter]] for 'X' definition*/ 27 | val validZones = Set( 28 | "", 29 | "Z", 30 | "-08", 31 | "-0830", 32 | "-08:30", 33 | "-083015", 34 | "-08:30:15", 35 | "+08", 36 | "+0830", 37 | "+08:30", 38 | "+083015", 39 | "+08:30:15" 40 | ) 41 | 42 | val validDates = Set( 43 | "1986-01-02", 44 | "1986-01-02 21:05", 45 | "1986-01-02 21:05:07", 46 | "1986-01-02 21:05:07.1", 47 | "1986-01-02 21:05:07.12", 48 | "1986-01-02 21:05:07.123" 49 | ) 50 | 51 | val datesAndDatesWithT = validDates 52 | .flatMap(date => Set(date) + date.replace(' ', 'T')) 53 | 54 | val allDates = for (date <- datesAndDatesWithT; zone <- validZones) yield { 55 | date + zone 56 | } 57 | 58 | allDates.foreach(TimestampParser.parseFastOrThrow) 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /driver/src/test/scala/com/datastax/spark/connector/types/TypeSerializationTest.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.types 20 | 21 | import org.apache.commons.lang3.SerializationUtils 22 | import org.junit.Assert._ 23 | import org.junit.Test 24 | 25 | class TypeSerializationTest { 26 | 27 | private def testSerialization(t: ColumnType[_]) { 28 | assertEquals(t, SerializationUtils.roundtrip(t)) 29 | } 30 | 31 | @Test 32 | def testSerializationOfPrimitiveTypes() { 33 | testSerialization(AsciiType) 34 | testSerialization(TextType) 35 | testSerialization(IntType) 36 | testSerialization(BigIntType) 37 | testSerialization(DoubleType) 38 | testSerialization(FloatType) 39 | testSerialization(BooleanType) 40 | testSerialization(UUIDType) 41 | testSerialization(TimeUUIDType) 42 | testSerialization(TimestampType) 43 | testSerialization(DecimalType) 44 | testSerialization(BigIntType) 45 | testSerialization(InetType) 46 | testSerialization(CounterType) 47 | testSerialization(SmallIntType) 48 | testSerialization(TinyIntType) 49 | testSerialization(DateType) 50 | } 51 | 52 | @Test 53 | def testSerializationOfCollectionTypes() { 54 | testSerialization(ListType(IntType)) 55 | testSerialization(ListType(ListType(IntType))) 56 | testSerialization(SetType(TextType)) 57 | testSerialization(MapType(BigIntType, TimestampType)) 58 | } 59 | 60 | 61 | } 62 | -------------------------------------------------------------------------------- /generateDocs.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | SCC_HOME="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" 3 | OUTPUT="/tmp/SCC_DOC_TEMP" 4 | 5 | if [ "$#" -lt 1 ]; then 6 | echo "At least one version parameter is required, e.g. 3.0.1 (no leading 'v')." 7 | exit 1 8 | fi 9 | 10 | 11 | rm -r $OUTPUT 12 | mkdir -p $OUTPUT 13 | echo "SPARK CASSANDRA CONNECTOR HOME IS $SCC_HOME" 14 | for VERSION in $@ ;do 15 | echo "Making docs for $VERSION" 16 | git checkout "v$VERSION" 17 | if [ $? -ne 0 ]; then 18 | echo "Unable to checkout version $VERSION, skipping" 19 | continue 20 | fi 21 | sbt clean 22 | sbt doc 23 | mkdir $OUTPUT/$VERSION 24 | 25 | for MODULE in connector driver test-support; do 26 | FOLDER=$SCC_HOME/$MODULE 27 | echo "COPYING $FOLDER to $OUTPUT/$VERSION/$MODULE" 28 | cp -vr $FOLDER/target/scala-2.12/api $OUTPUT/$VERSION/$MODULE 29 | done 30 | done 31 | git checkout gh-pages 32 | cp -r $OUTPUT/* $SCC_HOME/ApiDocs 33 | -------------------------------------------------------------------------------- /project/Versions.scala: -------------------------------------------------------------------------------- 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 | object Versions { 20 | 21 | val CommonsExec = "1.3" 22 | val CommonsIO = "2.8.0" 23 | val CommonsLang3 = "3.10" 24 | val Paranamer = "2.8" 25 | 26 | val CassandraJavaDriver = "4.18.1" 27 | val EsriGeometry = "2.2.4" 28 | 29 | val ScalaCheck = "1.14.0" 30 | val ScalaTest = "3.0.8" 31 | val JUnit = "4.12" 32 | val JUnitInterface = "0.11" 33 | val Mockito = "1.10.19" 34 | 35 | val ApacheSpark = "3.5.0" 36 | val SparkJetty = "9.4.51.v20230217" 37 | val SolrJ = "8.3.0" 38 | 39 | val ScalaCompat = "2.11.0" 40 | val ParallelCollections = "1.0.4" 41 | } 42 | -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version=1.10.0 2 | -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.10") 2 | -------------------------------------------------------------------------------- /project/updates.sbt: -------------------------------------------------------------------------------- 1 | addSbtPlugin("com.timushev.sbt" % "sbt-updates" % "0.4.1") -------------------------------------------------------------------------------- /rootdoc.txt: -------------------------------------------------------------------------------- 1 | Apache Spark Connector for Apache Cassandra. 2 | See documentation of package [[com.datastax.spark.connector]]. -------------------------------------------------------------------------------- /sbt/sbt: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one or more 5 | # contributor license agreements. See the NOTICE file distributed with 6 | # this work for additional information regarding copyright ownership. 7 | # The ASF licenses this file to You under the Apache License, Version 2.0 8 | # (the "License"); you may not use this file except in compliance with 9 | # the License. You may obtain a copy of the License at 10 | # 11 | # http://www.apache.org/licenses/LICENSE-2.0 12 | # 13 | # Unless required by applicable law or agreed to in writing, software 14 | # distributed under the License is distributed on an "AS IS" BASIS, 15 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | # See the License for the specific language governing permissions and 17 | # limitations under the License. 18 | # 19 | 20 | # This script launches sbt for this project. If present it uses the system 21 | # version of sbt. If there is no system version of sbt it attempts to download 22 | # sbt locally. 23 | SBT_VERSION=1.10.0 24 | URL=https://repo1.maven.org/maven2/org/scala-sbt/sbt-launch/${SBT_VERSION}/sbt-launch-${SBT_VERSION}.jar 25 | JAR=sbt/sbt-launch-${SBT_VERSION}.jar 26 | 27 | # Download sbt launch jar if it hasn't been downloaded yet 28 | if [ ! -f ${JAR} ]; then 29 | # Download 30 | printf "Attempting to fetch sbt\n" 31 | JAR_DL=${JAR}.part 32 | if hash curl 2>/dev/null; then 33 | curl -f -L --progress-bar ${URL} -o ${JAR_DL} && mv ${JAR_DL} ${JAR} 34 | elif hash wget 2>/dev/null; then 35 | wget --progress=bar ${URL} -O ${JAR_DL} && mv ${JAR_DL} ${JAR} 36 | else 37 | printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" 38 | exit -1 39 | fi 40 | fi 41 | if [ ! -f ${JAR} ]; then 42 | # We failed to download 43 | printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" 44 | exit -1 45 | fi 46 | printf "Launching sbt from ${JAR}\n" 47 | java \ 48 | -Xmx1200m -XX:MaxPermSize=350m -XX:ReservedCodeCacheSize=256m \ 49 | -jar ${JAR} \ 50 | "$@" 51 | -------------------------------------------------------------------------------- /test-support/src/main/resources/client.crt: -------------------------------------------------------------------------------- 1 | -----BEGIN CERTIFICATE----- 2 | MIIDqTCCApGgAwIBAgIERLZiJzANBgkqhkiG9w0BAQsFADCBhDELMAkGA1UEBhMCVVMxEzARBgNV 3 | BAgTCkNhbGlmb3JuaWExFDASBgNVBAcTC1NhbnRhIENsYXJhMRYwFAYDVQQKEw1EYXRhU3RheCBJ 4 | bmMuMRowGAYDVQQLExFEcml2ZXJzIGFuZCBUb29sczEWMBQGA1UEAxMNRHJpdmVyIENsaWVudDAe 5 | Fw0xNTAzMTIwMTA4MjRaFw0xNTA2MTAwMTA4MjRaMIGEMQswCQYDVQQGEwJVUzETMBEGA1UECBMK 6 | Q2FsaWZvcm5pYTEUMBIGA1UEBxMLU2FudGEgQ2xhcmExFjAUBgNVBAoTDURhdGFTdGF4IEluYy4x 7 | GjAYBgNVBAsTEURyaXZlcnMgYW5kIFRvb2xzMRYwFAYDVQQDEw1Ecml2ZXIgQ2xpZW50MIIBIjAN 8 | BgkqhkiG9w0BAQEFAAOCAQ8AMIIBCgKCAQEAq0J0EoZQnOv2KRrvwA+1ZL9VZ3hDdQMwkDfitoGN 9 | B6upvMUZpf8W+ReQmaY6yacYJthHzsZTd3G97Bw81/3VNHQB9PnXGmbupMLVXeFXysSCs1nPEdJl 10 | TBbJXWHSh41AE4ejJaoCoTuigKGwI9lTbOOPDz/WMcio9nagsCJdsdG2+TxmR7RlyzEIANJ0wpnL 11 | JEIeJmRS2loLVuCU4lZ9hDLN57cP9jEVD4Hk2kJD4Exx7G9HQFH+/63H6XtEDZsJcYldR7yBNsGr 12 | pz9CupULCS1R40ePQEIlUXhM4ft/hsljQybLQvvfXNVTvk5WgY7LNaBJy6A/Tfg32SXEn3wUvwID 13 | AQABoyEwHzAdBgNVHQ4EFgQUt+JDOeziZzHNYTFU/FL9PhDGqSQwDQYJKoZIhvcNAQELBQADggEB 14 | ADOYpa1f9dPcVLq3RiMytajHo3YJ0AQqGRzVgngkeRFSdhyy/y+/8D0/V5s6QbNt/l6x3FxkoiTR 15 | 1Lptf96eylnS5AkGQTgogJP53cSNrqkDL0IyyvErSiATEXNpBKz6ivY+e5J1GLTfX9Ylu8limzIq 16 | Y6YBnr8fMLD6XWraxtzzkJ9NIPhhaz696rxqr8ix6uy0mgxR/7/jUglreimZkLW40/qiABgX7Evw 17 | UqpuJWmqNbQP9UXecx/UJ0hdxxxuxkZsoRoQwWYhkeT4aGCLJv/hjiNTfFAt23uHe0LVfW/HqykW 18 | KoEj8F08mJVe5ZfpjF974i5qO9PU9XxvLfLjNvo= 19 | -----END CERTIFICATE----- 20 | -------------------------------------------------------------------------------- /test-support/src/main/resources/client.key: -------------------------------------------------------------------------------- 1 | -----BEGIN PRIVATE KEY----- 2 | MIIEvAIBADANBgkqhkiG9w0BAQEFAASCBKYwggSiAgEAAoIBAQCrQnQShlCc6/Yp 3 | Gu/AD7Vkv1VneEN1AzCQN+K2gY0Hq6m8xRml/xb5F5CZpjrJpxgm2EfOxlN3cb3s 4 | HDzX/dU0dAH0+dcaZu6kwtVd4VfKxIKzWc8R0mVMFsldYdKHjUATh6MlqgKhO6KA 5 | obAj2VNs448PP9YxyKj2dqCwIl2x0bb5PGZHtGXLMQgA0nTCmcskQh4mZFLaWgtW 6 | 4JTiVn2EMs3ntw/2MRUPgeTaQkPgTHHsb0dAUf7/rcfpe0QNmwlxiV1HvIE2waun 7 | P0K6lQsJLVHjR49AQiVReEzh+3+GyWNDJstC+99c1VO+TlaBjss1oEnLoD9N+DfZ 8 | JcSffBS/AgMBAAECggEAMHATNEoY8skqTmX3+XJ3847KMQGq0qWcTq3/yW7K3KiI 9 | 0YNNxc1oSfuIQmzpo69G/XWembUuVlItTWKPMufwLW3CP++KD0WdqawRfQQHOKpr 10 | 7R4xmvDPBb5MJcVNLlmdDekHE9gJ9mBPjeItV3ZYSivygnWjt2DxqQPUXvzZUzlu 11 | munh3H5x6ehXVHDYGzosPgTpfmLHdlNfvF4x9bcklMMbCOyoPttXB2uHWOvUIS+/ 12 | 2YEkPmJfZdpudI7RqN75yYi7N8+gpnCTp530zA2yONyZ8THqEG/0nWy+02/zm5sm 13 | gs1saYNwXME2IPekZNM/pJh2DtnTcxZaUt84q2nhAQKBgQDi8mgvE8ekbs6DLfKK 14 | YAtTuOcLRpuvJqxtiQecqaumzgZnmHtkm6yuDNjieqB6OITudP4NdhPpyvOFJw46 15 | zTHMpGqZboxHuxoxMOgmyeiO+cdSwGHobr1zUcT8jVmLH7A+LtL5hHi+733EbCRh 16 | sF04Vq9L46Q52mhcZKbs56U8MQKBgQDBLwotnOJH7hZD5sKS0o8/Sfj3pgzXEDpL 17 | RfnrBPGhLn+1zhPEYsEW3mKI/yHiOZHNXZMQ6oYmxThg03qKTjaY8OIm8sg/zrlZ 18 | M+o3wVnAzayrhw5gZ8DzqioHhEUMOAwwRFXRpfxqj8regrLjE9KaYty8ZYAFtwuH 19 | W2S3+MVT7wKBgGQx7XlLXErmeNpFgN1Cxf1ylt7Nj5Jmmp3Jb8jkx9ne/8jg8ylZ 20 | 6YT2OxLSXONY7Kdyk29SADyp05WnxoqDaUcWF9IhkmFg45FwLC5j2f61nCCWuyMp 21 | MQ8mvLdbmHrpxJ/PgGmU6NIzXe1IaU+P07g53S6+FBVOreCMt33ET5khAoGAGgKz 22 | ZCDTdsvfw5S2bf5buzHCi9WXtP1CXBA37iTkQ8d2+oucrbx+Mw4ORlPTxBnsP7Jx 23 | sr1hAqdbR+4xeZ2+TCliycu2mqDC4/fReWBXLVaEATRWAzT1DdnDfu+YPGTvfzA0 24 | Pd4TdmWV8w+19k0c9hyJi/Q+oIZczwTHMt4T85ECgYAe4J0ht6b6kPEG3d9vxmMN 25 | T23S+ucYLHnfT1nacTuBZnMphWHhSqf8UJloIGpusxDU84MdAp22Jpd9SfPi9KK9 26 | yZY9WDJGeb0Yk7ML1R5GcAAkM78lUw/rS2VfMjQFnnUl2jVMS8adcm8/vHcpkcn7 27 | MufMEZzDpeO/aI8nbClktw== 28 | -----END PRIVATE KEY----- 29 | -------------------------------------------------------------------------------- /test-support/src/main/resources/client.keystore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-spark-connector/1c2ffa11ca9751ae36ab07a68f83ffa33d91b89c/test-support/src/main/resources/client.keystore -------------------------------------------------------------------------------- /test-support/src/main/resources/client.truststore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-spark-connector/1c2ffa11ca9751ae36ab07a68f83ffa33d91b89c/test-support/src/main/resources/client.truststore -------------------------------------------------------------------------------- /test-support/src/main/resources/server.keystore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-spark-connector/1c2ffa11ca9751ae36ab07a68f83ffa33d91b89c/test-support/src/main/resources/server.keystore -------------------------------------------------------------------------------- /test-support/src/main/resources/server.truststore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-spark-connector/1c2ffa11ca9751ae36ab07a68f83ffa33d91b89c/test-support/src/main/resources/server.truststore -------------------------------------------------------------------------------- /test-support/src/main/resources/server_localhost.keystore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/cassandra-spark-connector/1c2ffa11ca9751ae36ab07a68f83ffa33d91b89c/test-support/src/main/resources/server_localhost.keystore -------------------------------------------------------------------------------- /test-support/src/main/scala/com/datastax/spark/connector/ccm/mode/DebugModeExecutor.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.ccm.mode 20 | 21 | import java.nio.file.{Files, Path, Paths} 22 | 23 | import com.datastax.spark.connector.ccm.CcmConfig 24 | import org.slf4j.{Logger, LoggerFactory} 25 | 26 | private[ccm] class DebugModeExecutor(val config: CcmConfig) extends DefaultExecutor { 27 | 28 | private val logger: Logger = LoggerFactory.getLogger(classOf[StandardModeExecutor]) 29 | 30 | private val Cwd = Paths.get("").toAbsolutePath().toString(); 31 | 32 | override val dir: Path = { 33 | sys.env.get("PRESERVE_LOGS") match { 34 | case Some(dir) => 35 | val subPath = s"$Cwd/$dir/ccm_${config.ipPrefix 36 | .replace(".","_") 37 | .stripSuffix("_")}" 38 | 39 | val path = Files.createDirectories(Paths.get(subPath)) 40 | logger.debug(s"Preserving CCM Install Directory at [$path]. It will not be removed") 41 | logger.debug(s"Checking directory exists [${Files.exists(path)}]") 42 | path 43 | case None => 44 | val tmp = Files.createTempDirectory("ccm") 45 | tmp.toFile.deleteOnExit() 46 | tmp 47 | } 48 | } 49 | 50 | // stop nodes, don't remove logs 51 | override def remove(): Unit = { 52 | execute("stop") 53 | } 54 | 55 | } 56 | -------------------------------------------------------------------------------- /test-support/src/main/scala/com/datastax/spark/connector/ccm/mode/ExistingModeExecutor.scala: -------------------------------------------------------------------------------- 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 com.datastax.spark.connector.ccm.mode 20 | import com.datastax.spark.connector.ccm.CcmConfig 21 | 22 | import java.nio.file.{Files, Path} 23 | 24 | /** 25 | * A special ClusterModeExecutor which bypasses ccm and assumes a Cassandra instance on localhost 26 | * with default ports and no authentication. 27 | * */ 28 | private[ccm] class ExistingModeExecutor(val config: CcmConfig) extends ClusterModeExecutor { 29 | override protected val dir: Path = Files.createTempDirectory("test") 30 | 31 | override def create(clusterName: String): Unit = { 32 | // do nothing 33 | } 34 | 35 | override def start(nodeNo: Int): Unit = { 36 | // do nothing 37 | } 38 | 39 | override def remove(): Unit = { 40 | // do nothing 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /testing/scripts/mac-localhost-aliases.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | tmpfile=$TMPDIR/localhost-aliases-root.sh 3 | tmpfile2=$TMPDIR/localhost-aliases-root2.sh 4 | 5 | pre="alias" 6 | post=" up" 7 | if [[ "$1" == "-" ]] ; then 8 | pre="-alias" 9 | post="" 10 | fi 11 | 12 | echo "#!/bin/bash" > "$tmpfile" 13 | for dc in $(seq 0 15); do 14 | for rack in $(seq 0 2); do 15 | for node in $(seq 0 2); do 16 | echo "ifconfig lo0 ${pre} 127.${dc}.${rack}.${node}${post}" >> "$tmpfile" 17 | done 18 | done 19 | done 20 | grep -v 127.0.0.1 "$tmpfile" | grep -v 127.0.0.0 > "$tmpfile2" 21 | chmod 755 "$tmpfile2" 22 | sudo "$tmpfile2" 23 | rm -f "$tmpfile" "$tmpfile2" 24 | --------------------------------------------------------------------------------