Showing preview only (1,311K chars total). Download the full file or copy to clipboard to get everything.
Repository: memsql/memsql-spark-connector
Branch: master
Commit: c381848aa5e9
Files: 125
Total size: 1.2 MB
Directory structure:
gitextract_u_sh6j0w/
├── .arcconfig
├── .github/
│ └── workflows/
│ └── test-and-publish.yml
├── .gitignore
├── .java-version
├── .scalafmt.conf
├── CHANGELOG
├── LICENSE
├── README.md
├── build.sbt
├── ci/
│ └── secring.asc.enc
├── demo/
│ ├── Dockerfile
│ ├── README.md
│ └── notebook/
│ ├── pyspark-singlestore-demo_2F8XQUKFG.zpln
│ ├── scala-singlestore-demo_2F6Y3APTX.zpln
│ └── spark-sql-singlestore-demo_2F7PZ81H6.zpln
├── project/
│ ├── build.properties
│ └── plugins.sbt
├── scripts/
│ ├── jwt/
│ │ └── jwt_auth_config.json
│ └── setup-cluster.sh
└── src/
├── main/
│ ├── resources/
│ │ └── META-INF/
│ │ └── services/
│ │ └── org.apache.spark.sql.sources.DataSourceRegister
│ ├── scala/
│ │ └── com/
│ │ ├── memsql/
│ │ │ └── spark/
│ │ │ └── DefaultSource.scala
│ │ └── singlestore/
│ │ └── spark/
│ │ ├── AggregatorParallelReadListener.scala
│ │ ├── AvroSchemaHelper.scala
│ │ ├── CompletionIterator.scala
│ │ ├── DefaultSource.scala
│ │ ├── ExpressionGen.scala
│ │ ├── JdbcHelpers.scala
│ │ ├── LazyLogging.scala
│ │ ├── Loan.scala
│ │ ├── MetricsHandler.scala
│ │ ├── OverwriteBehavior.scala
│ │ ├── ParallelReadEnablement.scala
│ │ ├── ParallelReadType.scala
│ │ ├── SQLGen.scala
│ │ ├── SQLHelper.scala
│ │ ├── SQLPushdownRule.scala
│ │ ├── SinglestoreBatchInsertWriter.scala
│ │ ├── SinglestoreConnectionPool.scala
│ │ ├── SinglestoreConnectionPoolOptions.scala
│ │ ├── SinglestoreDialect.scala
│ │ ├── SinglestoreLoadDataWriter.scala
│ │ ├── SinglestoreOptions.scala
│ │ ├── SinglestorePartitioner.scala
│ │ ├── SinglestoreRDD.scala
│ │ ├── SinglestoreReader.scala
│ │ └── vendor/
│ │ └── apache/
│ │ ├── SchemaConverters.scala
│ │ └── third_party_license
│ ├── scala-sparkv3.1/
│ │ └── spark/
│ │ ├── MaxNumConcurentTasks.scala
│ │ ├── VersionSpecificAggregateExpressionExtractor.scala
│ │ ├── VersionSpecificAggregateExtractor.scala
│ │ ├── VersionSpecificExpressionGen.scala
│ │ ├── VersionSpecificSortExtractor.scala
│ │ ├── VersionSpecificUtil.scala
│ │ ├── VersionSpecificWindowBoundaryExpressionExtractor.scala
│ │ └── VersionSpecificWindowExtractor.scala
│ ├── scala-sparkv3.2/
│ │ └── spark/
│ │ ├── MaxNumConcurrentTasks.scala
│ │ ├── VersionSpecificAggregateExpressionExtractor.scala
│ │ ├── VersionSpecificAggregateExtractor.scala
│ │ ├── VersionSpecificExpressionGen.scala
│ │ ├── VersionSpecificSortExtractor.scala
│ │ ├── VersionSpecificUtil.scala
│ │ ├── VersionSpecificWindowBoundaryExpressionExtractor.scala
│ │ └── VersionSpecificWindowExtractor.scala
│ ├── scala-sparkv3.3/
│ │ └── spark/
│ │ ├── MaxNumConcurrentTasks.scala
│ │ ├── VersionSpecificAggregateExpressionExtractor.scala
│ │ ├── VersionSpecificAggregateExtractor.scala
│ │ ├── VersionSpecificExpressionGen.scala
│ │ ├── VersionSpecificSortExtractor.scala
│ │ ├── VersionSpecificUtil.scala
│ │ ├── VersionSpecificWindowBoundaryExpressionExtractor.scala
│ │ └── VersionSpecificWindowExtractor.scala
│ ├── scala-sparkv3.4/
│ │ └── spark/
│ │ ├── MaxNumConcurrentTasks.scala
│ │ ├── VersionSpecificAggregateExpressionExtractor.scala
│ │ ├── VersionSpecificAggregateExtractor.scala
│ │ ├── VersionSpecificExpressionGen.scala
│ │ ├── VersionSpecificSortExtractor.scala
│ │ ├── VersionSpecificUtil.scala
│ │ ├── VersionSpecificWindowBoundaryExpressionExtractor.scala
│ │ └── VersionSpecificWindowExtractor.scala
│ ├── scala-sparkv3.5/
│ │ └── spark/
│ │ ├── MaxNumConcurrentTasks.scala
│ │ ├── VersionSpecificAggregateExpressionExtractor.scala
│ │ ├── VersionSpecificAggregateExtractor.scala
│ │ ├── VersionSpecificExpressionGen.scala
│ │ ├── VersionSpecificSortExtractor.scala
│ │ ├── VersionSpecificUtil.scala
│ │ ├── VersionSpecificWindowBoundaryExpressionExtractor.scala
│ │ └── VersionSpecificWindowExtractor.scala
│ └── scala-sparkv4.0/
│ └── spark/
│ ├── MaxNumConcurrentTasks.scala
│ ├── VersionSpecificAggregateExpressionExtractor.scala
│ ├── VersionSpecificAggregateExtractor.scala
│ ├── VersionSpecificExpressionGen.scala
│ ├── VersionSpecificSortExtractor.scala
│ ├── VersionSpecificUtil.scala
│ ├── VersionSpecificWindowBoundaryExpressionExtractor.scala
│ └── VersionSpecificWindowExtractor.scala
└── test/
├── resources/
│ ├── data/
│ │ ├── movies.json
│ │ ├── movies_rating.json
│ │ ├── reviews.json
│ │ └── users.json
│ ├── log4j.properties
│ ├── log4j2.properties
│ └── mockito-extensions/
│ └── org.mockito.plugins.MockMaker
└── scala/
└── com/
└── singlestore/
└── spark/
├── BatchInsertBenchmark.scala
├── BatchInsertTest.scala
├── BenchmarkSerializingTest.scala
├── BinaryTypeBenchmark.scala
├── CustomDatatypesTest.scala
├── ExternalHostTest.scala
├── IntegrationSuiteBase.scala
├── IssuesTest.scala
├── LoadDataBenchmark.scala
├── LoadDataTest.scala
├── LoadbalanceTest.scala
├── MaxErrorsTest.scala
├── OutputMetricsTest.scala
├── ReferenceTableTest.scala
├── SQLHelperTest.scala
├── SQLOverwriteTest.scala
├── SQLPermissionsTest.scala
├── SQLPushdownTest.scala
├── SanityTest.scala
├── SinglestoreConnectionPoolTest.scala
├── SinglestoreOptionsTest.scala
├── TestHelper.scala
└── VersionTest.scala
================================================
FILE CONTENTS
================================================
================================================
FILE: .arcconfig
================================================
{
"project_id" : "memsql-spark-connector",
"conduit_uri" : "https:\/\/grizzly.internal.memcompute.com\/api\/"
}
================================================
FILE: .github/workflows/test-and-publish.yml
================================================
name: Test and Publish
on:
pull_request:
types: [ opened, synchronize, reopened ]
schedule:
- cron: "0 0 * * 0"
push:
tags:
- "v*"
jobs:
fetch-s2-versions:
runs-on: ubuntu-latest
outputs:
versions: ${{ steps.get_versions.outputs.versions }}
steps:
- name: Get supported versions of Singlestore
id: get_versions
uses: singlestore-labs/singlestore-supported-versions@main
test:
needs: fetch-s2-versions
runs-on: ubuntu-latest
strategy:
matrix:
singlestore_version: ${{ fromJson(needs.fetch-s2-versions.outputs.versions) }}
spark_version: ['3.1.3', '3.2.4', '3.3.4', '3.4.2', '3.5.0', '4.0.0']
force_read_from_leaves: ['FALSE']
include:
- spark_version: '3.1.3'
tag: 'ExcludeFromSpark31'
scala_version: '2.12.12'
java_version: '11'
- spark_version: '3.2.4'
tag: 'ExcludeFromSpark32'
scala_version: '2.12.12'
java_version: '11'
- spark_version: '3.3.4'
tag: 'ExcludeFromSpark33'
scala_version: '2.12.12'
java_version: '11'
- spark_version: '3.4.2'
tag: 'ExcludeFromSpark34'
scala_version: '2.12.12'
java_version: '11'
- spark_version: '3.5.0'
tag: 'ExcludeFromSpark35'
scala_version: '2.12.12'
java_version: '11'
- spark_version: '4.0.0'
tag: 'ExcludeFromSpark40'
scala_version: '2.13.8'
java_version: '17'
- force_read_from_leaves: 'TRUE'
singlestore_version: '9.0'
spark_version: '4.0.0'
tag: 'ExcludeFromSpark40'
scala_version: '2.13.8'
java_version: '17'
steps:
- name: Remove unnecessary pre-installed toolchains for free disk spaces
run: |
echo "=== BEFORE ==="
df -h
sudo rm -rf /usr/share/dotnet
sudo rm -rf /opt/ghc
sudo rm -rf /usr/local/share/boost
sudo rm -rf "$AGENT_TOOLSDIRECTORY"
sudo rm -rf /usr/local/lib/android
sudo rm -rf /opt/hostedtoolcache/CodeQL
sudo rm -rf /opt/hostedtoolcache/Ruby
sudo rm -rf /opt/hostedtoolcache/Go
docker system prune -af || true
sudo apt-get clean
echo "=== AFTER ==="
df -h
- uses: actions/checkout@v4
- name: Set up test cluster
env:
SINGLESTORE_LICENSE: ${{ secrets.SINGLESTORE_LICENSE }}
ROOT_PASSWORD: ${{ secrets.SINGLESTORE_PASSWORD }}
SINGLESTORE_VERSION: ${{ matrix.singlestore_version }}
run: ./scripts/setup-cluster.sh
- name: Set up JDK
uses: actions/setup-java@v4
with:
java-version: ${{ matrix.java_version }}
distribution: 'temurin'
cache: sbt
- name: Set up sbt launcher
uses: sbt/setup-sbt@v1
- name: Run tests for Spark ${{ matrix.spark_version }}
env:
SINGLESTORE_JWT_PASSWORD: ${{ secrets.SINGLESTORE_JWT_PASSWORD }}
SINGLESTORE_PASSWORD: ${{ secrets.SINGLESTORE_PASSWORD }}
FORCE_READ_FROM_LEAVES: ${{ matrix.force_read_from_leaves }}
run: sbt ++${{ matrix.scala_version }} "testOnly -- -l ${{ matrix.tag }}" -Dspark.version=${{ matrix.spark_version }}
publish:
needs: test
if: startsWith(github.ref, 'refs/tags/v')
runs-on: ubuntu-latest
strategy:
matrix:
spark_version: ['3.1.3', '3.2.4', '3.3.4', '3.4.2', '3.5.0', '4.0.0']
include:
- spark_version: '3.1.3'
scala_version: '2.12.12'
java_version: '11'
- spark_version: '3.2.4'
scala_version: '2.12.12'
java_version: '11'
- spark_version: '3.3.4'
scala_version: '2.12.12'
java_version: '11'
- spark_version: '3.4.2'
scala_version: '2.12.12'
java_version: '11'
- spark_version: '3.5.0'
scala_version: '2.12.12'
java_version: '11'
- spark_version: '4.0.0'
scala_version: '2.13.8'
java_version: '17'
steps:
- uses: actions/checkout@v4
- name: Set up JDK
uses: actions/setup-java@v4
with:
java-version: ${{ matrix.java_version }}
distribution: 'temurin'
cache: sbt
- name: Set up sbt launcher
uses: sbt/setup-sbt@v1
- name: Set up GPG
env:
ENCRYPTION_KEY: ${{ secrets.ENCRYPTION_KEY }}
ENCRYPTION_IV: ${{ secrets.ENCRYPTION_IV }}
run: |
openssl enc -d -aes-256-cbc -K $ENCRYPTION_KEY -iv $ENCRYPTION_IV -in ci/secring.asc.enc -out ci/secring.asc
gpg --import ci/secring.asc
- name: Publish Spark ${{ matrix.spark_version }}
env:
SONATYPE_USERNAME: ${{ secrets.SONATYPE_USERNAME }}
SONATYPE_PASSWORD: ${{ secrets.SONATYPE_PASSWORD }}
run: |
sbt ++${{ matrix.scala_version }} -Dspark.version=${{ matrix.spark_version }} clean publishSigned sonatypeBundleRelease
================================================
FILE: .gitignore
================================================
# Covers JetBrains IDEs: IntelliJ, RubyMine, PhpStorm, AppCode, PyCharm, CLion, Android Studio and WebStorm
# Reference: https://intellij-support.jetbrains.com/hc/en-us/articles/206544839
# User-specific stuff
/.idea/*
!/.idea/codeStyles/*
!/.idea/runConfigurations/*
# JIRA plugin
atlassian-ide-plugin.xml
# IntelliJ
/out/
/target/
# mpeltonen/sbt-idea plugin
/.idea_modules/
# File-based project format
/*.iws
# sbt project stuff
/project/*
!/project/build.properties
!/project/plugins.sbt
/target
/build
/spark-warehouse
/ci/secring.asc
scripts/ssl
================================================
FILE: .java-version
================================================
1.8
================================================
FILE: .scalafmt.conf
================================================
maxColumn = 100
align = more
================================================
FILE: CHANGELOG
================================================
2026-04-02 Version 5.0.1
* Added customSchema option
2026-04-02 Version 5.0.0
* Removed setting of the default server colation
* Fixed parallel read from leaves for SingleStore versions 9.0.0 and later
2025-09-24 Version 4.2.2
* Restored proper connection closure logic
2025-08-15 Version 4.2.1
* Fixed compatibility issue with Databricks 16.4
2025-07-22 Version 4.2.0
* Added support of Spark 4.0.0
* Changed type conversion for TINYINT to ByteType
2024-11-22 Version 4.1.10
* Updated JDBC driver to 1.2.7
2024-11-22 Version 4.1.9
* Changed to work with Databricks runtime 15.4
2024-06-14 Version 4.1.8
* Changed retry during reading from result table to use exponential backoff
* Used ForkJoinPool instead of FixedThreadPool
* Added more logging
2024-05-13 Version 4.1.7
* Fixed bug that caused reading from the wrong result table when the task was restarted
2024-04-11 Version 4.1.6
* Changed LoadDataWriter to send data in batches
* Added numPartitions parameter to specify exact number of resulting partition during parallel read
2023-10-05 Version 4.1.5
* Added support of Spark 3.5
* Updated dependencies
2023-07-18 Version 4.1.4
* Added support of Spark 3.4
* Added connection attributes
* Fixed conflicts of result table names during parallel read
* Updated version of the SingleStore JDBC driver
2023-03-31 Version 4.1.3
* Updated version of the SingleStore JDBC driver
* Fixed error handling when `onDuplicateKeySQL` option is used
2023-02-21 Version 4.1.2
* Fixed an issue that would cause a `Table has reached its quota of 1 reader(s)`` error to be displayed when a parallel read was retried
2022-07-13 Version 4.1.1
* Added clientEndpoint option for Cloud deployment of the SingleStoreDB
* Fixed bug in the error handling that caused deadlock
* Added support of the Spark 3.3
2022-06-22 Version 4.1.0
* Added support of more SQL expressions in pushdown
* Added multi-partition to the parallel read
* Updated SingleStore JDBC Driver to 1.1.0
* Added JWT authentication
* Added connection pooling
2022-01-20 Version 4.0.0
* Changed connector to use SingleStore JDBC Driver instead of MariaDB JDBC Driver
2021-12-23 Version 3.2.2
* Added possibility to repartition result by columns in parallel read from aggregators
* Replaced usages of `transformDown` with `transform` in order to make connector work with Databricks 9.1 LTS
2021-12-14 Version 3.2.1
* Added support of the Spark 3.2
* Fixed links in the README
2021-11-29 Version 3.2.0
* Added support for reading in parallel from aggregator nodes instead of leaf nodes
2021-09-16 Version 3.1.3
* Added Spark 3.1 support
* Deleted Spark 2.3 and 2.4 support
2021-04-29 Version 3.1.2
* Added using external host and port by default while using `useParallelRead`
2021-02-05 Version 3.1.1
* Added support of `com.memsql.spark` data source name for backward compatibility
2021-01-22 Version 3.1.0
* Rebranded `memsql-spark-connector` to `singlestore-spark-connector`
* Spark data source format changed from `memsql` to `singlestore`
* Configuration prefix changed from `spark.datasource.memsql.<config_name>` to `spark.datasource.singlestore.<config_name>`
2020-10-19 Version 3.0.5
* Fixed bug with load balance connections to dml endpoint
2020-09-29 Version 3.1.0-beta
* Added Spark 3.0 support
* Fixed bugs in pushdowns
* Fixed bug with wrong SQL code generation of attribute names that contains special characters
* Added methods that allow you to run SQL queries on a MemSQL database directly
2020-08-20 Version 3.0.4
* Added trim pushdown
2020-08-14 Version 3.0.3
* Fixed bug with pushdown of the join condition
2020-08-03 Version 3.0.2
* added maxErrors option
* changed aliases in SQL queries to be more deterministic
* disabled comments inside of the SQL queries when logging level is not TRACE
2020-06-12 Version 3.0.1
* The connector now updates task metrics with the number of records written during write operations
2020-05-27 Version 3.0.0
* Introduces SQL Optimization & Rewrite for most query shapes and compatible expressions
* Implemented as a native Spark SQL plugin
* Supports both the DataSource and DataSourceV2 API for maximum support of current and future functionality
* Contains deep integrations with the Catalyst query optimizer
* Is compatible with Spark 2.3 and 2.4
* Leverages MemSQL LOAD DATA to accelerate ingest from Spark via compression, vectorized cpu instructions, and optimized segment sizes
* Takes advantage of all the latest and greatest features in MemSQL 7.x
2020-05-06 Version 3.0.0-rc1
* Support writing into MemSQL reference tables
* Deprecated truncate option in favor of overwriteBehavior
* New option overwriteBehavior allows you to specify how to overwrite or merge rows during ingest
* The Ignore SaveMode now correctly skips all duplicate key errors during ingest
2020-04-30 Version 3.0.0-beta12
* Improved performance of new batch insert functionality for `ON DUPLICATE KEY UPDATE` feature
2020-04-30 Version 3.0.0-beta11
* Added support for merging rows on ingest via `ON DUPLICATE KEY UPDATE`
* Added docker-based demo for running a Zeppelin notebook using the Spark connector
2020-04-20 Version 3.0.0-beta10
* Additional functions supported in SQL Pushdown: toUnixTimestamp, unixTimestamp, nextDay, dateDiff, monthsAdd, hypot, rint
* Now tested against MemSQL 6.7, and all tests use SSL
* Fixed bug with disablePushdown
2020-04-09 Version 3.0.0-beta9
* Add null handling to address Spark bug which causes incorrect handling of null literals (https://issues.apache.org/jira/browse/SPARK-31403)
2020-04-01 Version 3.0.0-beta8
* Added support for more datetime expressions:
* addition/subtraction of datetime objects
* to_utc_timestamp, from_utc_timestamp
* date_trunc, trunc
2020-03-25 Version 3.0.0-beta7
* The connector now respects column selection when loading dataframes into MemSQL
2020-03-24 Version 3.0.0-beta6
* Fix bug when you use an expression in an explicit query
2020-03-23 Version 3.0.0-beta5
* Increase connection timeout to increase connector reliability
2020-03-20 Version 3.0.0-beta4
* Set JDBC driver to MariaDB explicitely to avoid issues with the mysql driver
2020-03-19 Version 3.0.0-beta3
* Created tables default to Columnstore
* User can override keys attached to new tables
* New parallelRead option which enables reading directly from MemSQL leaf nodes
* Created tables now set case-sensitive collation on all columns
to match Spark semantics
* More SQL expressions supported in pushdown (tanh, sinh, cosh)
2020-02-08 Version 3.0.0-beta2
* Removed options: masterHost and masterPort
* Added ddlEndpoint and ddlEndpoints options
* Added path option to support specifying the dbtable via `.load("mytable")` when creating a dataframe
2020-01-30 Version 3.0.0-beta
* Full re-write of the Spark Connector
2019-02-27 Version 2.0.7
* Add support for EXPLAIN JSON in MemSQL versions 6.7 and later to fix partition pushdown.
2018-09-14 Version 2.0.6
* Force utf-8 encoding when loading data into MemSQL
2018-01-18 Version 2.0.5
* Explicitly sort MemSQLRDD partitions due to MemSQL 6.0 no longer returning partitions in sorted order by ordinal.
2017-08-31 Version 2.0.4
* Switch threads in LoadDataStrategy so that the parent thread reads from the RDD and the new thread writes
to MemSQL so that Spark has access to the thread-local variables it expects
2017-07-19 Version 2.0.3
* Handle special characters column names in query
* Add option to enable jdbc connector to stream result sets row-by-row
* Fix groupby queries incorrectly pushed down to leaves
* Add option to write to master aggregator only
* Add support for reading MemSQL columns of type unsigned bigint and unsigned int
2017-04-17
* Pull MemSQL configuration from runtime configuration in sparkSession.conf instead of static config in sparkContext
* Fix connection pooling bug where extraneous connections were created
* Add MemSQL configuration to disable partition pushdown
2017-02-06 Version 2.0.1
* Fixed bug to enable partition pushdown for MemSQL DataFrames loaded from a custom user query
2017-02-01 Version 2.0.0
* Compatible with Apache Spark 2.0.0+
* Removed experimental strategy SQL pushdown to instead use the more stable Data Sources API for reading
data from MemSQL
* Removed memsql-spark-interface, memsql-etl
2015-12-15 Version 1.2.1
* Python support for extractors and transformers
* More extensive SQL pushdown for DataFrame operations
* Use DataFrames as common interface between extractor, transformer, and loader
* Rewrite connectorLib internals to support SparkSQL relation provider API
* Remove RDD.saveToMemSQL
2015-11-19 Version 1.1.1
* Set JDBC login timeout to 10 seconds
2015-11-02 Version 1.1.0
* Available on Maven Central Repository
* More events for batches
* Deprecated the old Kafka extractor and replaced it with a new one that takes in a Zookeeper quorum address
* Added a new field to pipeline API responses indicating whether or not a pipeline is currently running
* Renamed projects: memsqlsparkinterface -> memsql-spark-interface, memsqletl -> memsql-etl, memsqlrdd -> memsql-connector.
* Robustness and bug fixes
2015-09-24 Version 1.0.0
* Initial release of MemSQL Streamliner
================================================
FILE: LICENSE
================================================
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
APPENDIX: How to apply the Apache License to your work.
To apply the Apache License to your work, attach the following
boilerplate notice, with the fields enclosed by brackets "[]"
replaced with your own identifying information. (Don't include
the brackets!) The text should be enclosed in the appropriate
comment syntax for the file format. We also recommend that a
file or class name and description of purpose be included on the
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright 2019 MemSQL (https://www.memsql.com)
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
================================================
FILE: README.md
================================================
# SingleStoreDB Spark Connector
## Version: 5.0.1 [](http://www.apache.org/licenses/LICENSE-2.0.txt)
## Getting Started
You can find the latest version of the connector on Maven Central and
spark-packages.org. The group is `com.singlestore` and the artifact is
`singlestore-spark-connector_2.11` for Spark 2 and `singlestore-spark-connector_2.12` for Spark 3.
* [Maven Central (Spark 2)](https://search.maven.org/artifact/com.singlestore/singlestore-spark-connector_2.11)
* [Maven Central (Spark 3)](https://search.maven.org/artifact/com.singlestore/singlestore-spark-connector_2.12)
* [spark-packages.org](https://spark-packages.org/package/memsql/memsql-spark-connector)
You can add the connector to your Spark application using: spark-shell, pyspark, or spark-submit
```
$SPARK_HOME/bin/spark-shell --packages com.singlestore:singlestore-spark-connector_2.12:5.0.1-spark-4.0.0
```
We release multiple versions of the `singlestore-spark-connector`, one for each supported Spark version.
The connector follows the `x.x.x-spark-y.y.y` naming convention, where `x.x.x` represents the connector version
and `y.y.y` represents the corresponding Spark version.
For example, in connector `5.0.1-spark-4.0.0`, 4.1.11 is the version of the connector,
compiled and tested against Spark version 4.0.0.
It is critical to select the connector version that corresponds to the Spark version in use.
## Configuration
The `singlestore-spark-connector` is configurable globally via Spark options and
locally when constructing a DataFrame. The options are named the same, however
global options have the prefix `spark.datasource.singlestore.`.
#### Basic options
| Option | Default value | Description
| - | - | -
| `ddlEndpoint` (On-Premise deployment) (required) | - | The hostname or IP address of the SingleStoreDB Master Aggregator in the `host[:port]` format, where port is an optional parameter. Example: `master-agg.foo.internal:3308` or `master-agg.foo.internal`.
| `dmlEndpoints` (On-Premise deployment) | ddlEndpoint | The hostname or IP address of SingleStoreDB Aggregator nodes to run queries against in the `host[:port],host[:port],...` format, where :port is an optional parameter (multiple hosts separated by comma). Example: `child-agg:3308,child-agg2`.
| `clientEndpoint` (Cloud deployment) (required) | - | The hostname or IP address to the SingleStoreDB Cloud workspace to run queries against in the format `host[:port]` (port is optional). Ex. `svc-b093ff56-7d9e-499f-b970-7913852facc4-ddl.aws-oregon-2.svc.singlestore.com:3306`
| `user` | `root` | The SingleStoreDB username.
| `password` | - | Password of the SingleStoreDB user.
| `query` | - | The query to run (mutually exclusive with dbtable).
| `dbtable` | - | The table to query (mutually exclusive with query).
| `database` | - | If set, all connections use the specified database by default.
#### Read options
| Option | Default value | Description
|---------------------------------------------------|-------------------------------------------------------| -
| `disablePushdown` | `false` |Disable SQL Pushdown when running queries.
| `enableParallelRead` | `automaticLite` | Enables reading data in parallel for some query shapes. It can have of the following values: `disabled`, `automaticLite`, `automatic`, and `forced`. For more information, see [Parallel Read Support](#parallel-read-support).
| `customSchema` | - | Custom schema to use when reading data. Expects a Spark schema in DDL format. To avoid syntax or escaping errors, you can generate this string programmatically using the `toDDL` method on a Spark `StructType`. When set, the connector uses this schema instead of querying the database for metadata, which improves performance when schema inference is expensive or unnecessary. This option is incompatible with SQL pushdown; `disablePushdown` must be set to `true`.
| `parallelRead.Features` | `ReadFromAggregators,ReadFromAggregatorsMaterialized` | Specifies a comma separated list of parallel read features that are tried in the order they are listed. SingleStore supports the following features: `ReadFromLeaves`, `ReadFromAggregators`, and `ReadFromAggregatorsMaterialized`. Example: `ReadFromAggregators,ReadFromAggregatorsMaterialized`. For more information, see [Parallel Read Support](#parallel-read-support).
| `parallelRead.tableCreationTimeoutMS` | `0` | Specifies the amount of time (in milliseconds) the reader waits for the result table creation when using the `ReadFromAggregators` feature. If set to `0`, timeout is disabled.
| `parallelRead.materializedTableCreationTimeoutMS` | `0` | Specifies the amount of time (in milliseconds) the reader waits for the result table creation when using the `ReadFromAggregatorsMaterialized` feature. If set to `0`, timeout is disabled.
| `parallelRead.numPartitions` | `0` | Specifies the exact number of partitions in the resulting DataFrame. If set to `0`, value is ignored.
| `parallelRead.maxNumPartitions` | `0` | Specifies the Maximum number of partitions in the resulting DataFrame. If set to `0`, no limit is applied.
| `parallelRead.repartition` | `false` | Repartition data before reading.
| `parallelRead.repartition.columns` | `RAND()` | Specifies a comma separated list of columns that are used for repartitioning (when `parallelRead.repartition` is enabled). By default, an additional column with `RAND()` value is used for repartitioning.
#### Write options
| Option | Default value | Description
| - | - | -
| `overwriteBehavior` | `dropAndCreate` | Specifies the behavior during Overwrite. It can have one of the following values: `dropAndCreate`, `truncate`, `merge`.
| `truncate` | `false` | :warning: **This option is deprecated, please use `overwriteBehavior` instead.** Truncates instead of dropping an existing table during Overwrite.
| `loadDataCompression` | `Gzip` | Compresses data on load. It can have one of the following three values: `GZip`, `LZ4`, and `Skip`.
| `loadDataFormat` | `CSV` | Serializes data on load. It can have one of the following values: `Avro` or `CSV`.
| `tableKey` | - | Specifies additional keys to add to tables created by the connector. See [Specifying keys for tables created by the Spark Connector](#specifying-keys-for-tables-created-by-the-spark-connector) for more information.
| `onDuplicateKeySQL` | - | If this option is specified and a new row with duplicate `PRIMARY KEY` or `UNIQUE` index is inserted, SingleStoreDB performs an `UPDATE` operation on the existing row. See [Inserting rows into the table with ON DUPLICATE KEY UPDATE](#inserting-rows-into-the-table-with-on-duplicate-key-update) for more information.
| `insertBatchSize` | `10000` | Specifies the size of the batch for row insertion.
| `maxErrors` | `0` | The maximum number of errors in a single LOAD DATA request. When this limit is reached, the load fails. If this property is set to `0`, no error limit exists.
| `createRowstoreTable` | `rowstore` | If enabled, the connector creates a rowstore table.
#### Connection pool options
| Option | Default value | Description
| - | - | -
| `driverConnectionPool.Enabled` | `true` | Enables using of connection pool on the driver. (default: `true`)
| `driverConnectionPool.MaxOpenConns` | `-1` | The maximum number of active connections with the same options that can be allocated from the driver pool at the same time, or negative for no limit. (default: `-1`)
| `driverConnectionPool.MaxIdleConns` | `8` | The maximum number of connections with the same options that can remain idle in the driver pool, without extra ones being released, or negative for no limit. (default: `8`)
| `driverConnectionPool.MinEvictableIdleTimeMs` | `30000` (30 sec) | The minimum amount of time an object may sit idle in the driver pool before it is eligible for eviction by the idle object evictor (if any). (default: `30000` - 30 sec)
| `driverConnectionPool.TimeBetweenEvictionRunsMS` | `1000` (1 sec) | The number of milliseconds to sleep between runs of the idle object evictor thread on the driver. When non-positive, no idle object evictor thread will be run. (default: `1000` - 1 sec)
| `driverConnectionPool.MaxWaitMS` | `-1` | The maximum number of milliseconds that the driver pool will wait (when there are no available connections) for a connection to be returned before throwing an exception, or `-1` to wait indefinitely. (default: `-1`)
| `driverConnectionPool.MaxConnLifetimeMS` | `-1` | The maximum lifetime in milliseconds of a connection. After this time is exceeded the connection will fail the next activation, passivation, or validation test and won’t be returned by the driver pool. A value of zero or less means the connection has an infinite lifetime. (default: `-1`)
| `executorConnectionPool.Enabled` | `true` | Enables using of connection pool on executors. (default: `true`)
| `executorConnectionPool.MaxOpenConns` | `true` | The maximum number of active connections with the same options that can be allocated from the executor pool at the same time, or negative for no limit. (default: `true`)
| `executorConnectionPool.MaxIdleConns` | `8` | The maximum number of connections with the same options that can remain idle in the executor pool, without extra ones being released, or negative for no limit. (default: `8`)
| `executorConnectionPool.MinEvictableIdleTimeMs` | `2000` | The minimum amount of time an object may sit idle in the executor pool before it is eligible for eviction by the idle object evictor (if any). (default: `2000` - 2 sec)
| `executorConnectionPool.TimeBetweenEvictionRunsMS` | `1000` | The number of milliseconds to sleep between runs of the idle object evictor thread on the executor. When non-positive, no idle object evictor thread will be run. (default: `1000` - 1 sec)
| `executorConnectionPool.MaxWaitMS` | `-1` | The maximum number of milliseconds that the executor pool will wait (when there are no available connections) for a connection to be returned before throwing an exception, or `-1` to wait indefinitely. (default: `-1`)
| `executorConnectionPool.MaxConnLifetimeMS` | `-1` | The maximum lifetime in milliseconds of a connection. After this time is exceeded the connection will fail the next activation, passivation, or validation test and won’t be returned by the executor pool. A value of zero or less means the connection has an infinite lifetime. (default: `-1`)
## Examples
### Configure `singlestore-spark-connector` for SingleStoreDB Cloud
The following example configures the `singlestore-spark-connector` globally:
```scala
spark.conf.set("spark.datasource.singlestore.clientEndpoint", "singlestore-host")
spark.conf.set("spark.datasource.singlestore.user", "admin")
spark.conf.set("spark.datasource.singlestore.password", "s3cur3-pa$$word")
```
The following example configures the `singlestore-spark-connector` using the read API:
```scala
val df = spark.read
.format("singlestore")
.option("clientEndpoint", "singlestore-host")
.option("user", "admin")
.load("foo")
```
The following example configures the `singlestore-spark-connector` using an external table in Spark SQL:
```sql
CREATE TABLE bar USING singlestore OPTIONS ('clientEndpoint'='singlestore-host','dbtable'='foo.bar')
```
> note: `singlestore-spark-connector`doesn't support writing to the reference table for SingleStoreDB Cloud
> note: `singlestore-spark-connector`doesn't support read-only databases for SingleStoreDB Cloud
### Configure `singlestore-spark-connector` for SingleStoreDB On-Premises
The following example configures the `singlestore-spark-connector` globally:
```scala
spark.conf.set("spark.datasource.singlestore.ddlEndpoint", "singlestore-master.cluster.internal")
spark.conf.set("spark.datasource.singlestore.dmlEndpoints", "singlestore-master.cluster.internal,singlestore-child-1.cluster.internal:3307")
spark.conf.set("spark.datasource.singlestore.user", "admin")
spark.conf.set("spark.datasource.singlestore.password", "s3cur3-pa$$word")
```
The following example configures the `singlestore-spark-connector` using the read API:
```scala
val df = spark.read
.format("singlestore")
.option("ddlEndpoint", "singlestore-master.cluster.internal")
.option("user", "admin")
.load("foo")
```
The following example configures the `singlestore-spark-connector` using an external table in Spark SQL:
```sql
CREATE TABLE bar USING singlestore OPTIONS ('ddlEndpoint'='singlestore-master.cluster.internal','dbtable'='foo.bar')
```
For Java/Python versions of some of these examples, visit the section ["Java & Python Example"](#java-python-example)
## Passing Parameters to the SingleStore JDBC Driver
The SingleStore Spark Connector uses the SingleStore JDBC Driver under the hood. Any configuration options provided to the connector that do not belong to the SingleStore Spark connector specifically (and are not described in the [Configuration](#configuration) section above) are treated as JDBC parameters and are passed directly to the underlying JDBC driver.
You can find a comprehensive list of the available JDBC driver parameters in the [SingleStore JDBC Driver documentation](https://docs.singlestore.com/cloud/developer-resources/connect-with-application-development-tools/connect-with-java-jdbc/the-singlestore-jdbc-driver/).
### Examples
**Setting JDBC parameters globally:**
You can set JDBC parameters globally across your Spark session by using the `spark.datasource.singlestore.` prefix, similar to standard connector options.
```scala
// 'connectTimeout' and 'createDatabaseIfNotExist' are JDBC parameters, not Spark connector options
spark.conf.set("spark.datasource.singlestore.connectTimeout", "10000")
spark.conf.set("spark.datasource.singlestore.createDatabaseIfNotExist", "true")
```
**Setting JDBC parameters using the Read API:**
You can also pass JDBC parameters in the `.option()` method when constructing a DataFrame.
```scala
val df = spark.read
.format("singlestore")
.option("ddlEndpoint", "singlestore-master.cluster.internal")
.option("user", "admin")
.option("connectTimeout", "10000") // This is passed directly to the JDBC driver
.option("createDatabaseIfNotExist", "true") // This is passed directly to the JDBC driver
.load("foo")
```
## Writing to SingleStoreDB
The `singlestore-spark-connector` supports saving dataframes to SingleStoreDB using the Spark write API. Here is a basic example of using this API:
```scala
df.write
.format("singlestore")
.option("loadDataCompression", "LZ4")
.option("overwriteBehavior", "dropAndCreate")
.mode(SaveMode.Overwrite)
.save("foo.bar") // in format: database.table
```
If the target table ("foo" in the example above) does not exist in SingleStoreDB the
`singlestore-spark-connector` will automatically attempt to create the table. If you
specify SaveMode.Overwrite, if the target table already exists, it will be
recreated or truncated before load. Specify `overwriteBehavior = truncate` to truncate rather
than re-create.
### Retrieving the number of written rows from taskMetrics
It is possible to add the listener and get the number of written rows.
```scala
spark.sparkContext.addSparkListener(new SparkListener() {
override def onTaskEnd(taskEnd: SparkListenerTaskEnd) {
println("Task id: " + taskEnd.taskInfo.id.toString)
println("Records written: " + taskEnd.taskMetrics.outputMetrics.recordsWritten.toString)
}
})
df.write.format("singlestore").save("example")
```
### Specifying keys for tables created by the Spark Connector
When creating a table, the `singlestore-spark-connector` will read options prefixed
with `tableKey`. These options must be formatted in a specific way in order to
correctly specify the keys.
> :warning: The default table type is a SingleStoreDB columnstore.
> To create a rowstore table instead, enable the `createRowstoreTable` option.
To explain we will refer to the following example:
```scala
df.write
.format("singlestore")
.option("tableKey.primary", "id")
.option("tableKey.key.created_firstname", "created, firstName")
.option("tableKey.unique", "username")
.mode(SaveMode.Overwrite)
.save("foo.bar") // in format: database.table
```
In this example, we are creating three keys:
1. A primary key on the `id` column
2. A regular key on the combination of the `firstname` and `created` columns, with the key name `created_firstname`
3. A unique key on the `username` column
Note on (2): Any key can optionally specify a name, just put it after the key type.
Key names must be unique.
To change the default ColumnStore sort key you can specify it explicitly:
```scala
df.write
.option("tableKey.columnstore", "id")
```
You can also customize the shard key like so:
```scala
df.write
.option("tableKey.shard", "id, timestamp")
```
## Inserting rows into the table with ON DUPLICATE KEY UPDATE
When updating a table it is possible to insert rows with `ON DUPLICATE KEY UPDATE` option.
See [sql reference](https://docs.singlestore.com/db/latest/en/reference/sql-reference/data-manipulation-language-dml/insert.html) for more details.
> :warning: This feature doesn't work for columnstore tables with SingleStoreDB 7.1.
```scala
df.write
.option("onDuplicateKeySQL", "age = age + 1")
.option("insertBatchSize", 300)
.mode(SaveMode.Append)
.save("foo.bar")
```
As a result of the following query, all new rows will be appended without changes.
If a row with the same `PRIMARY KEY` or `UNIQUE` index already exists then the corresponding `age` value will be increased.
When you use ON DUPLICATE KEY UPDATE, all rows of the DataFrame are split into batches, and every insert query will contain no more than the specified `insertBatchSize` rows setting.
## Save Modes
Save operations can optionally take a SaveMode, that specifies how to handle existing data if present.
It is important to realize that these save modes do not utilize any locking and are not atomic.
1. `SaveMode.Append` means that when saving a DataFrame to a data source, if data/table already exists,
contents of the DataFrame are expected to be appended to existing data.
2. `SaveMode.Overwrite` means that when saving a DataFrame to a data source,
if data/table already exists, existing data is expected to be overwritten by the contents of the DataFrame.
> `Overwrite` mode depends on `overwriteBehavior` option, for better understanding look at the section ["Merging on save"](#merging-on-save)
3. `SaveMode.ErrorIfExists` means that when saving a DataFrame to a data source,
if data already exists, an exception is expected to be thrown.
4. `SaveMode.Ignore` means that when saving a DataFrame to a data source, if data already exists,
contents of the DataFrame are expected to be appended to existing data and all rows with duplicate key are ignored.
### Example of `SaveMode` option
```scala
df.write
.mode(SaveMode.Append)
.save("foo.bar")
```
<h2 id="merging-on-save">Merging on save</h2>
When saving dataframes or datasets to SingleStoreDB, you can manage how SaveMode.Overwrite is interpreted by the connector via the option overwriteBehavior.
This option can take one of the following values:
1. `dropAndCreate`(default) - drop and create the table before writing new values.
2. `truncate` - truncate the table before writing new values.
3. `merge` - replace rows with new rows by matching on the primary key.
(Use this option only if you need to fully rewrite existing rows with new ones.
To specify some rule for the update, use the `onDuplicateKeySQL` option instead.)
All these options are case-insensitive.
### Example of `merge` option
Suppose you have the following table, and the `Id` column is the primary key.
`SELECT * FROM <table>;`
| Id | Name | Age |
| ----- |:-------------:| ---:|
| 1 | Alice | 20 |
| 2 | Bob | 25 |
| 3 | Charlie | 30 |
If you save the following dataframe with `overwriteBehavior = merge`:
| Id | Name | Age |
| ----- |:-------------:| ---:|
| 2 | Daniel | 22 |
| 3 | Eve | 27 |
| 4 | Franklin | 35 |
```scala
df.write
.format("singlestore")
.option("overwriteBehavior", "merge")
.mode(SaveMode.Overwrite)
.save("<yourdb>.<table>")
```
After the save is complete, the table will look like this:
> note: rows with Id=2 and Id=3 were overwritten with new rows <br />
> note: the row with Id=1 was not touched and still exists in the result
`SELECT * FROM <table>;`
| Id | Name | Age |
| ----- |:-------------:| ---:|
| 1 | Alice | 20 |
| 2 | Daniel | 22 |
| 3 | Eve | 27 |
| 4 | Franklin | 35 |
## SQL Pushdown
The `singlestore-spark-connector` has extensive support for rewriting Spark SQL
and dataframe operation query plans into standalone SingleStoreDB queries.
This allows most of the computation to be pushed into the SingleStoreDB distributed system
without any manual intervention. The SQL rewrites are enabled automatically,
but they can be disabled using the `disablePushdown` option.
The `singlestore-spark-connector` also support partial pushdown,
where certain parts of a query can be evaluated in SingleStoreDB
and certain parts can be evaluated in Spark.
> :warning: SQL Pushdown is either enabled or disabled on the *entire* Spark
> Session. If you want to run multiple queries in parallel with different
> values of `disablePushdown`, make sure to run them on separate Spark Sessions.
We currently support most of the primary Logical Plan nodes in Spark SQL
including:
* Project
* Filter
* Aggregate
* Window
* Join
* Limit
* Sort
We also support most Spark SQL expressions. A full list of supported
operators/functions can be found in the
[ExpressionGen.scala](src/main/scala/com/singlestore/spark/ExpressionGen.scala) file.
The best place to look for examples of fully supported queries is in the tests.
Check out this file as a starting point:
[SQLPushdownTest.scala](src/test/scala/com/singlestore/spark/SQLPushdownTest.scala).
### Debugging SQL Pushdown
If you encounter an issue with SQL Pushdown the first step is to look at the
explain. You can do this easily from any dataframe using the function
`df.explain()`. If you pass the argument `true` you will get a lot more output
that includes pre and post optimization passes.
In addition, the `singlestore-spark-connector` outputs a lot of helpful information
when the TRACE log level is enabled for the `com.singlestore.spark` package.
To enable TRACE log level, add the following line(s) to the log4j configuration:
- Log4j
```
log4j.logger.com.singlestore.spark=TRACE
```
- Log4j 2
```
logger.singlestore.name = com.singlestore.spark
logger.singlestore.level = TRACE
logger.singlestore.additivity = false
```
Make sure not to leave it in place since it generates a huge amount of tracing
output.
## SQL Pushdown Incompatibilities
* `ToUnixTimestamp` and `UnixTimestamp` only handle time values less than `2038-01-19 03:14:08`, if they get `DateType` or `TimestampType` as a first argument.
* `FromUnixTime` with `yyyy-MM-dd HH:mm:ss` as the default format, only handles time less than `2147483648` (`2^31`).
* `DecimalType` is truncated on overflow (by default, Spark either throws an exception or returns null).
* `greatest` and `least` return `null` if at least one argument is `null` (in Spark these functions skip nulls).
* When a value can not be converted to numeric or fractional type SingleStoreDB returns 0 (Spark returns `null`).
* `Atanh(x)`, for x ∈ (-∞, -1] ∪ [1, ∞) returns, `null` (Spark returns `NaN`).
* When a string is cast to a numeric type, SingleStoreDB takes the prefix of it which is numeric (Spark returns `null` if the whole string is not numeric).
* When a numeric type is cast to a smaller one (in size), SingleStoreDB truncates it. For example `500` cast to the `Byte` will be `127`.
Note: Spark optimizer can optimize casts for literals and then the behaviour for literals matches custom Spark behaviour.
* When a fractional type is cast to an integral type, SingleStoreDB rounds it to the closest value.
* `Log` returns `null` instead of `NaN`, `Infinity`, `-Infinity`.
* `Round` rounds down if the number to be rounded is followed by 5, and it is `DOUBLE` or `FLOAT` (`DECIMAL` is rounded up).
* `Conv` works differently if the number contains non-alphanumeric characters.
* `ShiftLeft`, `ShiftRight`, and `ShiftRightUnsigned` convert the value to an `UNSIGNED BIGINT` and then produce the shift.
In case of an overflow, it returns `0` (`1<<64` = `0` and `10>>20` = `0`).
* `BitwiseGet` returns 0 when the bit position is negative or exceeds the bit upper limit.
* `Initcap` defines a letter as the beginning of a word even if it is enclosed in quotation marks, brackets, etc. For example "dear sir/madam (miss)" is converted to "Dear Sir/Madam (Miss)".
* `Skewness(x)`, in Spark 3.0, for `STD(x) = 0` returns `null` instead of `NaN`.
## Parallel Read Support
Parallel read can be enabled using `enableParallelRead` option. This can drastically improve performance in some cases.
The `enableParallelRead` option can have one of the following values:
* `disabled`: Disables parallel reads and performs non-parallel reads.
* `automaticLite`: Performs parallel reads if at least one parallel read feature specified in `parallelRead.Features` is supported.
Otherwise performs a non-parallel read. In `automaticLite` mode, after push down of the outer sorting operation
(for example, a nested `SELECT` statement where sorting is done in a top-level `SELECT`) into SingleStoreDB is done, a non-parallel read is used.
* `automatic`: Performs parallel reads if at least one parallel read feature specified in `parallelRead.Features` is supported.
Otherwise performs a non-parallel read. In `automatic` mode, the `singlestore-spark-connector` is unable to push down an outer sorting operation into SingleStore.
Final sorting is done at the Spark end of the operation.
* `forced`: Performs parallel reads if at least one parallel read feature specified in `parallelRead.Features` is supported.
Otherwise it returns an error. In `forced` mode, the `singlestore-spark-connector` is unable to push down an outer sorting operation into SingleStore.
Final sorting is done at the Spark end of the operation.
By default, `enableParallelRead` is set to `automaticLite`.
### Parallel read features
The SingleStoreDB Spark Connector supports the following parallel read features:
* `readFromAggregators`
* `readFromAggregatorsMaterialized`
* `readFromLeaves`
The connector uses the first feature specified in `parallelRead.Features` which meets all the requirements.
The requirements for each feature are specified below.
By default, the connector uses the `readFromAggregators` feature.
You can repartition the result set for `readFromAggregators` and `readFromAggregatorsMaterialized` features.
See [Parallel Read Repartitioning](#parallel-read-repartitioning) for more information.
#### readFromAggregators
When this feature is used, the `singlestore-spark-connector` will use [SingleStoreDB parallel read functionality](https://docs.singlestore.com/db/latest/en/query-data/query-procedures/read-query-results-in-parallel.html).
By default, the number of partitions in the resulting DataFrame is the least of the number of partitions in the SingleStoreDB database and Spark parallelism level
(i.e., sum of `(spark.executor.cores/spark.task.cpus)` for all executors).
Number of partitions in the resulting DataFrame can be controlled using `parallelRead.maxNumPartitions` and `parallelRead.numPartitions` options.
To use this feature, all reading queries must start at the same time.
Connector tries to retrieve maximum number of tasks that can be run concurrently and uses this value to distribute reading queries.
In some cases, connector is not able to retrieve this value (for example, with AWS Glue). In such cases, `parallelRead.numPartitions` option is required.
Use the `parallelRead.tableCreationTimeoutMS` option to specify a timeout for result table creation.
Requirements:
* SingleStoreDB version 7.5+
* Either the `database` option is set, or the database name is specified in the `load` option
* SingleStoreDB parallel read functionality supports the generated query
* `parallelRead.numPartitioins` option is set, or connector is able to compute maximum number of concurrent tasks that can be run on Spark cluster
#### readFromAggregatorsMaterialized
This feature is very similar to `readFromAggregators`. The only difference is that `readFromAggregatorsMaterialized` uses the
`MATERIALIZED` option to create the result table. When this feature is used, the reading tasks do not have to start at the same time.
Hence, the parallelism level on the Spark cluster does not affect the reading tasks.
Although, using the `MATERIALIZED` option may cause a query to fail if SingleStoreDB does not have enough memory to materialize the result set.
By default, the number of partitions in the resulting DataFrame is equal to the number of partitions in the SingleStoreDB database.
Number of partitions in the resulting DataFrame can be controlled using `parallelRead.maxNumPartitions` and `parallelRead.numPartitions` options.
Use the `parallelRead.materializedTableCreationTimeoutMS` option to specify a timeout for materialized result table creation.
Requirements:
* SingleStoreDB version 7.5+
* Either the `database` option is set, or the database name is specified in the `load` option
* SingleStoreDB parallel read functionality supports the generated query
#### readFromLeaves
When this feature is used, the `singlestore-spark-connector` skips the transaction layer and reads directly from partitions on the leaf nodes.
Hence, each individual read task sees an independent version of the database's distributed state.
If some queries (other than read operation) are run on the database, they may affect the current read operation.
Make sure to take this into account when using `readFromLeaves` feature.
This feature supports only those query-shapes that do not perform any operation on the aggregator and can be pushed down to the leaf nodes.
In order to use `readFromLeaves` feature, the username and password provided to the
`singlestore-spark-connector` must be the same across all nodes in the cluster.
By default, the number of partitions in the resulting DataFrame is equal to the number of partitions in the SingleStoreDB database.
Number of partitions in the resulting DataFrame can be controlled using `parallelRead.maxNumPartitions` and `parallelRead.numPartitions` options.
Requirements:
* Either the `database` option is set, or the database name is specified in the `load` option
* The username and password provided to the `singlestore-spark-connector` must be uniform across all the nodes in the cluster,
because parallel reads require consistent authentication and connectible leaf nodes
* The hostnames and ports listed by `SHOW LEAVES` must be directly connectible from Spark
* The generated query can be pushed down to the leaf nodes
### Parallel read repartitioning
You can repartition the result using `parallelRead.repartition` option for the `readFromAggregators` and `readFromAggregatorsMaterialized` features
to ensure that each task reads approximately the same amount of data.
This option is very useful for queries with top level limit clauses as without repartitioning it is possible that all rows will belong to one partition.
Use the `parallelRead.repartition.columns` option to specify a comma separated list of columns that will be used for repartitioning.
Column names that contain leading or trailing whitespaces or commas must be escaped as:
- Column name must be enclosed in backticks
```
"a" -> "`a`"
```
- Each backtick (`) in the column name must be replaced with two backticks (``)
```
"a`a``" -> "a``a````"
```
By default, repartitioning is done using an additional column with `RAND()` value.
### Example
```scala
spark.read.format("singlestore")
.option("enableParallelRead", "automatic")
.option("parallelRead.Features", "readFromAggregators,readFromLeaves")
.option("parallelRead.repartition", "true")
.option("parallelRead.repartition.columns", "a, b")
.option("parallelRead.TableCreationTimeout", "1000")
.load("db.table")
```
In the following example, connector will check requirements for `readFromAggregators`.
If they are satisfied, it will use this feature.
Otherwise, it will check requirements for `readFromLeaves`.
If they are satisfied, connector will use this feature. Otherwise, it will use non-parallel read.
If the connector uses `readFromAggregators`, it will repartition the result on the SingleStoreDB side before reading it,
and it will fail if creation of the result table will take longer than `1000` milliseconds.
## Running SQL queries
The methods `executeSinglestoreQuery(query: String, variables: Any*)` and `executeSinglestoreQueryDB(db: String, query: String, variables: Any*)`
allow you to run SQL queries on a SingleStoreDB database directly using the existing `SparkSession` object. The method `executeSinglestoreQuery`
uses the database defined in the `SparkContext` object you use. `executeSinglestoreQueryDB` allows you to specify the database that
will be used for querying.
The following examples demonstrate their usage (assuming you already have
initialized `SparkSession` object named `spark`). The methods return `Iterator[org.apache.spark.sql.Row]` object.
```scala
// this imports the implicit class QueryMethods which adds the methods
// executeSinglestoreQuery and executeSinglestoreQueryDB to SparkSession class
import com.singlestore.spark.SQLHelper.QueryMethods
// You can pass an empty database to executeSinglestoreQueryDB to connect to SingleStoreDB without specifying a database.
// This allows you to create a database which is defined in the SparkSession config for example.
spark.executeSinglestoreQueryDB("", "CREATE DATABASE foo")
// the next query can be used if the database field has been specified in spark object
s = spark.executeSinglestoreQuery("CREATE TABLE user(id INT, name VARCHAR(30), status BOOLEAN)")
// you can create another database
spark.executeSinglestoreQuery("CREATE DATABASE bar")
// the database specified as the first argument will override the database set in the SparkSession object
s = spark.executeSinglestoreQueryDB("bar", "CREATE TABLE user(id INT, name VARCHAR(30), status BOOLEAN)")
```
You can pass query parameters to the functions as arguments following `query`. The supported types for parameters are `String, Int, Long, Short, Float, Double, Boolean, Byte, java.sql.Date, java.sql.Timestamp`.
```scala
import com.singlestore.spark.SQLHelper.QueryMethods
var userRows = spark.executeSinglestoreQuery("SELECT id, name FROM USER WHERE id > ? AND status = ? AND name LIKE ?", 10, true, "%at%")
for (row <- userRows) {
println(row.getInt(0), row.getString(1))
}
```
Alternatively, these functions can take `SparkSession` object as the first argument, as in the example below
```scala
import com.singlestore.spark.SQLHelper.{executeSinglestoreQuery, executeSinglestoreQueryDB}
executeSinglestoreQuery(spark, "CREATE DATABASE foo")
var s = executeSinglestoreQueryDB(spark, "foo", "SHOW TABLES")
```
## Security
### SQL Permissions
The [permission matrix](https://docs.singlestore.com/db/latest/en/reference/sql-reference/security-management-commands/permissions-matrix.html)
describes the permissions required to run each command.
To perform any SQL operation through the SingleStore Spark Connector,
you must have the permissions required for that specific operation.
The following matrix describes the minimum permissions required to perform some operations.
> Note: The ALL PRIVILEGES permission allows you to perform any operation.
| Operation | Min. Permission | Alternative Permission |
| ------------------------------- |:------------------------:| ----------------------:|
| `READ` from collection | `SELECT` | `ALL PRIVILEGES` |
| `WRITE` to collection | `SELECT, INSERT` | `ALL PRIVILEGES` |
| `DROP` database or collection | `SELECT, INSERT, DROP` | `ALL PRIVILEGES` |
| `CREATE` database or collection | `SELECT, INSERT, CREATE` | `ALL PRIVILEGES` |
For more information on granting privileges, see [GRANT](https://docs.singlestore.com/db/latest/en/reference/sql-reference/security-management-commands/grant.html)
### Connecting with a Kerberos-authenticated User
You can use the SingleStoreDB Spark Connector with a Kerberized user without any additional configuration.
To use a Kerberized user, you need to configure the connector with the given SingleStoreDB database user that is authenticated with Kerberos
(via the `user` option). Please visit our documentation [here](https://docs.singlestore.com/db/latest/en/security/authentication/kerberos-authentication.html)
to learn about how to configure SingleStoreDB users with Kerberos.
Here is an example of configuring the Spark connector globally with a Kerberized SingleStoreDB user called `krb_user`.
```scala
spark = SparkSession.builder()
.config("spark.datasource.singlestore.user", "krb_user")
.getOrCreate()
```
You do not need to provide a password when configuring a Spark Connector user that is Kerberized.
The connector driver (SingleStoreDB JDBC driver) will be able to authenticate the Kerberos user from the cache by the provided username.
Other than omitting a password with this configuration, using a Kerberized user with the Connector is no different than using a standard user.
Note that if you do provide a password, it will be ignored.
### SSL Support
The SingleStoreDB Spark Connector uses the SingleStoreDB JDBC Driver under the hood and thus
supports SSL configuration out of the box. In order to configure SSL, first
ensure that your SingleStoreDB cluster has SSL configured. Documentation on how to set
this up can be found here:
https://docs.singlestore.com/latest/guides/security/encryption/ssl/
Once you have setup SSL on your server, you can enable SSL via setting the following options:
```scala
spark.conf.set("spark.datasource.singlestore.useSSL", "true")
spark.conf.set("spark.datasource.singlestore.serverSslCert", "PATH/TO/CERT")
```
**Note:** the `serverSslCert` option may be server's certificate in DER form, or the server's
CA certificate. Can be used in one of 3 forms:
* `serverSslCert=/path/to/cert.pem` (full path to certificate)
* `serverSslCert=classpath:relative/cert.pem` (relative to current classpath)
* or as verbatim DER-encoded certificate string `------BEGIN CERTIFICATE-----...`
You may also want to set these additional options depending on your SSL configuration:
```scala
spark.conf.set("spark.datasource.singlestore.trustServerCertificate", "true")
spark.conf.set("spark.datasource.singlestore.disableSslHostnameVerification", "true")
```
See [The SingleStoreDB JDBC Driver](https://docs.singlestore.com/db/latest/en/developer-resources/connect-with-application-development-tools/connect-with-java-jdbc/the-singlestore-jdbc-driver.html#tls-parameters)
for more information.
### JWT authentication
You may authenticate your connection to the SingleStoreDB cluster using the SingleStoreDB Spark connector with a JWT.
To use JWT-based authentication, specify the following parameters:
- `credentialType=JWT`
- `password=<jwt-token>`
Here's a sample configuration that uses JWT-based authentication:
```
SparkConf conf = new SparkConf();
conf.set("spark.datasource.singlestore.ddlEndpoint", "singlestore-master.cluster.internal")
conf.set("spark.datasource.singlestore.dmlEndpoints", "singlestore-master.cluster.internal,singlestore-child-1.cluster.internal:3307")
conf.set("spark.datasource.singlestore.credentialType", "JWT")
conf.set("spark.datasource.singlestore.useSsl", "true")
conf.set("spark.datasource.singlestore.user", "s2user")
conf.set("spark.datasource.singlestore.password", "eyJhbGci.eyJzdWIiOiIxMjM0NTY3.masHf")
```
> note: To authenticate your connection to the SingleStoreDB cluster using the SingleStoreDB Spark connector with a JWT,
> the SingleStoreDB user must connect via SSL and use a JWT for authentication.
>
> See [Create a JWT User](https://docs.singlestore.com/managed-service/en/security/authentication/authenticate-via-jwt.html#create-a-jwt-user-751086) for more information.
See [Authenticate via JWT](https://docs.singlestore.com/managed-service/en/security/authentication/authenticate-via-jwt.html) for more information.
## Filing issues
When filing issues please include as much information as possible as well as any
reproduction steps. It's hard for us to reproduce issues if the problem depends
on specific data in your SingleStoreDB table for example. Whenever possible please try
to construct a minimal reproduction of the problem and include the table
definition and table contents in the issue.
If the issue is related to SQL Pushdown (or you aren't sure) make sure to
include the TRACE output (from the com.singlestore.spark package) or the full explain
of the plan. See the debugging SQL Pushdown section above for more information
on how to do this.
Happy querying!
## Setting up development environment
* install Oracle JDK 8 from this url: https://www.oracle.com/java/technologies/javase/javase-jdk8-downloads.html
* install the community edition of Intellij IDEA from https://www.jetbrains.com/idea/
* clone the repository https://github.com/memsql/singlestore-spark-connector.git
* in Intellij IDEA choose `Configure->Plugins` and install Scala plugin
* in Intellij IDEA run `Import Project` and select path to singlestore-spark-connector `build.sbt` file
* choose `import project from external model` and `sbt`
* in `Project JDK` select `New...->JDK` and choose the path to the installed JDK
* `Finish`
* it will overwrite some files and create build files (which are in .gitignore)
* you may need to remove the `.idea` directory for IDEA to load the project properly
* in Intellij IDEA choose `File->Close Project`
* run `git checkout .` to revert all changes made by Intellij IDEA
* in Intellij IDEA choose `Open` and select path to singlestore-spark-connector
* run `Test Spark 3.0` (it should succeed)
<h2 id="java-python-example">Java & Python Examples</h2>
### Java
#### Configuration
```
SparkConf conf = new SparkConf();
conf.set("spark.datasource.singlestore.ddlEndpoint", "singlestore-master.cluster.internal")
conf.set("spark.datasource.singlestore.dmlEndpoints", "singlestore-master.cluster.internal,singlestore-child-1.cluster.internal:3307")
conf.set("spark.datasource.singlestore.user", "admin")
conf.set("spark.datasource.singlestore.password", "s3cur3-pa$$word")
```
#### Read Data
```
DataFrame df = spark
.read()
.format("singlestore")
.option("ddlEndpoint", "singlestore-master.cluster.internal")
.option("user", "admin")
.load("foo");
```
#### Write Data
```
df.write()
.format("singlestore")
.option("loadDataCompression", "LZ4")
.option("overwriteBehavior", "dropAndCreate")
.mode(SaveMode.Overwrite)
.save("foo.bar")
```
### Python
#### Configuration
```
spark.conf.set("spark.datasource.singlestore.ddlEndpoint", "singlestore-master.cluster.internal")
spark.conf.set("spark.datasource.singlestore.dmlEndpoints", "singlestore-master.cluster.internal,singlestore-child-1.cluster.internal:3307")
spark.conf.set("spark.datasource.singlestore.user", "admin")
spark.conf.set("spark.datasource.singlestore.password", "s3cur3-pa$$word")
```
#### Read Data
```
df = spark \
.read \
.format("singlestore") \
.option("ddlEndpoint", "singlestore-master.cluster.internal") \
.option("user", "admin") \
.load("foo")
```
#### Write Data
```
df.write \
.format("singlestore") \
.option("loadDataCompression", "LZ4") \
.option("overwriteBehavior", "dropAndCreate") \
.mode("overwrite") \
.save("foo.bar")
```
================================================
FILE: build.sbt
================================================
import xerial.sbt.Sonatype._
/*
To run tests or publish with a specific spark version use this java option:
-Dspark.version=3.0.0
*/
val sparkVersion = sys.props.get("spark.version").getOrElse("4.0.0")
val scalaVersionStr = sparkVersion match {
case "3.1.3" => "2.12.12"
case "3.2.4" => "2.12.12"
case "3.3.4" => "2.12.12"
case "3.4.2" => "2.12.12"
case "3.5.0" => "2.12.12"
case "4.0.0" => "2.13.8"
}
val scalaVersionPrefix = scalaVersionStr.substring(0, 4)
val jacksonDatabindVersion = sparkVersion match {
case "3.1.3" => "2.10.0"
case "3.2.4" => "2.12.3"
case "3.3.4" => "2.13.4.2"
case "3.4.2" => "2.14.2"
case "3.5.0" => "2.15.2"
case "4.0.0" => "2.18.2"
}
lazy val root = project
.withId("singlestore-spark-connector")
.in(file("."))
.enablePlugins(BuildInfoPlugin)
.settings(
name := "singlestore-spark-connector",
organization := "com.singlestore",
scalaVersion := scalaVersionStr,
Compile / unmanagedSourceDirectories += (Compile / sourceDirectory).value / (sparkVersion match {
case "3.1.3" => "scala-sparkv3.1"
case "3.2.4" => "scala-sparkv3.2"
case "3.3.4" => "scala-sparkv3.3"
case "3.4.2" => "scala-sparkv3.4"
case "3.5.0" => "scala-sparkv3.5"
case "4.0.0" => "scala-sparkv4.0"
}),
version := s"5.0.1-spark-${sparkVersion}",
licenses += "Apache-2.0" -> url(
"http://opensource.org/licenses/Apache-2.0"
),
resolvers += "Spark Packages Repo" at "https://dl.bintray.com/spark-packages/maven",
libraryDependencies ++= Seq(
// runtime dependencies
"org.apache.spark" %% "spark-core" % sparkVersion % "provided, test",
"org.apache.spark" %% "spark-sql" % sparkVersion % "provided, test",
"org.apache.avro" % "avro" % "1.11.3",
"org.apache.commons" % "commons-dbcp2" % "2.7.0",
"org.scala-lang.modules" %% "scala-java8-compat" % "0.9.0",
"com.singlestore" % "singlestore-jdbc-client" % "1.2.7",
"io.spray" %% "spray-json" % "1.3.5",
"io.netty" % "netty-buffer" % "4.1.70.Final",
"org.apache.commons" % "commons-dbcp2" % "2.9.0",
// test dependencies
"org.mariadb.jdbc" % "mariadb-java-client" % "2.+" % Test,
"org.scalatest" %% "scalatest" % "3.1.0" % Test,
"org.scalacheck" %% "scalacheck" % "1.14.1" % Test,
"org.mockito" %% "mockito-scala" % "2.0.0" % Test,
"com.github.mrpowers" %% "spark-fast-tests" % "1.1.0" % Test,
"com.github.mrpowers" %% "spark-daria" % "1.2.3" % Test
),
dependencyOverrides += "com.fasterxml.jackson.core" % "jackson-databind" % jacksonDatabindVersion,
Test / testOptions += Tests.Argument("-oF"),
Test / fork := true,
buildInfoKeys := Seq[BuildInfoKey](version),
buildInfoPackage := "com.singlestore.spark"
)
credentials += Credentials(
"GnuPG Key ID",
"gpg",
"CDD996495CF08BB2041D86D8D1EB3D14F1CD334F",
"ignored" // this field is ignored; passwords are supplied by pinentry
)
assemblyMergeStrategy in assembly := {
case PathList("META-INF", _*) => MergeStrategy.discard
case _ => MergeStrategy.first
}
publishTo := sonatypePublishToBundle.value
publishMavenStyle := true
sonatypeSessionName := s"[sbt-sonatype] ${name.value} ${version.value}"
sonatypeProjectHosting := Some(GitHubHosting("memsql", "memsql-spark-connector", "carl@memsql.com"))
Test / javaOptions += "--add-opens=java.base/sun.util.calendar=ALL-UNNAMED"
sonatypeCredentialHost := sonatypeCentralHost
================================================
FILE: demo/Dockerfile
================================================
FROM apache/zeppelin:0.9.0
ENV SPARK_VERSION=4.0.0
USER root
RUN wget https://apache.ip-connect.vn.ua/spark/spark-${SPARK_VERSION}/spark-${SPARK_VERSION}-bin-hadoop2.7.tgz
RUN tar xf spark-${SPARK_VERSION}-bin-hadoop2.7.tgz -C /
RUN rm -rf spark-${SPARK_VERSION}-bin-hadoop2.7.tgz
ENV SPARK_HOME=/spark-${SPARK_VERSION}-bin-hadoop2.7
ENV ZEPPELIN_PORT=8082
RUN rm -rf /zeppelin/notebook/*
EXPOSE ${ZEPPELIN_PORT}/tcp
================================================
FILE: demo/README.md
================================================
## singlestore-spark-connector demo
This is Dockerfile which uses the upstream [Zeppelin Image](https://hub.docker.com/r/apache/zeppelin/) as it's base
and has two notebooks with examples of singlestore-spark-connector.
To run this docker with [MemSQL CIAB](https://hub.docker.com/r/memsql/cluster-in-a-box) follow the instructions
* Create a docker network to be able to connect zeppelin and memsql-ciab
```
docker network create zeppelin-ciab-network
```
* Pull memsql-ciab docker image
```
docker pull memsql/cluster-in-a-box
```
* Run and start the SingleStore Cluster in a Box docker container
```
docker run -i --init \
--name singlestore-ciab-for-zeppelin \
-e LICENSE_KEY=[INPUT_YOUR_LICENSE_KEY] \
-e ROOT_PASSWORD=my_password \
-p 3306:3306 -p 8081:8080 \
--net=zeppelin-ciab-network \
memsql/cluster-in-a-box
```
```
docker start singlestore-ciab-for-zeppelin
```
> :note: in this step you can hit a port collision error
>
> ```
> docker: Error response from daemon: driver failed programming external connectivity on endpoint singlestore-ciab-for-zeppelin
> (38b0df3496f1ec83f120242a53a7023d8a0b74db67f5e487fb23641983c67a76):
> Bind for 0.0.0.0:8080 failed: port is already allocated.
> ERRO[0000] error waiting for container: context canceled
> ```
>
> If it happened then remove the container
>
>`docker rm singlestore-ciab-for-zeppelin`
>
> and run the first command with other ports `-p {new_port1}:3306 -p {new_port2}:8080`
* Build zeppelin docker image in `singlestore-spark-connector/demo` folder
```
docker build -t zeppelin .
```
* Run zeppelin docker container
```
docker run -d --init \
--name zeppelin \
-p 8082:8082 \
--net=zeppelin-ciab-network \
-v $PWD/notebook:/opt/zeppelin/notebook/singlestore \
-v $PWD/notebook:/zeppelin/notebook/singlestore \
zeppelin
```
> :note: in this step you can hit a port collision error
>
> ```
> docker: Error response from daemon: driver failed programming external connectivity on endpoint zeppelin
> (38b0df3496f1ec83f120242a53a7023d8a0b74db67f5e487fb23641983c67a76):
> Bind for 0.0.0.0:8082 failed: port is already allocated.
> ERRO[0000] error waiting for container: context canceled
> ```
>
> If it happened then remove the container
>
>`docker rm zeppelin`
>
> and run this command with other port `-p {new_port}:8082`
* open [zeppelin](http://localhost:8082/next) in your browser and try
[scala](http://localhost:8082/next/#/notebook/2F8XQUKFG),
[pyspark](http://localhost:8082/next/#/notebook/2F6Y3APTX)
and [spark sql](http://localhost:8082/next/#/notebook/2F7PZ81H6) notebooks
For setting up more powerful SingleStore trial cluster use [SingleStore Managed Service](https://www.singlestore.com/managed-service/)
================================================
FILE: demo/notebook/pyspark-singlestore-demo_2F8XQUKFG.zpln
================================================
{
"paragraphs": [
{
"text": "%md\n## This is a small demo that illustrates the usage of the SingleStore-Spark connector. \n#### It connects to the ciab docker container (https://hub.docker.com/r/singlestore/cluster-in-a-box) and runs some basic queries on it.",
"user": "anonymous",
"dateUpdated": "2021-09-23 10:50:07.311",
"progress": 0,
"config": {
"editorSetting": {
"language": "markdown",
"editOnDblClick": true,
"completionKey": "TAB",
"completionSupport": false
},
"colWidth": 12.0,
"editorMode": "ace/mode/markdown",
"fontSize": 15.0,
"editorHide": true,
"results": {},
"enabled": true,
"tableHide": false
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": [
{
"type": "HTML",
"data": "\u003cdiv class\u003d\"markdown-body\"\u003e\n\u003ch2\u003eThis is a small demo that illustrates the usage of the SingleStore-Spark connector.\u003c/h2\u003e\n\u003ch4\u003eIt connects to the ciab docker container (\u003ca href\u003d\"https://hub.docker.com/r/singlestore/cluster-in-a-box\"\u003ehttps://hub.docker.com/r/singlestore/cluster-in-a-box\u003c/a\u003e) and runs some basic queries on it.\u003c/h4\u003e\n\n\u003c/div\u003e"
}
]
},
"apps": [],
"runtimeInfos": {},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587553845758_1676143899",
"id": "paragraph_1587550420891_1388924274",
"dateCreated": "2020-04-22 11:10:45.758",
"dateStarted": "2021-09-23 10:50:07.317",
"dateFinished": "2021-09-23 10:50:07.331",
"status": "FINISHED"
},
{
"title": "Configure Spark",
"text": "%spark.conf\n\n// Comma-separated list of Maven coordinates of jars to include on the driver and executor classpaths\nspark.jars.packages com.singlestore:singlestore-spark-connector_2.12:5.0.1-spark-4.0.0\n\n// The hostname or IP address of the SingleStore Master Aggregator in the `host[:port]` format, where port is an optional parameter\n// singlestore-ciab-for-zeppelin - hostname of the docker created by https://hub.docker.com/r/singlestore/cluster-in-a-box\n// 3306 - port on which SingleStore Master Aggregator is started\nspark.datasource.singlestore.ddlEndpoint singlestore-ciab-for-zeppelin:3306\n\n// The hostname or IP address of SingleStore Aggregator nodes to run queries against in the `host[:port],host[:port],...` format, \n// where :port is an optional parameter (multiple hosts separated by comma) (default: ddlEndpoint)\n// Example\n// spark.datasource.singlestore.dmlEndpoints child-agg:3308,child-agg2\nspark.datasource.singlestore.dmlEndpoints singlestore-ciab-for-zeppelin:3306\n\n// SingleStore username (default: root)\nspark.datasource.singlestore.user root\n\n// SingleStore password (default: no password)\nspark.datasource.singlestore.password my_password",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:26:15.232",
"progress": 0,
"config": {
"lineNumbers": false,
"tableHide": false,
"editorSetting": {
"language": "text",
"editOnDblClick": false,
"completionKey": "TAB",
"completionSupport": true
},
"colWidth": 12.0,
"editorMode": "ace/mode/text",
"fontSize": 13.0,
"title": true,
"results": {},
"enabled": true
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": []
},
"apps": [],
"runtimeInfos": {},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587553845761_760134801",
"id": "paragraph_1587546884632_-2089202077",
"dateCreated": "2020-04-22 11:10:45.761",
"dateStarted": "2022-07-06 11:26:15.237",
"dateFinished": "2022-07-06 11:26:15.245",
"status": "FINISHED"
},
{
"title": "Create a database using SQLHelpers",
"text": "import com.singlestore.spark.SQLHelper.QueryMethods\n\nspark.executeSinglestoreQuery(\"create database if not exists demoDB\")",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:32:33.191",
"progress": 0,
"config": {
"lineNumbers": true,
"tableHide": false,
"editorSetting": {},
"colWidth": 12.0,
"fontSize": 13.0,
"editorHide": false,
"title": true,
"results": {},
"enabled": true
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": [
{
"type": "TEXT",
"data": "import com.singlestore.spark.SQLHelper.QueryMethods\n\u001b[1m\u001b[34mres1\u001b[0m: \u001b[1m\u001b[32mIterator[org.apache.spark.sql.Row]\u001b[0m \u003d \u003citerator\u003e\n"
}
]
},
"apps": [],
"runtimeInfos": {},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1657106390307_996443657",
"id": "paragraph_1657106390307_996443657",
"dateCreated": "2022-07-06 11:19:50.307",
"dateStarted": "2022-07-06 11:26:22.057",
"dateFinished": "2022-07-06 11:27:04.138",
"status": "FINISHED"
},
{
"title": "Writing to SingleStore",
"text": "%spark.pyspark\n\npeople1 \u003d spark.createDataFrame([\n (1, \"andy\", 5, \"USA\"), \n (2, \"jeff\", 23, \"China\"), \n (3, \"james\", 62, \"USA\")\n ]).toDF(\"id\", \"name\", \"age\", \"country\")\npeople1.printSchema\npeople1.show()\n\npeople1.write \\\n .format(\"singlestore\") \\\n .mode(\"overwrite\") \\\n .save(\"demoDB.people\") # write to table `people` in database `demoDB`\n \npeople2 \u003d people1.withColumn(\"age2\", people1[\"age\"] + 1)\npeople1.printSchema\npeople2.show()\n\npeople2.write \\\n .format(\"singlestore\") \\\n .option(\"loadDataCompression\", \"LZ4\") \\\n .mode(\"overwrite\") \\\n .save(\"demoDB.people\") # write to table `people` in database `demoDB` ",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:27:10.614",
"progress": 0,
"config": {
"lineNumbers": true,
"tableHide": false,
"editorSetting": {
"language": "python",
"editOnDblClick": false,
"completionKey": "TAB",
"completionSupport": true
},
"colWidth": 6.0,
"editorMode": "ace/mode/python",
"fontSize": 13.0,
"editorHide": false,
"title": true,
"results": {},
"enabled": true
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": [
{
"type": "TEXT",
"data": "+---+-----+---+-------+\n| id| name|age|country|\n+---+-----+---+-------+\n| 1| andy| 5| USA|\n| 2| jeff| 23| China|\n| 3|james| 62| USA|\n+---+-----+---+-------+\n\n+---+-----+---+-------+----+\n| id| name|age|country|age2|\n+---+-----+---+-------+----+\n| 1| andy| 5| USA| 6|\n| 2| jeff| 23| China| 24|\n| 3|james| 62| USA| 63|\n+---+-----+---+-------+----+\n\n"
}
]
},
"apps": [],
"runtimeInfos": {
"jobUrl": {
"propertyName": "jobUrl",
"label": "SPARK JOB",
"tooltip": "View in Spark web UI",
"group": "spark",
"values": [
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d0"
},
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d1"
},
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d2"
},
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d3"
},
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d4"
},
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d5"
},
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d6"
},
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d7"
}
],
"interpreterSettingId": "spark"
}
},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587553845761_-1027258033",
"id": "paragraph_1587547555609_-348809680",
"dateCreated": "2020-04-22 11:10:45.761",
"dateStarted": "2022-07-06 11:27:10.622",
"dateFinished": "2022-07-06 11:27:22.543",
"status": "FINISHED"
},
{
"title": "Reading from SingleStore",
"text": "%spark.pyspark\n\npeople \u003d spark.read \\\n .format(\"singlestore\") \\\n .load(\"demoDB.people\")\npeople.printSchema\npeople.show()\n\nchildren \u003d spark.read \\\n .format(\"singlestore\") \\\n .load(\"demoDB.people\") \\\n .filter(\"age \u003c 10\")\npeople.printSchema\nchildren.show()",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:27:30.331",
"progress": 0,
"config": {
"tableHide": false,
"editorSetting": {
"language": "python",
"editOnDblClick": false,
"completionKey": "TAB",
"completionSupport": true
},
"colWidth": 6.0,
"editorMode": "ace/mode/python",
"fontSize": 13.0,
"title": true,
"results": {},
"enabled": true
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": [
{
"type": "TEXT",
"data": "+---+-----+---+-------+----+\n| id| name|age|country|age2|\n+---+-----+---+-------+----+\n| 3|james| 62| USA| 63|\n| 1| andy| 5| USA| 6|\n| 2| jeff| 23| China| 24|\n+---+-----+---+-------+----+\n\n+---+----+---+-------+----+\n| id|name|age|country|age2|\n+---+----+---+-------+----+\n| 1|andy| 5| USA| 6|\n+---+----+---+-------+----+\n\n"
}
]
},
"apps": [],
"runtimeInfos": {
"jobUrl": {
"propertyName": "jobUrl",
"label": "SPARK JOB",
"tooltip": "View in Spark web UI",
"group": "spark",
"values": [
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d8"
},
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d9"
}
],
"interpreterSettingId": "spark"
}
},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587553845762_-1342936354",
"id": "paragraph_1587548897148_-478225566",
"dateCreated": "2020-04-22 11:10:45.762",
"dateStarted": "2022-07-06 11:27:30.333",
"dateFinished": "2022-07-06 11:27:33.067",
"status": "FINISHED"
}
],
"name": "pyspark-singlestore-demo",
"id": "2F8XQUKFG",
"defaultInterpreterGroup": "spark",
"version": "0.9.0-preview1",
"noteParams": {},
"noteForms": {},
"angularObjects": {},
"config": {
"isZeppelinNotebookCronEnable": false
},
"info": {}
}
================================================
FILE: demo/notebook/scala-singlestore-demo_2F6Y3APTX.zpln
================================================
{
"paragraphs": [
{
"text": "%md\n## This is a small demo that illustrates the usage of the SingleStore-Spark connector. \n#### It connects to the ciab docker container (https://hub.docker.com/r/singlestore/cluster-in-a-box) and runs some basic queries on it.",
"user": "anonymous",
"dateUpdated": "2021-09-23 10:55:20.428",
"progress": 0,
"config": {
"editorSetting": {
"language": "markdown",
"editOnDblClick": true,
"completionKey": "TAB",
"completionSupport": false
},
"colWidth": 12.0,
"editorMode": "ace/mode/markdown",
"fontSize": 15.0,
"results": {},
"enabled": true,
"editorHide": true,
"tableHide": false
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": [
{
"type": "HTML",
"data": "\u003cdiv class\u003d\"markdown-body\"\u003e\n\u003ch2\u003eThis is a small demo that illustrates the usage of the SingleStore-Spark connector.\u003c/h2\u003e\n\u003ch4\u003eIt connects to the ciab docker container (\u003ca href\u003d\"https://hub.docker.com/r/singlestore/cluster-in-a-box\"\u003ehttps://hub.docker.com/r/singlestore/cluster-in-a-box\u003c/a\u003e) and runs some basic queries on it.\u003c/h4\u003e\n\n\u003c/div\u003e"
}
]
},
"apps": [],
"runtimeInfos": {},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587553015155_-1751456317",
"id": "paragraph_1587550420891_1388924274",
"dateCreated": "2020-04-22 10:56:55.155",
"dateStarted": "2021-09-23 10:55:20.434",
"dateFinished": "2021-09-23 10:55:20.452",
"status": "FINISHED"
},
{
"title": "Configure Spark",
"text": "%spark.conf\n\n// Comma-separated list of Maven coordinates of jars to include on the driver and executor classpaths\nspark.jars.packages com.singlestore:singlestore-spark-connector_2.12:5.0.1-spark-4.0.0\n\n// The hostname or IP address of the SingleStore Master Aggregator in the `host[:port]` format, where port is an optional parameter\n// singlestore-ciab-for-zeppelin - hostname of the docker created by https://hub.docker.com/r/singlestore/cluster-in-a-box\n// 3306 - port on which SingleStore Master Aggregator is started\nspark.datasource.singlestore.ddlEndpoint singlestore-ciab-for-zeppelin:3306\n\n// The hostname or IP address of SingleStore Aggregator nodes to run queries against in the `host[:port],host[:port],...` format, \n// where :port is an optional parameter (multiple hosts separated by comma) (default: ddlEndpoint)\n// Example\n// spark.datasource.singlestore.dmlEndpoints child-agg:3308,child-agg2\nspark.datasource.singlestore.dmlEndpoints singlestore-ciab-for-zeppelin:3306\n\n// SingleStore username (default: root)\nspark.datasource.singlestore.user root\n\n// SingleStore password (default: no password)\nspark.datasource.singlestore.password my_password",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:31:08.311",
"progress": 0,
"config": {
"lineNumbers": false,
"tableHide": false,
"editorSetting": {
"language": "text",
"editOnDblClick": false,
"completionKey": "TAB",
"completionSupport": true
},
"colWidth": 12.0,
"editorMode": "ace/mode/text",
"fontSize": 13.0,
"title": true,
"results": {},
"enabled": true
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": []
},
"apps": [],
"runtimeInfos": {},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587553015155_499760817",
"id": "paragraph_1587546884632_-2089202077",
"dateCreated": "2020-04-22 10:56:55.155",
"dateStarted": "2022-07-06 11:31:08.315",
"dateFinished": "2022-07-06 11:31:08.323",
"status": "FINISHED"
},
{
"title": "Create a database using SQLHelpers",
"text": "import com.singlestore.spark.SQLHelper.QueryMethods\n\nspark.executeSinglestoreQuery(\"create database if not exists demoDB\")",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:31:11.146",
"progress": 0,
"config": {
"colWidth": 12.0,
"fontSize": 13.0,
"enabled": true,
"results": {},
"editorSetting": {
"language": "scala",
"editOnDblClick": false,
"completionKey": "TAB",
"completionSupport": true
},
"editorMode": "ace/mode/scala",
"title": true
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": [
{
"type": "TEXT",
"data": "import com.singlestore.spark.SQLHelper.QueryMethods\n\u001b[1m\u001b[34mres2\u001b[0m: \u001b[1m\u001b[32mIterator[org.apache.spark.sql.Row]\u001b[0m \u003d \u003citerator\u003e\n"
}
]
},
"apps": [],
"runtimeInfos": {},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587581984336_-994182625",
"id": "paragraph_1587581984336_-994182625",
"dateCreated": "2020-04-22 18:59:44.336",
"dateStarted": "2022-07-06 11:31:11.153",
"dateFinished": "2022-07-06 11:31:11.475",
"status": "FINISHED"
},
{
"title": "Writing to SingleStore",
"text": "import org.apache.spark.sql.{SaveMode}\n\nval people1 \u003d spark.createDataFrame(Seq(\n (1, \"andy\", 5, \"USA\"), \n (2, \"jeff\", 23, \"China\"), \n (3, \"james\", 62, \"USA\")\n )).toDF(\"id\", \"name\", \"age\", \"country\")\npeople1.show()\n\npeople1.write\n .format(\"singlestore\")\n .mode(SaveMode.Overwrite) // recreate table if it exists\n .save(\"demoDB.people\") // write to table `people` in database `demoDB`\n \nval people2 \u003d people1.withColumn(\"age2\", $\"age\" + 1)\npeople2.show()\n\npeople2.write\n .format(\"singlestore\")\n .option(\"loadDataCompression\", \"LZ4\") // compress data on load with LZ4\n .mode(SaveMode.Overwrite) // recreate table if it exists\n .save(\"demoDB.people\") // write to table `people` in database `demoDB` ",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:31:13.348",
"progress": 100,
"config": {
"lineNumbers": true,
"tableHide": false,
"editorSetting": {
"language": "scala",
"editOnDblClick": false,
"completionKey": "TAB",
"completionSupport": true
},
"colWidth": 6.0,
"editorMode": "ace/mode/scala",
"fontSize": 13.0,
"editorHide": false,
"title": true,
"results": {},
"enabled": true
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": [
{
"type": "TEXT",
"data": "+---+-----+---+-------+\n| id| name|age|country|\n+---+-----+---+-------+\n| 1| andy| 5| USA|\n| 2| jeff| 23| China|\n| 3|james| 62| USA|\n+---+-----+---+-------+\n\n+---+-----+---+-------+----+\n| id| name|age|country|age2|\n+---+-----+---+-------+----+\n| 1| andy| 5| USA| 6|\n| 2| jeff| 23| China| 24|\n| 3|james| 62| USA| 63|\n+---+-----+---+-------+----+\n\nimport org.apache.spark.sql.SaveMode\n\u001b[1m\u001b[34mpeople1\u001b[0m: \u001b[1m\u001b[32morg.apache.spark.sql.DataFrame\u001b[0m \u003d [id: int, name: string ... 2 more fields]\n\u001b[1m\u001b[34mpeople2\u001b[0m: \u001b[1m\u001b[32morg.apache.spark.sql.DataFrame\u001b[0m \u003d [id: int, name: string ... 3 more fields]\n"
}
]
},
"apps": [],
"runtimeInfos": {
"jobUrl": {
"propertyName": "jobUrl",
"label": "SPARK JOB",
"tooltip": "View in Spark web UI",
"group": "spark",
"values": [
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d10"
},
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d11"
}
],
"interpreterSettingId": "spark"
}
},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587553015156_498470796",
"id": "paragraph_1587547555609_-348809680",
"dateCreated": "2020-04-22 10:56:55.156",
"dateStarted": "2022-07-06 11:31:13.357",
"dateFinished": "2022-07-06 11:31:15.846",
"status": "FINISHED"
},
{
"title": "Reading from SingleStore",
"text": "val people \u003d spark.read\n .format(\"singlestore\")\n .load(\"demoDB.people\")\npeople.show()\n\nval children \u003d spark.read\n .format(\"singlestore\")\n .load(\"demoDB.people\")\n .filter($\"age\" \u003c 10)\nchildren.show()",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:31:17.899",
"progress": 6,
"config": {
"tableHide": false,
"editorSetting": {
"language": "scala",
"editOnDblClick": false,
"completionKey": "TAB",
"completionSupport": true
},
"colWidth": 6.0,
"editorMode": "ace/mode/scala",
"fontSize": 13.0,
"title": true,
"results": {},
"enabled": true
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": [
{
"type": "TEXT",
"data": "+---+-----+---+-------+----+\n| id| name|age|country|age2|\n+---+-----+---+-------+----+\n| 2| jeff| 23| China| 24|\n| 3|james| 62| USA| 63|\n| 1| andy| 5| USA| 6|\n+---+-----+---+-------+----+\n\n+---+----+---+-------+----+\n| id|name|age|country|age2|\n+---+----+---+-------+----+\n| 1|andy| 5| USA| 6|\n+---+----+---+-------+----+\n\n\u001b[1m\u001b[34mpeople\u001b[0m: \u001b[1m\u001b[32morg.apache.spark.sql.DataFrame\u001b[0m \u003d [id: int, name: string ... 3 more fields]\n\u001b[1m\u001b[34mchildren\u001b[0m: \u001b[1m\u001b[32morg.apache.spark.sql.Dataset[org.apache.spark.sql.Row]\u001b[0m \u003d [id: int, name: string ... 3 more fields]\n"
}
]
},
"apps": [],
"runtimeInfos": {
"jobUrl": {
"propertyName": "jobUrl",
"label": "SPARK JOB",
"tooltip": "View in Spark web UI",
"group": "spark",
"values": [
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d12"
},
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d13"
}
],
"interpreterSettingId": "spark"
}
},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587553015156_-836094162",
"id": "paragraph_1587548897148_-478225566",
"dateCreated": "2020-04-22 10:56:55.156",
"dateStarted": "2022-07-06 11:31:17.906",
"dateFinished": "2022-07-06 11:31:20.525",
"status": "FINISHED"
},
{
"text": "",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:31:28.014",
"progress": 0,
"config": {
"colWidth": 12.0,
"fontSize": 9.0,
"results": {},
"enabled": true,
"editorSetting": {}
},
"settings": {
"params": {},
"forms": {}
},
"apps": [],
"runtimeInfos": {},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1657107077904_308239458",
"id": "paragraph_1657107077904_308239458",
"dateCreated": "2022-07-06 11:31:17.904",
"status": "READY"
}
],
"name": "scala-singlestore-demo",
"id": "2F6Y3APTX",
"defaultInterpreterGroup": "spark",
"version": "0.9.0-preview1",
"noteParams": {},
"noteForms": {},
"angularObjects": {},
"config": {
"isZeppelinNotebookCronEnable": false
},
"info": {}
}
================================================
FILE: demo/notebook/spark-sql-singlestore-demo_2F7PZ81H6.zpln
================================================
{
"paragraphs": [
{
"text": "%md\n## This is a small demo that illustrates the usage of the SingleStore-Spark connector. \n#### It connects to the ciab docker container (https://hub.docker.com/r/singlestore/cluster-in-a-box) and runs some basic queries on it.",
"user": "anonymous",
"dateUpdated": "2021-09-23 11:00:50.089",
"progress": 0,
"config": {
"colWidth": 12.0,
"fontSize": 13.0,
"enabled": true,
"results": {},
"editorSetting": {
"language": "markdown",
"editOnDblClick": true,
"completionKey": "TAB",
"completionSupport": false
},
"editorMode": "ace/mode/markdown",
"editorHide": true,
"tableHide": false
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": [
{
"type": "HTML",
"data": "\u003cdiv class\u003d\"markdown-body\"\u003e\n\u003ch2\u003eThis is a small demo that illustrates the usage of the SingleStore-Spark connector.\u003c/h2\u003e\n\u003ch4\u003eIt connects to the ciab docker container (\u003ca href\u003d\"https://hub.docker.com/r/singlestore/cluster-in-a-box\"\u003ehttps://hub.docker.com/r/singlestore/cluster-in-a-box\u003c/a\u003e) and runs some basic queries on it.\u003c/h4\u003e\n\n\u003c/div\u003e"
}
]
},
"apps": [],
"runtimeInfos": {},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587645478876_994795668",
"id": "paragraph_1587645478876_994795668",
"dateCreated": "2020-04-23 12:37:58.876",
"dateStarted": "2021-09-23 11:00:50.095",
"dateFinished": "2021-09-23 11:00:50.108",
"status": "FINISHED"
},
{
"title": "Configure Spark",
"text": "%spark.conf\n\n// Comma-separated list of Maven coordinates of jars to include on the driver and executor classpaths\nspark.jars.packages com.singlestore:singlestore-spark-connector_2.12:5.0.1-spark-4.0.0\n\n// The hostname or IP address of the SingleStore Master Aggregator in the `host[:port]` format, where port is an optional parameter\n// singlestore-ciab-for-zeppelin - hostname of the docker created by https://hub.docker.com/r/singlestore/cluster-in-a-box\n// 3306 - port on which SingleStore Master Aggregator is started\nspark.datasource.singlestore.ddlEndpoint singlestore-ciab-for-zeppelin:3306\n\n// The hostname or IP address of SingleStore Aggregator nodes to run queries against in the `host[:port],host[:port],...` format, \n// where :port is an optional parameter (multiple hosts separated by comma) (default: ddlEndpoint)\n// Example\n// spark.datasource.singlestore.dmlEndpoints child-agg:3308,child-agg2\nspark.datasource.singlestore.dmlEndpoints singlestore-ciab-for-zeppelin:3306\n\n// SingleStore username (default: root)\nspark.datasource.singlestore.user root\n\n// SingleStore password (default: no password)\nspark.datasource.singlestore.password my_password",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:32:22.885",
"progress": 0,
"config": {
"colWidth": 12.0,
"fontSize": 13.0,
"enabled": true,
"results": {},
"editorSetting": {
"language": "text",
"editOnDblClick": false,
"completionKey": "TAB",
"completionSupport": true
},
"editorMode": "ace/mode/text",
"title": true
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": []
},
"apps": [],
"runtimeInfos": {},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587645537976_1278581633",
"id": "paragraph_1587645537976_1278581633",
"dateCreated": "2020-04-23 12:38:57.976",
"dateStarted": "2022-07-06 11:32:22.890",
"dateFinished": "2022-07-06 11:32:22.898",
"status": "FINISHED"
},
{
"title": "Create a database using SQLHelpers",
"text": "import com.singlestore.spark.SQLHelper.QueryMethods\n\nspark.executeSinglestoreQuery(\"create database if not exists demoDB\")",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:32:43.948",
"progress": 0,
"config": {
"colWidth": 6.0,
"fontSize": 13.0,
"enabled": true,
"results": {},
"editorSetting": {
"language": "scala",
"editOnDblClick": false,
"completionKey": "TAB",
"completionSupport": true
},
"editorMode": "ace/mode/scala",
"title": true
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": [
{
"type": "TEXT",
"data": "import com.singlestore.spark.SQLHelper.QueryMethods\n\u001b[1m\u001b[34mres5\u001b[0m: \u001b[1m\u001b[32mIterator[org.apache.spark.sql.Row]\u001b[0m \u003d \u003citerator\u003e\n"
}
]
},
"apps": [],
"runtimeInfos": {},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587645590204_-1422944272",
"id": "paragraph_1587645590204_-1422944272",
"dateCreated": "2020-04-23 12:39:50.204",
"dateStarted": "2022-07-06 11:32:43.951",
"dateFinished": "2022-07-06 11:32:44.203",
"status": "FINISHED"
},
{
"title": "Insert data to the table",
"text": "import org.apache.spark.sql.{SaveMode}\n\nval people1 \u003d spark.createDataFrame(Seq(\n (1, \"andy\", 5, \"USA\"), \n (2, \"jeff\", 23, \"China\"), \n (3, \"james\", 62, \"USA\"),\n (4, \"clara\", 82, \"USA\"),\n (5, \"emmy\", 9, \"Canada\")\n )).toDF(\"id\", \"name\", \"age\", \"country\")\npeople1.show()\n\npeople1.write\n .format(\"singlestore\")\n .mode(SaveMode.Overwrite) // recreate table if it exists\n .save(\"demoDB.people\") // write to table `people` in database `demoDB`\n",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:32:46.800",
"progress": 0,
"config": {
"colWidth": 6.0,
"fontSize": 13.0,
"enabled": true,
"results": {},
"editorSetting": {
"language": "scala",
"editOnDblClick": false,
"completionKey": "TAB",
"completionSupport": true
},
"editorMode": "ace/mode/scala",
"title": true
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": [
{
"type": "TEXT",
"data": "+---+-----+---+-------+\n| id| name|age|country|\n+---+-----+---+-------+\n| 1| andy| 5| USA|\n| 2| jeff| 23| China|\n| 3|james| 62| USA|\n| 4|clara| 82| USA|\n| 5| emmy| 9| Canada|\n+---+-----+---+-------+\n\nimport org.apache.spark.sql.SaveMode\n\u001b[1m\u001b[34mpeople1\u001b[0m: \u001b[1m\u001b[32morg.apache.spark.sql.DataFrame\u001b[0m \u003d [id: int, name: string ... 2 more fields]\n"
}
]
},
"apps": [],
"runtimeInfos": {
"jobUrl": {
"propertyName": "jobUrl",
"label": "SPARK JOB",
"tooltip": "View in Spark web UI",
"group": "spark",
"values": [
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d14"
}
],
"interpreterSettingId": "spark"
}
},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587646718045_-758896661",
"id": "paragraph_1587646718045_-758896661",
"dateCreated": "2020-04-23 12:58:38.045",
"dateStarted": "2022-07-06 11:32:46.807",
"dateFinished": "2022-07-06 11:32:47.382",
"status": "FINISHED"
},
{
"title": "Create a Spark SQL table backed by a SingleStore table",
"text": "%sql\nCREATE DATABASE IF NOT EXISTS demoDB;\nCREATE TABLE IF NOT EXISTS demoDB.people USING singlestore OPTIONS (\u0027dbtable\u0027\u003d\u0027demoDB.people\u0027);",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:33:29.031",
"progress": 0,
"config": {
"colWidth": 6.0,
"fontSize": 13.0,
"enabled": true,
"results": {},
"editorSetting": {
"language": "sql",
"editOnDblClick": false,
"completionKey": "TAB",
"completionSupport": true
},
"editorMode": "ace/mode/sql",
"title": true
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": []
},
"apps": [],
"runtimeInfos": {},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587645780167_-228404075",
"id": "paragraph_1587645780167_-228404075",
"dateCreated": "2020-04-23 12:43:00.167",
"dateStarted": "2022-07-06 11:33:29.037",
"dateFinished": "2022-07-06 11:33:29.285",
"status": "FINISHED"
},
{
"title": "Run a raw Spark SQL query over a Spark SQL table",
"text": "%sql\nSELECT * FROM demoDB.people;\nSELECT * FROM demoDB.people WHERE people.age \u003c 10;\nSELECT country, MAX(age) FROM demoDB.people GROUP BY country;",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:33:31.802",
"progress": 56,
"config": {
"colWidth": 6.0,
"fontSize": 13.0,
"enabled": true,
"results": {
"0": {
"graph": {
"mode": "table",
"height": 300.0,
"optionOpen": false,
"setting": {
"table": {
"tableGridState": {},
"tableColumnTypeState": {
"names": {
"id": "string",
"name": "string",
"age": "string",
"country": "string"
},
"updated": false
},
"tableOptionSpecHash": "[{\"name\":\"useFilter\",\"valueType\":\"boolean\",\"defaultValue\":false,\"widget\":\"checkbox\",\"description\":\"Enable filter for columns\"},{\"name\":\"showPagination\",\"valueType\":\"boolean\",\"defaultValue\":false,\"widget\":\"checkbox\",\"description\":\"Enable pagination for better navigation\"},{\"name\":\"showAggregationFooter\",\"valueType\":\"boolean\",\"defaultValue\":false,\"widget\":\"checkbox\",\"description\":\"Enable a footer for displaying aggregated values\"}]",
"tableOptionValue": {
"useFilter": false,
"showPagination": false,
"showAggregationFooter": false
},
"updated": false,
"initialized": false
}
},
"commonSetting": {}
}
},
"1": {
"graph": {
"mode": "table",
"height": 300.0,
"optionOpen": false,
"setting": {
"table": {
"tableGridState": {},
"tableColumnTypeState": {
"names": {
"id": "string",
"name": "string",
"age": "string",
"country": "string"
},
"updated": false
},
"tableOptionSpecHash": "[{\"name\":\"useFilter\",\"valueType\":\"boolean\",\"defaultValue\":false,\"widget\":\"checkbox\",\"description\":\"Enable filter for columns\"},{\"name\":\"showPagination\",\"valueType\":\"boolean\",\"defaultValue\":false,\"widget\":\"checkbox\",\"description\":\"Enable pagination for better navigation\"},{\"name\":\"showAggregationFooter\",\"valueType\":\"boolean\",\"defaultValue\":false,\"widget\":\"checkbox\",\"description\":\"Enable a footer for displaying aggregated values\"}]",
"tableOptionValue": {
"useFilter": false,
"showPagination": false,
"showAggregationFooter": false
},
"updated": false,
"initialized": false
}
},
"commonSetting": {}
}
},
"2": {
"graph": {
"mode": "table",
"height": 300.0,
"optionOpen": false,
"setting": {
"table": {
"tableGridState": {},
"tableColumnTypeState": {
"names": {
"country": "string",
"max(age)": "string"
},
"updated": false
},
"tableOptionSpecHash": "[{\"name\":\"useFilter\",\"valueType\":\"boolean\",\"defaultValue\":false,\"widget\":\"checkbox\",\"description\":\"Enable filter for columns\"},{\"name\":\"showPagination\",\"valueType\":\"boolean\",\"defaultValue\":false,\"widget\":\"checkbox\",\"description\":\"Enable pagination for better navigation\"},{\"name\":\"showAggregationFooter\",\"valueType\":\"boolean\",\"defaultValue\":false,\"widget\":\"checkbox\",\"description\":\"Enable a footer for displaying aggregated values\"}]",
"tableOptionValue": {
"useFilter": false,
"showPagination": false,
"showAggregationFooter": false
},
"updated": false,
"initialized": false
}
},
"commonSetting": {}
}
}
},
"editorSetting": {
"language": "sql",
"editOnDblClick": false,
"completionKey": "TAB",
"completionSupport": true
},
"editorMode": "ace/mode/sql",
"title": true
},
"settings": {
"params": {},
"forms": {}
},
"results": {
"code": "SUCCESS",
"msg": [
{
"type": "TABLE",
"data": "id\tname\tage\tcountry\n2\tjeff\t23\tChina\n4\tclara\t82\tUSA\n3\tjames\t62\tUSA\n5\temmy\t9\tCanada\n1\tandy\t5\tUSA\n"
},
{
"type": "TABLE",
"data": "id\tname\tage\tcountry\n5\temmy\t9\tCanada\n1\tandy\t5\tUSA\n"
},
{
"type": "TABLE",
"data": "country\tmax(age)\nChina\t23\nCanada\t9\nUSA\t82\n"
}
]
},
"apps": [],
"runtimeInfos": {
"jobUrl": {
"propertyName": "jobUrl",
"label": "SPARK JOB",
"tooltip": "View in Spark web UI",
"group": "spark",
"values": [
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d15"
},
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d16"
},
{
"jobUrl": "http://322bfd970e79:4040/jobs/job?id\u003d17"
}
],
"interpreterSettingId": "spark"
}
},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1587646114445_960936248",
"id": "paragraph_1587646114445_960936248",
"dateCreated": "2020-04-23 12:48:34.446",
"dateStarted": "2022-07-06 11:33:31.809",
"dateFinished": "2022-07-06 11:33:34.428",
"status": "FINISHED"
},
{
"text": "%sql\n",
"user": "anonymous",
"dateUpdated": "2022-07-06 11:33:31.807",
"progress": 0,
"config": {},
"settings": {
"params": {},
"forms": {}
},
"apps": [],
"runtimeInfos": {},
"progressUpdateIntervalMs": 500,
"jobName": "paragraph_1657107211807_874039426",
"id": "paragraph_1657107211807_874039426",
"dateCreated": "2022-07-06 11:33:31.807",
"status": "READY"
}
],
"name": "spark-sql-singlestore-demo",
"id": "2F7PZ81H6",
"defaultInterpreterGroup": "spark",
"version": "0.9.0-preview1",
"noteParams": {},
"noteForms": {},
"angularObjects": {},
"config": {
"isZeppelinNotebookCronEnable": false
},
"info": {}
}
================================================
FILE: project/build.properties
================================================
sbt.version=1.3.8
================================================
FILE: project/plugins.sbt
================================================
addSbtPlugin("org.xerial.sbt" % "sbt-sonatype" % "3.12.2")
addSbtPlugin("com.jsuereth" % "sbt-pgp" % "2.0.1")
addSbtPlugin("com.eed3si9n" % "sbt-buildinfo" % "0.11.0")
addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "2.2.0")
================================================
FILE: scripts/jwt/jwt_auth_config.json
================================================
{
"username_claim": "username",
"methods": [
{
"algorithms": [ "RS384" ],
"secret": "-----BEGIN PUBLIC KEY-----\nMIICIjANBgkqhkiG9w0BAQEFAAOCAg8AMIICCgKCAgEA0i0dDauX6iaOaocic99O\nUTruTYPWFUv50aHTgfKxenFKpJTTL43T8ON36whwyObM3r/ayhPoyPxvSfkkCwxd\nE7XEmTRARHkJKQfebkbN6SaKlEgIdmZ8UroZCslSzOcsX0N1KNc3WSyFeOigHSp/\nww+roVtaJC/OJQ95kMjIGdN3ooO5g/YvZJZTn9KQ/dFmNDPaSyseT9/MCE2Rp0g0\nT2yCwewxVdfR+D4QcicaLat7CAFXMnoSxV9ifGXYkv6JE33dc95U4BgPYECca2QA\nNe3ZQHSNxC1rc+uim3cgcn6PP4WKTgTG4u74F9xA8FbumZUIMB7rChsr+E8Z4/Iq\nSGb7/y0J6Auho7BDJPL7ZFE9peuSA3NudZlpkH+GIWRW7fBY7qu1Koh8kGfZeg4k\nLIucgT0CpSvrxaDVqRSEIiqy4zoczrLXcVJN9ThtxVPJZCVTW2dir8aCwO9Sk60W\nQ8DP4wRWhnqa+Irsd/2r5c7QgSSOSo4/EIBU65qP5oA6xUw3F4sE7G+Q9ofijxxJ\n8iGr2Y7SBk5ztvSYAX/ZNfoUJZQ0cXDCSCKus8a9kQQ9zCNBLco9pIv7XUcJf1bj\nzfz1o911OZn/mdcgqUq0uRne8x73J/Z4uIsaVubvcSgv0XTSF8qYgpgRd016+nAa\nFtyVd9Y5xmYgIIHaXzQ3l7MCAwEAAQ==\n-----END PUBLIC KEY-----"
}
]
}
================================================
FILE: scripts/setup-cluster.sh
================================================
set -eu
# this script must be run from the top-level of the repo
cd "$(git rev-parse --show-toplevel)"
DEFAULT_SINGLESTORE_VERSION=""
VERSION="${SINGLESTORE_VERSION:-$DEFAULT_SINGLESTORE_VERSION}"
IMAGE_NAME="ghcr.io/singlestore-labs/singlestoredb-dev:0.2.51"
CONTAINER_NAME="singlestore-integration"
SSL_DIR="${PWD}/scripts/ssl"
rm -rf "${SSL_DIR}"
mkdir -p "${SSL_DIR}"
echo "Create a Certificate Authority (CA)"
openssl genpkey -algorithm RSA -out "${SSL_DIR}"/ca-key.pem
openssl req -new -x509 -key "${SSL_DIR}"/ca-key.pem -out "${SSL_DIR}"/ca-cert.pem -days 365 -subj "/CN=SingleStoreDBCA"
echo "Generate the Server Certificate"
openssl genpkey -algorithm RSA -out "${SSL_DIR}"/server-key.pem
openssl req -new -key "${SSL_DIR}"/server-key.pem -out "${SSL_DIR}"/server-req.csr -subj "/CN=singlestore-server"
openssl x509 -req -in "${SSL_DIR}"/server-req.csr -CA "${SSL_DIR}"/ca-cert.pem -CAkey "${SSL_DIR}"/ca-key.pem -CAcreateserial -out "${SSL_DIR}"/server-cert.pem -days 365
echo "Create truststore"
keytool -import -trustcacerts -file "${SSL_DIR}"/ca-cert.pem -keystore "${SSL_DIR}"/truststore.jks -storepass password -alias singlestore-ca -noprompt
echo "Generate the Client Certificate"
openssl genpkey -algorithm RSA -out "${SSL_DIR}"/client-key.pem
openssl req -new -key "${SSL_DIR}"/client-key.pem -out "${SSL_DIR}"/client-req.csr -subj "/CN=singlestore-client"
openssl x509 -req -in "${SSL_DIR}"/client-req.csr -CA "${SSL_DIR}"/ca-cert.pem -CAkey "${SSL_DIR}"/ca-key.pem -CAcreateserial -out "${SSL_DIR}"/client-cert.pem -days 365
echo "Create keystore"
openssl pkcs12 -export -inkey "${SSL_DIR}"/client-key.pem -in "${SSL_DIR}"/client-cert.pem -out "${SSL_DIR}"/client-keystore.p12 -name client-cert -CAfile "${SSL_DIR}"/ca-cert.pem -caname root -passout pass:password
chmod -R 777 "${SSL_DIR}"
EXISTS=$(docker inspect ${CONTAINER_NAME} >/dev/null 2>&1 && echo 1 || echo 0)
if [[ "${EXISTS}" -eq 1 ]]; then
EXISTING_IMAGE_NAME=$(docker inspect -f '{{.Config.Image}}' ${CONTAINER_NAME})
if [[ "${IMAGE_NAME}" != "${EXISTING_IMAGE_NAME}" ]]; then
echo "Existing container ${CONTAINER_NAME} has image ${EXISTING_IMAGE_NAME} when ${IMAGE_NAME} is expected; recreating container."
docker rm -f ${CONTAINER_NAME}
EXISTS=0
fi
fi
if [[ "${EXISTS}" -eq 0 ]]; then
docker run -d \
--name ${CONTAINER_NAME} \
-v ${PWD}/scripts/ssl:/test-ssl \
-v ${PWD}/scripts/jwt:/test-jwt \
-e SINGLESTORE_LICENSE=${SINGLESTORE_LICENSE} \
-e ROOT_PASSWORD=${ROOT_PASSWORD} \
-e SINGLESTORE_VERSION=${VERSION} \
-p 5506:3306 -p 5507:3307 -p 5508:3308 \
${IMAGE_NAME}
fi
singlestore-wait-start() {
echo -n "Waiting for SingleStore to start..."
while true; do
if mysql -u root -h 127.0.0.1 -P 5506 -p"${ROOT_PASSWORD}" -e "select 1" >/dev/null 2>/dev/null; then
break
fi
echo -n "."
sleep 0.2
done
echo ". Success!"
}
singlestore-wait-start
if [[ "${EXISTS}" -eq 0 ]]; then
echo
echo "Creating aggregator node"
docker exec ${CONTAINER_NAME} memsqlctl create-node --yes --password ${ROOT_PASSWORD} --port 3308
docker exec ${CONTAINER_NAME} memsqlctl update-config --yes --all --key minimum_core_count --value 0
docker exec ${CONTAINER_NAME} memsqlctl update-config --yes --all --key minimum_memory_mb --value 0
docker exec ${CONTAINER_NAME} memsqlctl start-node --yes --all
docker exec ${CONTAINER_NAME} memsqlctl add-aggregator --yes --host 127.0.0.1 --password ${ROOT_PASSWORD} --port 3308
fi
echo
echo "Setting up SSL"
docker exec ${CONTAINER_NAME} memsqlctl update-config --yes --all --key ssl_ca --value /test-ssl/ca-cert.pem
docker exec ${CONTAINER_NAME} memsqlctl update-config --yes --all --key ssl_cert --value /test-ssl/server-cert.pem
docker exec ${CONTAINER_NAME} memsqlctl update-config --yes --all --key ssl_key --value /test-ssl/server-key.pem
echo "Setting up JWT"
docker exec ${CONTAINER_NAME} memsqlctl update-config --yes --all --key jwt_auth_config_file --value /test-jwt/jwt_auth_config.json
echo "Restarting cluster"
docker restart ${CONTAINER_NAME}
singlestore-wait-start
echo "Setting up root-ssl user"
mysql -u root -h 127.0.0.1 -P 5506 -p"${ROOT_PASSWORD}" -e 'create user "root-ssl"@"%" require ssl'
mysql -u root -h 127.0.0.1 -P 5506 -p"${ROOT_PASSWORD}" -e 'grant all privileges on *.* to "root-ssl"@"%" require ssl with grant option'
mysql -u root -h 127.0.0.1 -P 5507 -p"${ROOT_PASSWORD}" -e 'create user "root-ssl"@"%" require ssl'
mysql -u root -h 127.0.0.1 -P 5507 -p"${ROOT_PASSWORD}" -e 'grant all privileges on *.* to "root-ssl"@"%" require ssl with grant option'
mysql -u root -h 127.0.0.1 -P 5508 -p"${ROOT_PASSWORD}" -e 'grant all privileges on *.* to "root-ssl"@"%" require ssl with grant option'
echo "Done!"
echo "Setting up root-jwt user"
mysql -h 127.0.0.1 -u root -P 5506 -p"${ROOT_PASSWORD}" -e "CREATE USER 'test_jwt_user' IDENTIFIED WITH authentication_jwt"
mysql -h 127.0.0.1 -u root -P 5506 -p"${ROOT_PASSWORD}" -e "GRANT ALL PRIVILEGES ON *.* TO 'test_jwt_user'@'%'"
echo "Done!"
sleep 0.5
echo
echo "Ensuring child nodes are connected using container IP"
CONTAINER_IP=$(docker inspect -f '{{range .NetworkSettings.Networks}}{{.IPAddress}}{{end}}' ${CONTAINER_NAME})
CURRENT_LEAF_IP=$(mysql -u root -h 127.0.0.1 -P 5506 -p"${ROOT_PASSWORD}" --batch -N -e 'select host from information_schema.leaves')
if [[ ${CONTAINER_IP} != "${CURRENT_LEAF_IP}" ]]; then
# remove leaf with current ip
mysql -u root -h 127.0.0.1 -P 5506 -p"${ROOT_PASSWORD}" --batch -N -e "remove leaf '${CURRENT_LEAF_IP}':3307"
# add leaf with correct ip
mysql -u root -h 127.0.0.1 -P 5506 -p"${ROOT_PASSWORD}" --batch -N -e "add leaf root:'${ROOT_PASSWORD}'@'${CONTAINER_IP}':3307"
fi
CURRENT_AGG_IP=$(mysql -u root -h 127.0.0.1 -P 5506 -p"${ROOT_PASSWORD}" --batch -N -e 'select host from information_schema.aggregators where master_aggregator=0')
if [[ ${CONTAINER_IP} != "${CURRENT_AGG_IP}" ]]; then
# remove aggregator with current ip
mysql -u root -h 127.0.0.1 -P 5506 -p"${ROOT_PASSWORD}" --batch -N -e "remove aggregator '${CURRENT_AGG_IP}':3308"
# add aggregator with correct ip
mysql -u root -h 127.0.0.1 -P 5506 -p"${ROOT_PASSWORD}" --batch -N -e "add aggregator root:'${ROOT_PASSWORD}'@'${CONTAINER_IP}':3308"
fi
echo "Done!"
================================================
FILE: src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
================================================
com.singlestore.spark.DefaultSource
com.memsql.spark.DefaultSource
================================================
FILE: src/main/scala/com/memsql/spark/DefaultSource.scala
================================================
package com.memsql.spark
import com.singlestore.spark
class DefaultSource extends spark.DefaultSource {
override def shortName(): String = spark.DefaultSource.MEMSQL_SOURCE_NAME_SHORT
}
================================================
FILE: src/main/scala/com/singlestore/spark/AggregatorParallelReadListener.scala
================================================
package com.singlestore.spark
import java.sql.{Connection, SQLException}
import java.util.Properties
import com.singlestore.spark.JdbcHelpers.getDDLConnProperties
import com.singlestore.spark.SQLGen.VariableList
import org.apache.spark.SparkContext
import org.apache.spark.scheduler.{
SparkListener,
SparkListenerStageCompleted,
SparkListenerStageSubmitted
}
import org.apache.spark.sql.types.StructType
import scala.collection.mutable
class AggregatorParallelReadListener(applicationId: String) extends SparkListener with LazyLogging {
// connectionsMap is a map from the result table name to the connection with which this table was created
private val connectionsMap: mutable.Map[String, Connection] =
new mutable.HashMap[String, Connection]()
// rddInfos is a map from RDD id to the info needed to create result table for this RDD
private val rddInfos: mutable.Map[Int, SingleStoreRDDInfo] =
new mutable.HashMap[Int, SingleStoreRDDInfo]()
// SingleStoreRDDInfo is information needed to create a result table
private case class SingleStoreRDDInfo(sc: SparkContext,
query: String,
variables: VariableList,
schema: StructType,
connectionProperties: Properties,
materialized: Boolean,
needsRepartition: Boolean,
repartitionColumns: Seq[String])
def addRDDInfo(rdd: SinglestoreRDD): Unit = {
rddInfos.synchronized({
rddInfos += (rdd.id -> SingleStoreRDDInfo(
rdd.sparkContext,
rdd.query,
rdd.variables,
rdd.schema,
getDDLConnProperties(rdd.options, isOnExecutor = false),
rdd.parallelReadType.contains(ReadFromAggregatorsMaterialized),
rdd.options.parallelReadRepartition,
rdd.parallelReadRepartitionColumns,
))
})
}
def deleteRDDInfo(rdd: SinglestoreRDD): Unit = {
rddInfos.synchronized({
rddInfos -= rdd.id
})
}
def isEmpty: Boolean = {
rddInfos.synchronized({
rddInfos.isEmpty
})
}
override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = {
stageSubmitted.stageInfo.rddInfos.foreach(rddInfo => {
if (rddInfo.name.startsWith("SingleStoreRDD")) {
rddInfos
.synchronized(
rddInfos.get(rddInfo.id)
)
.foreach(singleStoreRDDInfo => {
val stageId = stageSubmitted.stageInfo.stageId
val attemptNumber = stageSubmitted.stageInfo.attemptNumber()
val randHex = rddInfo.name.substring("SingleStoreRDD".size)
val tableName =
JdbcHelpers
.getResultTableName(applicationId, stageId, rddInfo.id, attemptNumber, randHex)
// Create connection and save it in the map
val conn =
SinglestoreConnectionPool.getConnection(singleStoreRDDInfo.connectionProperties)
connectionsMap.synchronized(
connectionsMap += (tableName -> conn)
)
log.info(s"Creating result table '$tableName'")
try {
// Create result table
JdbcHelpers.createResultTable(
conn,
tableName,
singleStoreRDDInfo.query,
singleStoreRDDInfo.schema,
singleStoreRDDInfo.variables,
singleStoreRDDInfo.materialized,
singleStoreRDDInfo.needsRepartition,
singleStoreRDDInfo.repartitionColumns
)
log.info(s"Successfully created result table '$tableName'")
} catch {
// Cancel execution if we failed to create a result table
case e: SQLException => {
singleStoreRDDInfo.sc.cancelStage(stageId)
throw e
}
}
})
}
})
}
override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = {
stageCompleted.stageInfo.rddInfos.foreach(rddInfo => {
if (rddInfo.name.startsWith("SingleStoreRDD")) {
val stageId = stageCompleted.stageInfo.stageId
val attemptNumber = stageCompleted.stageInfo.attemptNumber()
val randHex = rddInfo.name.substring("SingleStoreRDD".size)
val tableName =
JdbcHelpers.getResultTableName(applicationId, stageId, rddInfo.id, attemptNumber, randHex)
connectionsMap.synchronized(
connectionsMap
.get(tableName)
.foreach(conn => {
// Drop result table
log.info(s"Dropping result table '$tableName'")
JdbcHelpers.dropResultTable(conn, tableName)
log.info(s"Successfully dropped result table '$tableName'")
// Close connection
conn.close()
// Delete connection from map
connectionsMap -= tableName
})
)
}
})
}
}
case object AggregatorParallelReadListenerAdder {
// listeners is a map from SparkContext hash code to the listener associated with this SparkContext
private val listeners = new mutable.HashMap[SparkContext, AggregatorParallelReadListener]()
def addRDD(rdd: SinglestoreRDD): Unit = {
this.synchronized({
val listener = listeners.getOrElse(
rdd.sparkContext, {
val newListener = new AggregatorParallelReadListener(rdd.sparkContext.applicationId)
rdd.sparkContext.addSparkListener(newListener)
listeners += (rdd.sparkContext -> newListener)
newListener
}
)
listener.addRDDInfo(rdd)
})
}
def deleteRDD(rdd: SinglestoreRDD): Unit = {
this.synchronized({
listeners
.get(rdd.sparkContext)
.foreach(listener => {
listener.deleteRDDInfo(rdd)
if (listener.isEmpty) {
listeners -= rdd.sparkContext
rdd.sparkContext.removeSparkListener(listener)
}
})
})
}
}
================================================
FILE: src/main/scala/com/singlestore/spark/AvroSchemaHelper.scala
================================================
package com.singlestore.spark
import org.apache.avro.Schema
import org.apache.avro.Schema.Type
import org.apache.avro.Schema.Type._
import scala.collection.JavaConverters._
object AvroSchemaHelper {
def resolveNullableType(avroType: Schema, nullable: Boolean): Schema = {
if (nullable && avroType.getType != NULL) {
// avro uses union to represent nullable type.
val fields = avroType.getTypes.asScala
assert(fields.length == 2)
val actualType = fields.filter(_.getType != Type.NULL)
assert(actualType.length == 1)
actualType.head
} else {
avroType
}
}
}
================================================
FILE: src/main/scala/com/singlestore/spark/CompletionIterator.scala
================================================
package com.singlestore.spark
// Copied from spark's CompletionIterator which is private even though it is generically useful
abstract class CompletionIterator[+A, +I <: Iterator[A]](sub: I) extends Iterator[A] {
private[this] var completed = false
def next(): A = sub.next()
def hasNext: Boolean = {
val r = sub.hasNext
if (!r && !completed) {
completed = true
completion()
}
r
}
def completion(): Unit
}
private[spark] object CompletionIterator {
def apply[A, I <: Iterator[A]](sub: I, completionFunction: => Unit): CompletionIterator[A, I] = {
new CompletionIterator[A, I](sub) {
def completion(): Unit = completionFunction
}
}
}
================================================
FILE: src/main/scala/com/singlestore/spark/DefaultSource.scala
================================================
package com.singlestore.spark
import com.singlestore.spark.SQLGen.SQLGenContext
import org.apache.spark.TaskContext
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.metrics.source.MetricsHandler
import org.apache.spark.sql.sources.{
BaseRelation,
CreatableRelationProvider,
DataSourceRegister,
RelationProvider
}
import org.apache.spark.sql.{DataFrame, Row, SQLContext, SaveMode}
object DefaultSource {
val SINGLESTORE_SOURCE_NAME = "com.singlestore.spark"
val SINGLESTORE_SOURCE_NAME_SHORT = "singlestore"
val SINGLESTORE_GLOBAL_OPTION_PREFIX = "spark.datasource.singlestore."
@Deprecated val MEMSQL_SOURCE_NAME = "com.memsql.spark"
@Deprecated val MEMSQL_SOURCE_NAME_SHORT = "memsql"
@Deprecated val MEMSQL_GLOBAL_OPTION_PREFIX = "spark.datasource.memsql."
}
class DefaultSource
extends RelationProvider
with DataSourceRegister
with CreatableRelationProvider
with LazyLogging {
override def shortName(): String = DefaultSource.SINGLESTORE_SOURCE_NAME_SHORT
private def includeGlobalParams(sqlContext: SQLContext,
params: Map[String, String]): Map[String, String] =
sqlContext.getAllConfs.foldLeft(params)({
case (params, (k, v)) if k.startsWith(DefaultSource.SINGLESTORE_GLOBAL_OPTION_PREFIX) =>
params + (k.stripPrefix(DefaultSource.SINGLESTORE_GLOBAL_OPTION_PREFIX) -> v)
case (params, (k, v)) if k.startsWith(DefaultSource.MEMSQL_GLOBAL_OPTION_PREFIX) =>
params + (k.stripPrefix(DefaultSource.MEMSQL_GLOBAL_OPTION_PREFIX) -> v)
case (params, _) => params
})
override def createRelation(sqlContext: SQLContext,
parameters: Map[String, String]): BaseRelation = {
val params = CaseInsensitiveMap(includeGlobalParams(sqlContext, parameters))
val options = SinglestoreOptions(params, sqlContext.sparkSession.sparkContext)
if (options.disablePushdown) {
SQLPushdownRule.ensureRemoved(sqlContext.sparkSession)
SinglestoreReaderNoPushdown(SinglestoreOptions.getQuery(params), options, sqlContext)
} else {
SQLPushdownRule.ensureInjected(sqlContext.sparkSession)
SinglestoreReader(SinglestoreOptions.getQuery(params),
Nil,
options,
sqlContext,
context = SQLGenContext(options))
}
}
override def createRelation(sqlContext: SQLContext,
mode: SaveMode,
parameters: Map[String, String],
data: DataFrame): BaseRelation = {
val opts = CaseInsensitiveMap(includeGlobalParams(sqlContext, parameters))
val conf = SinglestoreOptions(opts, sqlContext.sparkSession.sparkContext)
val table = SinglestoreOptions
.getTable(opts)
.getOrElse(
throw new IllegalArgumentException(
s"To write a dataframe to SingleStore you must specify a table name via the '${SinglestoreOptions.TABLE_NAME}' parameter"
)
)
JdbcHelpers.prepareTableForWrite(conf, table, mode, data.schema)
val isReferenceTable = JdbcHelpers.isReferenceTable(conf, table)
val partitionWriterFactory =
if (conf.onDuplicateKeySQL.isEmpty) {
new LoadDataWriterFactory(table, conf)
} else {
new BatchInsertWriterFactory(table, conf)
}
val schema = data.schema
var totalRowCount = 0L
data.foreachPartition((partition: Iterator[Row]) => {
val writer = partitionWriterFactory.createDataWriter(schema,
TaskContext.getPartitionId(),
0,
isReferenceTable,
mode)
try {
partition.foreach(record => {
writer.write(record)
totalRowCount += 1
})
writer.commit()
MetricsHandler.setRecordsWritten(totalRowCount)
} catch {
case e: Exception =>
writer.abort(e)
throw e
}
})
createRelation(sqlContext, parameters)
}
}
================================================
FILE: src/main/scala/com/singlestore/spark/ExpressionGen.scala
================================================
package com.singlestore.spark
import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
import scala.reflect.ClassTag
object ExpressionGen extends LazyLogging {
import SQLGen._
final val SINGLESTORE_DECIMAL_MAX_PRECISION = 65
final val SINGLESTORE_DECIMAL_MAX_SCALE = 30
final val SINGLESTORE_DEFAULT_TIME_FORMAT = UTF8String.fromString("yyyy-MM-dd HH:mm:ss")
// DAYS_OF_WEEK_OFFSET_MAP is a map from week day prefix to it's offset (sunday -> 1, saturday -> 7)
final val DAYS_OF_WEEK_OFFSET_MAP: Map[String, String] = {
val daysOfWeek =
List("sunday", "monday", "tuesday", "wednesday", "thursday", "friday", "saturday")
val prefix2 = daysOfWeek.map(day => day.slice(0, 2))
val prefix3 = daysOfWeek.map(day => day.slice(0, 3))
val dayFormats = daysOfWeek ::: prefix3 ::: prefix2
dayFormats.zipWithIndex.map { case (day, ind) => (day, (ind % 7 + 1).toString) }.toMap
}
// helpers to keep this code sane
def f(n: String, c: Joinable*): Statement = func(n, c: _*)
def op(o: String, l: Joinable, r: Joinable): Statement = block(l + o + r)
def ifNeg(value: Joinable, valTrue: Joinable, valFalse: Joinable): Statement =
f("IF", op("<", value, IntVar(0)), valTrue, valFalse)
def makeDecimal(child: Joinable, precision: Int, scale: Int): Joinable = {
val p = Math.min(SINGLESTORE_DECIMAL_MAX_PRECISION, precision)
val s = Math.min(SINGLESTORE_DECIMAL_MAX_SCALE, scale)
cast(child, s"DECIMAL($p, $s)")
}
def addMicroseconds(start: Joinable, v: CalendarInterval): Joinable =
addMicroseconds(start, v.microseconds)
def addMicroseconds(start: Joinable, v: Long): Joinable =
if (v == 0) {
start
} else {
f("DATE_ADD", start, Raw("INTERVAL") + v.toString + "MICROSECOND")
}
def addMonths(start: Joinable, v: CalendarInterval): Joinable =
addMonths(start, v.months)
def addMonths(start: Joinable, v: Int): Joinable =
if (v == 0) {
start
} else {
f("DATE_ADD", start, Raw("INTERVAL") + v.toString + "MONTH")
}
def subMicroseconds(start: Joinable, v: CalendarInterval): Joinable =
if (v.microseconds == 0) {
start
} else {
f("DATE_SUB", start, Raw("INTERVAL") + v.microseconds.toString + "MICROSECOND")
}
def subMonths(start: Joinable, v: CalendarInterval): Joinable =
if (v.months == 0) {
start
} else {
f("DATE_SUB", start, Raw("INTERVAL") + v.months.toString + "MONTH")
}
def longToDecimal(child: Joinable, p: Int, s: Int): Joinable =
makeDecimal(op("/", child, math.pow(10.0, s).toString), p, s)
def like(left: Joinable, right: Joinable): Joinable =
op("LIKE", left, right)
def likePatterns(child: Joinable, patterns: Seq[UTF8String], operation: String): Joinable = {
patterns.foldLeft(op("LIKE", child, s"'${patterns.head.toString}'"))((patternA, patternB) => {
op(operation, patternA, op("LIKE", child, s"'${patternB.toString}'"))
})
}
// regexpFromStart adds a ^ prefix for singlestore regexp to match the beginning of the string (as Java does)
def regexpFromStart(r: Joinable): Joinable = func("CONCAT", StringVar("^"), r)
// computeNextDay returns a statement for computing the next date after startDate with specified offset (sunday -> 1, saturday -> 7)
// ADDDATE(startDate,(dayOfWeekOffset - DAYOFWEEK(startDate) + 6)%7 +1)
def computeNextDay(startDate: Joinable, offset: Joinable): Statement = f(
"ADDDATE",
startDate,
op(
"+",
op(
"%",
op(
"+",
op(
"-",
offset,
f("DAYOFWEEK", startDate)
),
"6"
),
"7"
),
"1"
)
)
object GenLiteral {
def unapply(arg: Expression): Option[Joinable] = arg match {
case Literal(v, _) if v == null => Some(StringVar(null))
case Literal(v: Int, DateType) => Some(DateVar(DateTimeUtils.toJavaDate(v)))
case Literal(v: Long, TimestampType) => Some(TimestampVar(DateTimeUtils.toJavaTimestamp(v)))
case Literal(v, t) if !VersionSpecificUtil.isIntervalType(t) => {
convertLiteralValue.lift(v)
}
case _ => None
}
def unapply(vals: Iterable[Any]): Option[Joinable] =
vals
.map(convertLiteralValue.lift)
.reduce[Option[Joinable]] {
case (Some(left), Some(right)) => Some(left + "," + right)
case _ => None
}
def convertLiteralValue: PartialFunction[Any, Joinable] = {
case v if v == null => StringVar(null)
case v: String => StringVar(v)
case v: UTF8String => StringVar(v.toString)
case v: Byte => ByteVar(v)
case v: Boolean => Raw(if (v) "TRUE" else "FALSE")
case v: Short => Raw(v.toString)
case v: Int => Raw(v.toString)
case v: Integer => Raw(v.toString)
case v: Long => Raw(v.toString)
case v: Decimal => makeDecimal(Raw(v.toString), v.precision, v.scale)
case v: BigDecimal => makeDecimal(Raw(v.toString), v.precision, v.scale)
case v: Float if java.lang.Float.isFinite(v) => Raw(v.toString)
case v: Double if java.lang.Double.isFinite(v) => Raw(v.toString)
}
}
case class FoldableExtractor[T]() {
def unapply(e: Expression)(implicit tag: ClassTag[T]): Option[T] =
if (e.foldable) {
e.eval() match {
case expr: T =>
tag.unapply(expr)
case _ => None
}
} else None
}
case class DecimalExpressionExtractor(expressionExtractor: ExpressionExtractor) {
def unapply(e: Expression): Option[(Joinable, Int, Int)] = (e, e.dataType) match {
case (expressionExtractor(child), t: DecimalType) => Some((child, t.precision, t.scale))
case _ => None
}
}
case class WindowBoundaryExpressionExtractor(expressionExtractor: ExpressionExtractor) {
val versionSpecificWindowBoundaryExpressionExtractor
: VersionSpecificWindowBoundaryExpressionExtractor =
VersionSpecificWindowBoundaryExpressionExtractor(expressionExtractor)
def unapply(arg: Expression): Option[Joinable] = arg match {
case versionSpecificWindowBoundaryExpressionExtractor(statement) => Some(statement)
case e: SpecialFrameBoundary => Some(e.sql)
case Literal(n: Integer, IntegerType) =>
Some(Raw(Math.abs(n).toString) + (if (n < 0) "PRECEDING" else "FOLLOWING"))
case expressionExtractor(child) => Some(child + "FOLLOWING")
case _ => None
}
}
case class AggregateExpressionExtractor(expressionExtractor: ExpressionExtractor,
context: SQLGenContext) {
def unapply(arg: AggregateExpression): Option[Joinable] = {
val filterOption = arg.filter match {
case None => Some(None)
case Some(filter) =>
expressionExtractor
.unapply(filter)
.map(f => Some(f))
}
filterOption.flatMap(filter => {
val versionSpecificAggregateExpressionExtractor =
VersionSpecificAggregateExpressionExtractor(expressionExtractor, context, filter)
arg.aggregateFunction match {
case Count(expression) =>
(expression, arg.isDistinct, filter) match {
case (expressionExtractor(None), false, filter) =>
Some(aggregateWithFilter("COUNT", "1", filter))
case (expressionExtractor(Some(children)), false, filter) =>
Some(aggregateWithFilter("COUNT", children, filter))
// DISTINCT and FILTER can't be used together
case (expressionExtractor(Some(children)), true, None) =>
Some(Raw("COUNT") + block(Raw("DISTINCT") + children))
case _ => None
}
// Covariance.scala
// TODO: case CovPopulation(expressionExtractor(left), expressionExtractor(right)) => ???
// TODO: case CovSample(expressionExtractor(left), expressionExtractor(right)) => ???
// Max.scala
case Max(expressionExtractor(child)) =>
Some(aggregateWithFilter("MAX", child, filter))
// Min.scala
case Min(expressionExtractor(child)) =>
Some(aggregateWithFilter("MIN", child, filter))
// BitAnd.scala
case BitAndAgg(expressionExtractor(child))
if context.singlestoreVersionAtLeast("7.0.1") =>
Some(ExpressionGen.aggregateWithFilter("BIT_AND", child, filter))
// BitOr.scala
case BitOrAgg(expressionExtractor(child)) if context.singlestoreVersionAtLeast("7.0.1") =>
Some(ExpressionGen.aggregateWithFilter("BIT_OR", child, filter))
// BitXor.scala
case BitXorAgg(expressionExtractor(child))
if context.singlestoreVersionAtLeast("7.0.1") =>
Some(ExpressionGen.aggregateWithFilter("BIT_XOR", child, filter))
case versionSpecificAggregateExpressionExtractor(statement) => Some(statement)
// case AggregateExpression(MaxBy(expressionExtractor(valueExpr), expressionExtractor(orderingExpr)), _, _, None, _) =>
// case AggregateExpression(MinBy(expressionExtractor(valueExpr), expressionExtractor(orderingExpr)), _, _, None, _) =>
case _ => None
}
})
}
}
// we need to manually unwrap MonthsBetween since the roundOff argument
// does not exist in Spark 2.3
// The roundOff argument truncates the result to 8 digits of precision
// which we can safely ignore, the user can apply an explicit round if needed
case class MonthsBetweenExpressionExtractor(expressionExtractor: ExpressionExtractor) {
def unapply(arg: MonthsBetween): Option[(Joinable, Joinable)] =
for {
date1 <- expressionExtractor.unapply(arg.date1)
date2 <- expressionExtractor.unapply(arg.date2)
} yield (date1, date2)
}
case class CaseWhenExpressionExtractor(expressionExtractor: ExpressionExtractor) {
def unapply(arg: CaseWhen): Option[Joinable] = {
val condition =
arg.branches.foldLeft(Option(stringToJoinable("")))((prefix: Option[Joinable], branch) => {
prefix match {
case Some(actualPrefix) =>
branch match {
case (expressionExtractor(whenCondition), expressionExtractor(thenCondition)) =>
Some(actualPrefix + Raw("WHEN") + whenCondition + Raw("THEN") + thenCondition)
case _ => None
}
case None => None
}
})
val elseCondition = arg.elseValue match {
case Some(expressionExtractor(e)) => Some(Raw("ELSE") + e)
case None => Some(Raw(""))
case _ => None
}
(condition, elseCondition) match {
case (Some(c), Some(e)) => Some(block(Raw("CASE") + c + e + "END"))
case _ => None
}
}
}
def aggregateWithFilter(funcName: String, child: Joinable, filter: Option[Joinable]) = {
filter match {
case Some(filterExpression) =>
f(funcName, f("IF", filterExpression, child, StringVar(null)))
case None => f(funcName, child)
}
}
val intFoldableExtractor: FoldableExtractor[Int] = FoldableExtractor[Int]()
val utf8StringFoldableExtractor: FoldableExtractor[UTF8String] = FoldableExtractor[UTF8String]()
def apply(expressionExtractor: ExpressionExtractor): PartialFunction[Expression, Joinable] = {
val caseWhenExpressionExtractor = CaseWhenExpressionExtractor(expressionExtractor)
val windowBoundaryExpressionExtractor = WindowBoundaryExpressionExtractor(expressionExtractor)
val monthsBetweenExpressionExtractor = MonthsBetweenExpressionExtractor(expressionExtractor)
val context = expressionExtractor.context
val aggregateExpressionExtractor = AggregateExpressionExtractor(expressionExtractor, context)
val decimalExpressionExtractor = DecimalExpressionExtractor(expressionExtractor)
val versionSpecificExpressionGen = VersionSpecificExpressionGen(expressionExtractor)
return {
// ----------------------------------
// Attributes
// ----------------------------------
case a: Attribute => Attr(a, context)
case a @ Alias(expressionExtractor(child), name) =>
alias(child, name, a.exprId, context)
// ----------------------------------
// Literals
// ----------------------------------
case GenLiteral(v) => v
// ----------------------------------
// Variable Expressions
// ----------------------------------
case Coalesce(expressionExtractor(Some(child))) => f("COALESCE", child)
case Least(expressionExtractor(Some(child))) => f("LEAST", child)
case Greatest(expressionExtractor(Some(child))) => f("GREATEST", child)
case Concat(expressionExtractor(Some(child))) => f("CONCAT", child)
// ----------------------------------
// Aggregate Expressions
// ----------------------------------
case aggregateExpressionExtractor(expression) => expression
// windowExpressions.scala
case WindowExpression(expressionExtractor(child),
WindowSpecDefinition(expressionExtractor(partitionSpec),
expressionExtractor(orderSpec),
expressionExtractor(frameSpec))) =>
child + "OVER" + block(
partitionSpec.map(Raw("PARTITION BY") + _).getOrElse(empty) +
orderSpec.map(Raw("ORDER BY") + _).getOrElse(empty) +
frameSpec
)
case UnspecifiedFrame => ""
case SpecifiedWindowFrame(frameType,
windowBoundaryExpressionExtractor(lower),
windowBoundaryExpressionExtractor(upper)) =>
Raw(frameType.sql) + "BETWEEN" + lower + "AND" + upper
case RowNumber() => "ROW_NUMBER()"
case NTile(expressionExtractor(child)) => f("NTILE", child)
case Rank(_) => "RANK()"
case DenseRank(_) => "DENSE_RANK()"
case PercentRank(_) => "PERCENT_RANK()"
// TODO: case CumeDist() => ???
// ----------------------------------
// Binary Expressions
// ----------------------------------
// bitwiseExpressions.scala
case BitwiseAnd(expressionExtractor(left), expressionExtractor(right)) => op("&", left, right)
case BitwiseOr(expressionExtractor(left), expressionExtractor(right)) => op("|", left, right)
case BitwiseXor(expressionExtractor(left), expressionExtractor(right)) => op("^", left, right)
// datetimeExpressions.scala
// NOTE: we explicitly ignore the timeZoneId field in all of the following expressions
// The user is required to setup Spark and/or SingleStore with the timezone they want or they
// will get inconsistent results with/without pushdown.
case DateAdd(expressionExtractor(startDate), expressionExtractor(days)) =>
f("ADDDATE", startDate, days)
case DateSub(expressionExtractor(startDate), expressionExtractor(days)) =>
f("SUBDATE", startDate, days)
case TimeAdd(expressionExtractor(start),
Literal(v: CalendarInterval, CalendarIntervalType),
timeZoneId) => {
def addDays(start: Joinable) =
if (v.days == 0) {
start
} else {
f("DATE_ADD", start, Raw("INTERVAL") + v.days.toString + "DAY")
}
addMicroseconds(addDays(addMonths(start, v)), v)
}
case FromUTCTimestamp(expressionExtractor(timestamp), expressionExtractor(timezone)) =>
f("CONVERT_TZ", timestamp, StringVar("UTC"), timezone)
case ToUTCTimestamp(expressionExtractor(timestamp), expressionExtractor(timezone)) =>
f("CONVERT_TZ", timestamp, timezone, StringVar("UTC"))
case TruncTimestamp(expressionExtractor(format),
expressionExtractor(timestamp),
timeZoneId) => {
f(
"DATE_TRUNC",
sqlMapValueCaseInsensitive(
format,
Map(
// SingleStore doesn't support formats ("yyyy", "yy", "mon", "mm", "dd") so we map them here
"yyyy" -> "year",
"yy" -> "year",
"mon" -> "month",
"mm" -> "month",
"dd" -> "day"
),
format
),
timestamp
)
}
case TruncDate(expressionExtractor(date), expressionExtractor(format)) => {
f(
"DATE_TRUNC",
sqlMapValueCaseInsensitive(
format,
Map(
// SingleStore doesn't support formats ("yyyy", "yy", "mon", "mm") so we map them here
"yyyy" -> "year",
"yy" -> "year",
"mon" -> "month",
"mm" -> "month"
),
format
),
date
)
}
case monthsBetweenExpressionExtractor((date1, date2)) =>
f("MONTHS_BETWEEN", date1, date2)
case caseWhenExpressionExtractor(caseWhenStatement) => caseWhenStatement
case AddMonths(expressionExtractor(startDate), expressionExtractor(numMonths)) =>
f("DATE_ADD", startDate, Raw("INTERVAL") + numMonths + "MONTH")
// SingleStore and spark support other date formats
// UnixTime doesn't use format if time is already a dataType or TimestampType
case FromUnixTime(expressionExtractor(sec), utf8StringFoldableExtractor(format), timeZoneId)
if format == SINGLESTORE_DEFAULT_TIME_FORMAT =>
f("FROM_UNIXTIME", sec)
case DateDiff(expressionExtractor(endDate), expressionExtractor(startDate)) =>
f("DATEDIFF", endDate, startDate)
// mathExpressions.scala
case Atan2(expressionExtractor(left), expressionExtractor(right)) => f("ATAN2", left, right)
case Pow(expressionExtractor(left), expressionExtractor(right)) => f("POWER", left, right)
case ShiftLeft(expressionExtractor(left), expressionExtractor(right)) => op("<<", left, right)
case ShiftRight(expressionExtractor(left), expressionExtractor(right)) =>
op(">>", left, right)
case ShiftRightUnsigned(expressionExtractor(left), expressionExtractor(right)) =>
op(">>", left, right)
case Logarithm(expressionExtractor(left), expressionExtractor(right)) => f("LOG", left, right)
case Hypot(expressionExtractor(left), expressionExtractor(right)) =>
f("SQRT", op("+", f("POW", left, "2"), f("POW", right, "2")))
// TODO: case _: BRound => None
// predicates.scala
case And(expressionExtractor(left), expressionExtractor(right)) => op("AND", left, right)
case Or(expressionExtractor(left), expressionExtractor(right)) => op("OR", left, right)
case EqualTo(expressionExtractor(left), expressionExtractor(right)) => op("=", left, right)
case EqualNullSafe(expressionExtractor(left), expressionExtractor(right)) =>
op("<=>", left, right)
case LessThan(expressionExtractor(left), expressionExtractor(right)) => op("<", left, right)
case LessThanOrEqual(expressionExtractor(left), expressionExtractor(right)) =>
op("<=", left, right)
case GreaterThan(expressionExtractor(left), expressionExtractor(right)) =>
op(">", left, right)
case GreaterThanOrEqual(expressionExtractor(left), expressionExtractor(right)) =>
op(">=", left, right)
case If(expressionExtractor(predicate),
expressionExtractor(trueValue),
expressionExtractor(falseValue)) =>
f("IF", predicate, trueValue, falseValue)
case In(expressionExtractor(child), expressionExtractor(Some(elements))) =>
op("IN", child, block(elements))
case InSet(expressionExtractor(child), GenLiteral(elements)) =>
op(
"IN",
child,
block(elements)
)
// regexpExpressions.scala
case Like(expressionExtractor(left), expressionExtractor(right), escapeChar: Char) =>
if (escapeChar == '\\') {
op("LIKE", left, right)
} else {
op("LIKE", left, f("REPLACE", right, "'" + escapeChar.toString + "'", "'\\\\'"))
}
case RLike(expressionExtractor(left), expressionExtractor(right)) =>
op("RLIKE", left, regexpFromStart(right))
// stringExpressions.scala
case Contains(expressionExtractor(left), expressionExtractor(right)) =>
op(">", f("INSTR", left, right), "0")
case StartsWith(expressionExtractor(left), expressionExtractor(right)) =>
op("LIKE", left, f("CONCAT", right, StringVar("%")))
case EndsWith(expressionExtractor(left), expressionExtractor(right)) =>
op("LIKE", left, f("CONCAT", StringVar("%"), right))
case StringInstr(expressionExtractor(str), expressionExtractor(substr)) =>
f("INSTR", str, substr)
case FormatNumber(expressionExtractor(x), e @ expressionExtractor(d))
if e.dataType == IntegerType =>
ifNeg(d, StringVar(null), f("FORMAT", x, d))
case StringRepeat(expressionExtractor(child), expressionExtractor(times)) =>
f("LPAD",
StringVar(""),
ifNeg(times, IntVar(0), times) + "*" + f("CHAR_LENGTH", child),
child)
case StringTrim(expressionExtractor(srcStr), None) =>
f("TRIM", Raw("BOTH") + "FROM" + srcStr)
case StringTrim(expressionExtractor(srcStr), Some(utf8StringFoldableExtractor(trimStr)))
if trimStr == UTF8String.fromString(" ") =>
f("TRIM", Raw("BOTH") + "FROM" + srcStr)
case StringTrimLeft(expressionExtractor(srcStr), None) =>
f("LTRIM", srcStr)
case StringTrimLeft(expressionExtractor(srcStr), Some(utf8StringFoldableExtractor(trimStr)))
if trimStr == UTF8String.fromString(" ") =>
f("LTRIM", srcStr)
case StringTrimRight(expressionExtractor(srcStr), None) =>
f("RTRIM", srcStr)
case StringTrimRight(expressionExtractor(srcStr), Some(utf8StringFoldableExtractor(trimStr)))
if trimStr == UTF8String.fromString(" ") =>
f("RTRIM", srcStr)
case FindInSet(expressionExtractor(left), utf8StringFoldableExtractor(right)) => {
val str_array = right.toString.split(',')
var caseBranches = stringToJoinable("")
for (i <- 1 to str_array.length) {
caseBranches += Raw(s"WHEN '${str_array(i - 1)}'")
caseBranches += Raw(s"THEN '${i.toString}'")
}
block(Raw("CASE") + left + caseBranches + Raw("ELSE 0 END"))
}
// TODO: case _: Levenshtein => None
// ----------------------------------
// Leaf Expressions
// ----------------------------------
// datetimeExpressions.scala
case CurrentDate(_) => "CURRENT_DATE()"
case CurrentTimestamp() => "NOW(6)"
// mathExpressions.scala
case EulerNumber() => math.E.toString
case Pi() => "PI()"
// TODO: case RegExpExtract(expressionExtractor(subject), expressionExtractor(regexp), expressionExtractor(idx)) => ???
// stringExpressions.scala
case StringReplace(expressionExtractor(srcExpr),
expressionExtractor(searchExpr),
expressionExtractor(replaceExpr)) =>
f("REPLACE", srcExpr, searchExpr, replaceExpr)
case SubstringIndex(expressionExtractor(strExpr),
expressionExtractor(delimExpr),
expressionExtractor(countExpr)) =>
f("SUBSTRING_INDEX", strExpr, delimExpr, countExpr)
case StringLocate(expressionExtractor(substr),
expressionExtractor(str),
expressionExtractor(start)) =>
f("LOCATE", substr, str, start)
case StringLPad(expressionExtractor(str),
expressionExtractor(len),
expressionExtractor(pad)) =>
f("LPAD", str, ifNeg(len, IntVar(0), len), pad)
case StringRPad(expressionExtractor(str),
expressionExtractor(len),
expressionExtractor(pad)) =>
f("RPAD", str, ifNeg(len, IntVar(0), len), pad)
case Substring(expressionExtractor(str),
expressionExtractor(pos),
expressionExtractor(len)) =>
f("SUBSTR", str, pos, len)
case Overlay(expressionExtractor(input),
expressionExtractor(replace),
expressionExtractor(pos),
expressionExtractor(len)) =>
f(
"IF",
op("<", len, IntVar(0)),
f("CONCAT",
f("LEFT", input, op("-", pos, "1")),
replace,
f("SUBSTR", input, op("+", f("LENGTH", replace), pos))),
f("CONCAT",
f("LEFT", input, op("-", pos, "1")),
replace,
f("SUBSTR", input, op("+", pos, len)))
)
case StringTranslate(expressionExtractor(srcExpr),
utf8StringFoldableExtractor(matchingExpr),
utf8StringFoldableExtractor(replaceExpr)) => {
var replaceContent = srcExpr
val replaceExprLen = replaceExpr.toString.length
val matchingExprLen = matchingExpr.toString.length
for (i <- 0 to Math.max(replaceExprLen, matchingExprLen) - 1) {
val matchingCurrCharacter = if (i < matchingExprLen) {
s"'${matchingExpr.toString.charAt(i)}'"
} else {
"''"
}
val replaceCurrCharacter = if (i < replaceExprLen) {
s"'${replaceExpr.toString.charAt(i)}'"
} else {
"''"
}
replaceContent = f("REPLACE", replaceContent, matchingCurrCharacter, replaceCurrCharacter)
}
replaceContent
}
// ----------------------------------
// Unary Expressions
// ----------------------------------
// arithmetic.scala
case UnaryPositive(expressionExtractor(child)) => f("+", child)
// bitwiseExpression.scala
case BitwiseNot(expressionExtractor(expr)) => f("~", expr)
case BitwiseCount(expressionExtractor(child)) =>
f("BIT_COUNT", child)
// TODO: case UpCast(expressionExtractor(child), dataType, walkedTypePath) => ???
// datetimeExpressions.scala
case Hour(expressionExtractor(child), _) => f("HOUR", child)
case Minute(expressionExtractor(child), _) => f("MINUTE", child)
case Second(expressionExtractor(child), _) => f("SECOND", child)
case DayOfYear(expressionExtractor(child)) => f("DAYOFYEAR", child)
case Year(expressionExtractor(child)) => f("YEAR", child)
case Quarter(expressionExtractor(child)) => f("QUARTER", child)
case Month(expressionExtractor(child)) => f("MONTH", child)
case DayOfMonth(expressionExtractor(child)) => f("DAY", child)
case DayOfWeek(expressionExtractor(child)) => f("DAYOFWEEK", child)
case WeekDay(expressionExtractor(child)) => f("WEEKDAY", child)
case WeekOfYear(expressionExtractor(child)) => f("WEEK", child, "3")
case LastDay(expressionExtractor(startDate)) => f("LAST_DAY", startDate)
case Now() => f("NOW")
// case DatePart(expressionExtractor(field), expressionExtractor(source), expressionExtractor(child)) => // Converts to CAST(field)
// case Extract(expressionExtractor(field), expressionExtractor(source), expressionExtractor(child)) => // Converts to CAST(field)
// case MakeInterval(_, _, _, _, _, _, _) => ???
// MakeDecimal and UnscaledValue value are used in DecimalAggregates optimizer
// This optimizer replace Decimals inside of the sum and aggregate expressions to the Longs using UnscaledValue
// and then casts the result back to Decimal using MakeDecimal
case MakeDecimal(expressionExtractor(child), p, s, _) =>
longToDecimal(child, p, s)
case UnscaledValue(decimalExpressionExtractor(child, precision, scale)) =>
op("!:>", op("*", child, math.pow(10.0, scale).toString), "BIGINT")
// hash.scala
case Md5(expressionExtractor(child)) => f("MD5", child)
case Sha1(expressionExtractor(child)) => f("SHA1", child)
case Sha2(expressionExtractor(left), right)
if right.foldable &&
right.eval().isInstanceOf[Int] &&
right.eval().asInstanceOf[Int] != 224 =>
f("SHA2", left, right.toString)
case Crc32(expressionExtractor(child)) => f("CRC32", child)
//jsonExpressions.scala
case GetJsonObject(expressionExtractor(json), utf8StringFoldableExtractor(path))
if path.toString.length >= 2 & path.toString.startsWith("$.") => {
val pathParts = path.toString.substring(2).split("\\.")
val goalPath = pathParts.last
var jsonQuery = json
for (i <- 0 to (pathParts.length - 2)) {
jsonQuery = f("JSON_EXTRACT_JSON", jsonQuery, StringVar(pathParts(i)))
}
f(
"IF",
op("=",
f("JSON_GET_TYPE", f("JSON_EXTRACT_JSON", jsonQuery, StringVar(goalPath))),
StringVar("string")),
f("JSON_EXTRACT_STRING", jsonQuery, StringVar(goalPath)),
f("JSON_EXTRACT_JSON", jsonQuery, StringVar(goalPath))
)
}
// mathExpressions.scala
case Acos(expressionExtractor(child)) => f("ACOS", child)
case Asin(expressionExtractor(child)) => f("ASIN", child)
case Atan(expressionExtractor(child)) => f("ATAN", child)
case Ceil(expressionExtractor(child)) => f("CEIL", child)
case Cos(expressionExtractor(child)) => f("COS", child)
case Exp(expressionExtractor(child)) => f("EXP", child)
case Expm1(expressionExtractor(child)) => block(func("EXP", child) + "- 1")
case Floor(expressionExtractor(child)) => f("FLOOR", child)
case Log(expressionExtractor(child)) => f("LOG", child)
case Log2(expressionExtractor(child)) => f("LOG2", child)
case Log10(expressionExtractor(child)) => f("LOG10", child)
case Log1p(expressionExtractor(child)) => f("LOG", child + "+ 1")
case Signum(expressionExtractor(child)) => f("SIGN", child)
case Sin(expressionExtractor(child)) => f("SIN", child)
case Sqrt(expressionExtractor(child)) => f("SQRT", child)
case Tan(expressionExtractor(child)) => f("TAN", child)
case Cot(expressionExtractor(child)) => f("COT", child)
case ToDegrees(expressionExtractor(child)) => f("DEGREES", child)
case ToRadians(expressionExtractor(child)) => f("RADIANS", child)
case Bin(expressionExtractor(child)) => f("BIN", child)
case Hex(expressionExtractor(child)) => f("HEX", child)
// case BoolAnd(expressionExtractor(arg)) => // Spark can't apply bool_and to smallint (Input to function 'bool_and' should have been boolean, but it's [smallint])
// case BoolOr(expressionExtractor(arg)) => // Spark can't apply bool_or to smallint (Input to function 'bool_or' should have been boolean, but it's [smallint])
// case ArrayForAll(expressionExtractor(arg), expressionExtractor(function)) => ???
// case SchemaOfCsv(expressionExtractor(child), options) => ???
// case MapEntries(expressionExtractor(child)) => ???
// case MapFilter(expressionExtractor(arg), expressionExtractor(function)) => ???
// case MapZipWith(expressionExtractor(left), expressionExtractor(right), expressionExtractor(function)) => ???
// case CsvToStructs(schema, options, expressionExtractor(child), timeZoneId) => ???
// case StructsToCsv(options, expressionExtractor(child), timeZoneId) => ???
// case SparkVersion() => ???
// case TransformKeys(expressionExtractor(argument), expressionExtractor(function)) => ???
// case TransformValues(expressionExtractor(argument), expressionExtractor(function)) => ???
// case XxHash64(children, seed) => // we have 32-bit hash, but don't have 64-bit
// tanh(x) = (exp(x) - exp(-x)) / (exp(x) + exp(-x))
case Tanh(expressionExtractor(child)) =>
op("/",
op("-", f("EXP", child), f("EXP", f("-", child))),
op("+", f("EXP", child), f("EXP", f("-", child))))
// sinh(x) = (exp(x) - exp(-x)) / 2
case Sinh(expressionExtractor(child)) =>
op("/", op("-", f("EXP", child), f("EXP", f("-", child))), "2")
// cosh(x) = (exp(x) + exp(-x)) / 2
case Cosh(expressionExtractor(child)) =>
op("/", op("+", f("EXP", child), f("EXP", f("-", child))), "2")
// asinh(x) = ln(x + sqrt(x^2 + 1))
case Asinh(expressionExtractor(child)) =>
f("LN", op("+", child, f("SQRT", op("+", f("POW", child, "2"), "1"))))
// acosh(x) = ln(x + sqrt(x^2 - 1))
case Acosh(expressionExtractor(child)) =>
f("LN", op("+", child, f("SQRT", op("-", f("POW", child, "2"), "1"))))
// atanh(x) = 1/2 * ln((1 + x)/(1 - x))
case Atanh(expressionExtractor(child)) =>
op("/", f("LN", op("/", op("+", "1", child), op("-", "1", child))), "2")
case Rint(expressionExtractor(child)) => f("ROUND", child, "0")
// TODO: case Factorial(expressionExtractor(child)) => ???
// TODO: case Cbrt(expressionExtractor(child)) => f("POW", child, op("/", "1", "3"))
// We need to wait for the engine to implement precise cbrt
// nullExpressions.scala
case NullIf(expressionExtractor(left), expressionExtractor(right), _) =>
f("NULLIF", left, right)
case Nvl(expressionExtractor(left), expressionExtractor(right), _) =>
f("COALESCE", left, right)
case IsNull(expressionExtractor(child)) => block(child) + "IS NULL"
case IsNotNull(expressionExtractor(child)) => block(child) + "IS NOT NULL"
case Nvl2(expressionExtractor(expr1),
expressionExtractor(expr2),
expressionExtractor(expr3),
_) =>
f("IF", expr1 + "IS NOT NULL", expr2, expr3)
// predicates.scala
case Not(expressionExtractor(child)) => block(Raw("NOT") + child)
case If(expressionExtractor(predicate),
expressionExtractor(trueValue),
expressionExtractor(falseValue)) =>
f("IF", predicate, trueValue, falseValue)
// TODO: case Randn(expressionExtractor(child)) => ???
// SortOrder.scala
// in SingleStore, nulls always come first when direction = ascending
case SortOrder(expressionExtractor(child), Ascending, NullsFirst, _) => block(child) + "ASC"
// in SingleStore, nulls always come last when direction = descending
case SortOrder(expressionExtractor(child), Descending, NullsLast, _) => block(child) + "DESC"
// stringExpressions.scala
case Upper(expressionExtractor(child)) => f("UPPER", child)
case Lower(expressionExtractor(child)) => f("LOWER", child)
case ConcatWs(expressionExtractor(Some(child))) => f("CONCAT_WS", child)
case StringSpace(expressionExtractor(child)) =>
f("LPAD", StringVar(""), child, StringVar(" "))
case Length(expressionExtractor(child)) => f("CHAR_LENGTH", child)
case BitLength(expressionExtractor(child)) => block(func("LENGTH", child) + "* 8")
case OctetLength(expressionExtractor(child)) => f("LENGTH", child)
case Ascii(expressionExtractor(child)) => f("ASCII", child)
case Chr(expressionExtractor(child)) =>
f("IF", f("ISNULL", child), StringVar(null), f("CHAR", child))
case versionSpecificExpressionGen(child) => child
case Uuid(_) if context.singlestoreVersionAtLeast("7.5.0") => "UUID()"
case InitCap(expressionExtractor(child)) => f("INITCAP", child)
// TODO: case StringReverse(expressionExtractor(child)) => ???
// TODO: case SoundEx(expressionExtractor(child)) => ???
}
}
}
================================================
FILE: src/main/scala/com/singlestore/spark/JdbcHelpers.scala
================================================
package com.singlestore.spark
import java.sql.{
Connection,
PreparedStatement,
ResultSet,
SQLException,
SQLInvalidAuthorizationSpecException,
Statement
}
import java.util.Properties
import java.util.UUID.randomUUID
import com.singlestore.spark.SinglestoreOptions.{TableKey, TableKeyType}
import com.singlestore.spark.SQLGen.{SinglestoreVersion, StringVar, VariableList}
import org.apache.spark.sql.{Row, SaveMode}
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils}
import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects}
import org.apache.spark.sql.types.{StringType, StructType}
import scala.util.{Failure, Success, Try}
case class SinglestorePartitionInfo(ordinal: Int, name: String, hostport: String)
object JdbcHelpers extends LazyLogging {
final val SINGLESTORE_CONNECT_TIMEOUT = "10000" // 10 seconds in ms
// register the SingleStoreDialect
JdbcDialects.registerDialect(SinglestoreDialect)
// Connection implicits
implicit class ConnectionHelpers(val conn: Connection) {
def withStatement[T](handle: Statement => T): T =
Loan(conn.createStatement()).to(handle)
def withPreparedStatement[T](query: String, handle: PreparedStatement => T): T =
Loan(conn.prepareStatement(query)).to(handle)
}
def getConnProperties(conf: SinglestoreOptions,
isOnExecutor: Boolean,
hostPorts: String*): Properties = {
val url: String = {
val base = s"jdbc:singlestore:loadbalance://${hostPorts.mkString(",")}"
conf.database match {
case Some(d) => s"$base/$d"
case None => base
}
}
val sessionVariables = Seq(
"sql_select_limit=18446744073709551615",
"compile_only=false",
"sql_mode='STRICT_ALL_TABLES,ONLY_FULL_GROUP_BY'"
).mkString(",")
val properties = new Properties()
properties.setProperty("url", url)
properties.setProperty("driverClassName", "com.singlestore.jdbc.Driver")
properties.setProperty("username", conf.user)
properties.setProperty("password", conf.password)
properties.setProperty(
"connectionAttributes",
s"_connector_name:SingleStore Spark Connector,_connector_version:${BuildInfo.version},_product_version:${conf.sparkVersion}")
properties.setProperty(
"connectionProperties",
(Map(
JDBCOptions.JDBC_TABLE_NAME -> "XXX",
"zeroDateTimeBehavior" -> "convertToNull",
"allowLoadLocalInfile" -> "true",
"connectTimeout" -> SINGLESTORE_CONNECT_TIMEOUT,
"sessionVariables" -> sessionVariables,
"tinyInt1isBit" -> "false",
"allowLocalInfile" -> "true"
) ++ conf.jdbcExtraOptions)
.map(pair => pair._1 + "=" + pair._2)
.mkString(";")
)
// This property is ignored by DBCP during the connection creation
// It is needed, to have different connection pools for the driver and executor in a local mode
properties.setProperty("isOnSparkExecutor", isOnExecutor.toString)
val connectionPoolOptions = if (isOnExecutor) {
conf.executorConnectionPoolOptions
} else {
conf.driverConnectionPoolOptions
}
if (connectionPoolOptions.enabled) {
properties.setProperty("maxTotal", connectionPoolOptions.MaxOpenConns.toString)
properties.setProperty("maxIdle", connectionPoolOptions.MaxIdleConns.toString)
properties.setProperty("maxWaitMillis", connectionPoolOptions.MaxWaitMS.toString)
properties.setProperty("minEvictableIdleTimeMillis",
connectionPoolOptions.MinEvictableIdleTimeMs.toString)
properties.setProperty("maxConnLifetimeMillis",
connectionPoolOptions.MaxConnLifetimeMS.toString)
properties.setProperty("timeBetweenEvictionRunsMillis",
connectionPoolOptions.TimeBetweenEvictionRunsMS.toString)
}
properties
}
def getDDLConnProperties(conf: SinglestoreOptions, isOnExecutor: Boolean): Properties =
getConnProperties(conf, isOnExecutor, conf.ddlEndpoint)
def getDMLConnProperties(conf: SinglestoreOptions, isOnExecutor: Boolean): Properties =
getConnProperties(conf, isOnExecutor, conf.dmlEndpoints: _*)
def executeQuery(conn: Connection, query: String, variables: Any*): Iterator[Row] = {
val statement = conn.prepareStatement(query)
try {
fillStatementJdbc(statement, variables.toList)
if (!statement.execute()) {
// We don't have a ResultSet
// Return an empty iterator
Iterator[Row]()
} else {
val rs = statement.getResultSet
val schema = jdbcUtilGetSchema(conn, rs, SinglestoreDialect, alwaysNullable = true)
JdbcUtils.resultSetToRows(rs, schema)
}
} finally {
statement.close()
}
}
def loadSchema(conf: SinglestoreOptions, query: String, variables: VariableList): StructType = {
conf.customSchema.getOrElse({
val conn =
SinglestoreConnectionPool.getConnection(getDDLConnProperties(conf, isOnExecutor = false))
try {
val statement =
conn.prepareStatement(SinglestoreDialect.getSchemaQuery(s"($query) AS q"))
try {
fillStatement(statement, variables)
val rs = statement.executeQuery()
try {
jdbcUtilGetSchema(conn, rs, SinglestoreDialect, alwaysNullable = true)
} finally {
rs.close()
}
} finally {
statement.close()
}
} finally {
conn.close()
}
})
}
private def jdbcUtilGetSchema(conn: Connection,
resultSet: ResultSet,
dialect: JdbcDialect,
alwaysNullable: Boolean = false,
isTimestampNTZ: Boolean = false): StructType = {
// Databricks has a different implementation of getSchema method
// Here we check 3 possible implementations and use existing one
val clazz = JdbcUtils.getClass
try {
// Try to find and call the method with `Connection`
val method = clazz.getMethod(
"getSchema",
classOf[Connection],
classOf[ResultSet],
classOf[JdbcDialect],
classOf[Boolean],
classOf[Boolean]
)
method
.invoke(JdbcUtils,
conn,
resultSet,
dialect,
alwaysNullable: java.lang.Boolean,
isTimestampNTZ: java.lang.Boolean)
.asInstanceOf[StructType]
} catch {
case _: NoSuchMethodException =>
try {
// Fallback to method without `Connection`
val method = clazz.getMethod(
"getSchema",
classOf[ResultSet],
classOf[JdbcDialect],
classOf[Boolean],
classOf[Boolean]
)
method
.invoke(JdbcUtils,
resultSet,
dialect,
alwaysNullable: java.lang.Boolean,
isTimestampNTZ: java.lang.Boolean)
.asInstanceOf[StructType]
} catch {
case _: NoSuchMethodException =>
// Fallback to method without `isTimestampNTZ`
val method = clazz.getMethod(
"getSchema",
classOf[ResultSet],
classOf[JdbcDialect],
classOf[Boolean]
)
method
.invoke(JdbcUtils, resultSet, dialect, alwaysNullable: java.lang.Boolean)
.asInstanceOf[StructType]
}
}
}
def explainQuery(conf: SinglestoreOptions, query: String, variables: VariableList): String = {
val conn =
SinglestoreConnectionPool.getConnection(getDDLConnProperties(conf, isOnExecutor = false))
try {
val statement = conn.prepareStatement(s"EXPLAIN $query")
try {
fillStatement(statement, variables)
val rs = statement.executeQuery()
try {
var out = List.empty[String]
while (rs.next) {
out = rs.getString(1) :: out
}
out.reverseIterator.mkString("\n")
} finally {
rs.close()
}
} finally {
statement.close()
}
} finally {
conn.close()
}
}
// explainJSONQuery runs `EXPLAIN JSON` on the query and returns the String
// representing this queries plan as JSON.
def explainJSONQuery(conf: SinglestoreOptions, query: String, variables: VariableList): String = {
val conn =
SinglestoreConnectionPool.getConnection(getDDLConnProperties(conf, isOnExecutor = false))
try {
val statement = conn.prepareStatement(s"EXPLAIN JSON ${query}")
try {
fillStatement(statement, variables)
val rs = statement.executeQuery()
try {
// we only expect one row in the output
if (!rs.next()) { assert(false, "EXPLAIN JSON failed to return a row") }
val json = rs.getString(1)
assert(!rs.next(), "EXPLAIN JSON returned more than one row")
json
} finally {
rs.close()
}
} finally {
statement.close()
}
} finally {
conn.close()
}
}
// partitionHostPorts returns a list of (ordinal, name, host:port) for all master
// partitions in the specified database
def partitionHostPorts(conf: SinglestoreOptions,
database: String): List[SinglestorePartitionInfo] = {
val conn =
SinglestoreConnectionPool.getConnection(getDDLConnProperties(conf, isOnExecutor = false))
try {
val statement = conn.prepareStatement(s"""
SELECT HOST, PORT
FROM INFORMATION_SCHEMA.DISTRIBUTED_PARTITIONS
WHERE DATABASE_NAME = ? AND ROLE = "Master"
ORDER BY ORDINAL ASC
""")
try {
fillStatement(statement, List(StringVar(database)))
val rs = statement.executeQuery()
try {
var out = List.empty[SinglestorePartitionInfo]
var idx = 0
while (rs.next) {
out = SinglestorePartitionInfo(idx,
s"${database}_${idx}",
s"${rs.getString(1)}:${rs.getInt(2)}") :: out
idx += 1
}
out.reverse
} finally {
rs.close()
}
} finally {
statement.close()
}
} finally {
conn.close()
}
}
/**
* Return map from original host/port to external host/port
* @param conf options
* @return Map `host:port` -> `externalHost:externalPort`
*/
def externalHostPorts(conf: SinglestoreOptions): Map[String, String] = {
val conn =
SinglestoreConnectionPool.getConnection(getDDLConnProperties(conf, isOnExecutor = false))
try {
val statement = conn.prepareStatement(s"""
SELECT IP_ADDR,
PORT,
EXTERNAL_HOST,
EXTERNAL_PORT
FROM INFORMATION_SCHEMA.MV_NODES
WHERE TYPE = "LEAF";
""")
try {
val rs = statement.executeQuery()
try {
var out = Map.empty[String, String]
while (rs.next) {
val host = rs.getString(1)
val port = rs.getInt(2)
val externalHost = rs.getString(3)
val externalPortString = rs.getString(4)
if (externalHost != null && externalPortString != null) {
val externalPort = externalPortString.toInt
out = out + (s"$host:$port" -> s"$externalHost:$externalPort")
}
}
out
} finally {
rs.close()
}
} finally {
statement.close()
}
} finally {
conn.close()
}
}
def fillStatement(stmt: PreparedStatement, variables: VariableList): Unit = {
import SQLGen._
if (variables.isEmpty) { return }
variables.zipWithIndex.foreach {
case (StringVar(v), index) => stmt.setString(index + 1, v)
case (IntVar(v), index) => stmt.setInt(index + 1, v)
case (LongVar(v), index) => stmt.setLong(index + 1, v)
case (ShortVar(v), index) => stmt.setShort(index + 1, v)
case (FloatVar(v), index) => stmt.setFloat(index + 1, v)
case (DoubleVar(v), index) => stmt.setDouble(index + 1, v)
case (DecimalVar(v), index) =>
stmt.setBigDecimal(index + 1, v.toJavaBigDecimal)
case (BooleanVar(v), index) => stmt.setBoolean(index + 1, v)
case (ByteVar(v), index) => stmt.setByte(index + 1, v)
case (DateVar(v), index) => stmt.setDate(index + 1, v)
case (TimestampVar(v), index) => stmt.setTimestamp(index + 1, v)
case (v, _) =>
throw new IllegalArgumentException(
"Unexpected Variable Type: " + v.getClass.getName
)
}
}
def fillStatementJdbc(stmt: PreparedStatement, variables: List[Any]): Unit = {
// here we leave it to JDBC driver to do type conversions
if (variables.isEmpty) { return }
for ((v, index) <- variables.zipWithIndex) {
stmt.setObject(index + 1, v)
}
}
def schemaToString(schema: StructType,
tableKeys: List[TableKey],
createRowstoreTable: Boolean): String = {
// spark should never call any of our code if the schema is empty
assert(schema.length > 0)
val fieldsSql = schema.fields
.map(field => {
val name = SinglestoreDialect.quoteIdentifier(field.name)
val typ = SinglestoreDialect
.getJDBCType(field.dataType)
.getOrElse(
throw new IllegalArgumentException(
s"Can't get JDBC type for ${field.dataType.simpleString}"
)
)
val nullable = if (field.nullable) "" else " NOT NULL"
val collation = if (field.dataType == StringType) " COLLATE UTF8_BIN" else ""
s"${name} ${typ.databaseTypeDefinition}${collation}${nullable}"
})
// we want to default all tables to columnstore, but in 6.8 and below you *must*
// specify a sort key so we just pick the first column arbitrarily for now
var finalTableKeys = tableKeys
// if all the keys are shard keys it means there are no other keys so we can default
if (!createRowstoreTable && tableKeys.forall(_.keyType == TableKeyType.Shard)) {
finalTableKeys = TableKey(TableKeyType.Columnstore, columns = schema.head.name) :: tableKeys
}
def keyNameColumnsSQL(key: TableKey) =
s"${key.name.map(SinglestoreDialect.quoteIdentifier).getOrElse("")}(${key.columns})"
val keysSql = finalTableKeys.map {
case key @ TableKey(TableKeyType.Primary, _, _) => s"PRIMARY KEY ${keyNameColumnsSQL(key)}"
case key @ TableKey(TableKeyType.Columnstore, _, _) =>
s"KEY ${keyNameColumnsSQL(key)} USING CLUSTERED COLUMNSTORE"
case key @ TableKey(TableKeyType.Unique, _, _) => s"UNIQUE KEY ${keyNameColumnsSQL(key)}"
case key @ TableKey(TableKeyType.Shard, _, _) => s"SHARD KEY ${keyNameColumnsSQL(key)}"
case key @ TableKey(TableKeyType.Key, _, _) => s"KEY ${keyNameColumnsSQL(key)}"
}
(fieldsSql ++ keysSql).mkString("(\n ", ",\n ", "\n)")
}
def tableExists(conn: Connection, table: TableIdentifier): Boolean = {
conn.withStatement(
stmt =>
Try {
try {
stmt.execute(SinglestoreDialect.getTableExistsQuery(table.quotedString))
} finally {
stmt.close()
}
}.isSuccess
)
}
def getSinglestoreVersion(conf: SinglestoreOptions): String = {
val conn =
SinglestoreConnectionPool.getConnection(getDDLConnProperties(conf, isOnExecutor = false))
val sql = "select @@memsql_version"
log.trace(s"Executing SQL:\n$sql")
val resultSet = conn.withStatement(stmt => {
try {
stmt.executeQuery(sql)
} catch {
case _: SQLException => throw new IllegalArgumentException("Can't get SingleStore version")
} finally {
stmt.close()
conn.close()
}
})
if (resultSet.next()) {
resultSet.getString("@@memsql_version")
} else throw new IllegalArgumentException("Can't get SingleStore version")
}
def createTable(conn: Connection,
table: TableIdentifier,
schema: StructType,
tableKeys: List[TableKey],
createRowstoreTable: Boolean,
version: SinglestoreVersion): Unit = {
val sql =
s"CREATE ${if (createRowstoreTable && version.atLeast("7.3.0")) "ROWSTORE" else ""} TABLE ${table.quotedString} ${schemaToString(schema, tableKeys, createRowstoreTable)}"
log.trace(s"Executing SQL:\n$sql")
conn.withStatement(stmt => stmt.executeUpdate(sql))
}
def getPartitionsCount(conn: Connection, database: String): Int = {
val sql =
s"SELECT num_partitions FROM information_schema.DISTRIBUTED_DATABASES WHERE database_name = '$database'"
log.trace(s"Executing SQL:\n$sql")
val resultSet = conn.withStatement(stmt => stmt.executeQuery(sql))
if (resultSet.next()) {
resultSet.getInt("num_partitions")
} else {
throw new IllegalArgumentException(
s"Failed to get number of partitions for '$database' database")
}
}
def getResultTableName(applicationId: String,
stageId: Int,
rddId: Int,
attemptNumber: Int,
randHex: String): String = {
s"rt_${applicationId.replace("-", "")}_${stageId}_${rddId}_${attemptNumber}_${randHex}"
}
def getCreateResultTableQuery(tableName: String,
query: String,
schema: StructType,
materialized: Boolean,
needsRepartition: Boolean,
repartitionColumns: Seq[String]): String = {
val materializedStr = { if (materialized) { "MATERIALIZED" } else "" }
if (needsRepartition) {
if (repartitionColumns.isEmpty) {
val randColName = s"randColumn${randomU
gitextract_u_sh6j0w/
├── .arcconfig
├── .github/
│ └── workflows/
│ └── test-and-publish.yml
├── .gitignore
├── .java-version
├── .scalafmt.conf
├── CHANGELOG
├── LICENSE
├── README.md
├── build.sbt
├── ci/
│ └── secring.asc.enc
├── demo/
│ ├── Dockerfile
│ ├── README.md
│ └── notebook/
│ ├── pyspark-singlestore-demo_2F8XQUKFG.zpln
│ ├── scala-singlestore-demo_2F6Y3APTX.zpln
│ └── spark-sql-singlestore-demo_2F7PZ81H6.zpln
├── project/
│ ├── build.properties
│ └── plugins.sbt
├── scripts/
│ ├── jwt/
│ │ └── jwt_auth_config.json
│ └── setup-cluster.sh
└── src/
├── main/
│ ├── resources/
│ │ └── META-INF/
│ │ └── services/
│ │ └── org.apache.spark.sql.sources.DataSourceRegister
│ ├── scala/
│ │ └── com/
│ │ ├── memsql/
│ │ │ └── spark/
│ │ │ └── DefaultSource.scala
│ │ └── singlestore/
│ │ └── spark/
│ │ ├── AggregatorParallelReadListener.scala
│ │ ├── AvroSchemaHelper.scala
│ │ ├── CompletionIterator.scala
│ │ ├── DefaultSource.scala
│ │ ├── ExpressionGen.scala
│ │ ├── JdbcHelpers.scala
│ │ ├── LazyLogging.scala
│ │ ├── Loan.scala
│ │ ├── MetricsHandler.scala
│ │ ├── OverwriteBehavior.scala
│ │ ├── ParallelReadEnablement.scala
│ │ ├── ParallelReadType.scala
│ │ ├── SQLGen.scala
│ │ ├── SQLHelper.scala
│ │ ├── SQLPushdownRule.scala
│ │ ├── SinglestoreBatchInsertWriter.scala
│ │ ├── SinglestoreConnectionPool.scala
│ │ ├── SinglestoreConnectionPoolOptions.scala
│ │ ├── SinglestoreDialect.scala
│ │ ├── SinglestoreLoadDataWriter.scala
│ │ ├── SinglestoreOptions.scala
│ │ ├── SinglestorePartitioner.scala
│ │ ├── SinglestoreRDD.scala
│ │ ├── SinglestoreReader.scala
│ │ └── vendor/
│ │ └── apache/
│ │ ├── SchemaConverters.scala
│ │ └── third_party_license
│ ├── scala-sparkv3.1/
│ │ └── spark/
│ │ ├── MaxNumConcurentTasks.scala
│ │ ├── VersionSpecificAggregateExpressionExtractor.scala
│ │ ├── VersionSpecificAggregateExtractor.scala
│ │ ├── VersionSpecificExpressionGen.scala
│ │ ├── VersionSpecificSortExtractor.scala
│ │ ├── VersionSpecificUtil.scala
│ │ ├── VersionSpecificWindowBoundaryExpressionExtractor.scala
│ │ └── VersionSpecificWindowExtractor.scala
│ ├── scala-sparkv3.2/
│ │ └── spark/
│ │ ├── MaxNumConcurrentTasks.scala
│ │ ├── VersionSpecificAggregateExpressionExtractor.scala
│ │ ├── VersionSpecificAggregateExtractor.scala
│ │ ├── VersionSpecificExpressionGen.scala
│ │ ├── VersionSpecificSortExtractor.scala
│ │ ├── VersionSpecificUtil.scala
│ │ ├── VersionSpecificWindowBoundaryExpressionExtractor.scala
│ │ └── VersionSpecificWindowExtractor.scala
│ ├── scala-sparkv3.3/
│ │ └── spark/
│ │ ├── MaxNumConcurrentTasks.scala
│ │ ├── VersionSpecificAggregateExpressionExtractor.scala
│ │ ├── VersionSpecificAggregateExtractor.scala
│ │ ├── VersionSpecificExpressionGen.scala
│ │ ├── VersionSpecificSortExtractor.scala
│ │ ├── VersionSpecificUtil.scala
│ │ ├── VersionSpecificWindowBoundaryExpressionExtractor.scala
│ │ └── VersionSpecificWindowExtractor.scala
│ ├── scala-sparkv3.4/
│ │ └── spark/
│ │ ├── MaxNumConcurrentTasks.scala
│ │ ├── VersionSpecificAggregateExpressionExtractor.scala
│ │ ├── VersionSpecificAggregateExtractor.scala
│ │ ├── VersionSpecificExpressionGen.scala
│ │ ├── VersionSpecificSortExtractor.scala
│ │ ├── VersionSpecificUtil.scala
│ │ ├── VersionSpecificWindowBoundaryExpressionExtractor.scala
│ │ └── VersionSpecificWindowExtractor.scala
│ ├── scala-sparkv3.5/
│ │ └── spark/
│ │ ├── MaxNumConcurrentTasks.scala
│ │ ├── VersionSpecificAggregateExpressionExtractor.scala
│ │ ├── VersionSpecificAggregateExtractor.scala
│ │ ├── VersionSpecificExpressionGen.scala
│ │ ├── VersionSpecificSortExtractor.scala
│ │ ├── VersionSpecificUtil.scala
│ │ ├── VersionSpecificWindowBoundaryExpressionExtractor.scala
│ │ └── VersionSpecificWindowExtractor.scala
│ └── scala-sparkv4.0/
│ └── spark/
│ ├── MaxNumConcurrentTasks.scala
│ ├── VersionSpecificAggregateExpressionExtractor.scala
│ ├── VersionSpecificAggregateExtractor.scala
│ ├── VersionSpecificExpressionGen.scala
│ ├── VersionSpecificSortExtractor.scala
│ ├── VersionSpecificUtil.scala
│ ├── VersionSpecificWindowBoundaryExpressionExtractor.scala
│ └── VersionSpecificWindowExtractor.scala
└── test/
├── resources/
│ ├── data/
│ │ ├── movies.json
│ │ ├── movies_rating.json
│ │ ├── reviews.json
│ │ └── users.json
│ ├── log4j.properties
│ ├── log4j2.properties
│ └── mockito-extensions/
│ └── org.mockito.plugins.MockMaker
└── scala/
└── com/
└── singlestore/
└── spark/
├── BatchInsertBenchmark.scala
├── BatchInsertTest.scala
├── BenchmarkSerializingTest.scala
├── BinaryTypeBenchmark.scala
├── CustomDatatypesTest.scala
├── ExternalHostTest.scala
├── IntegrationSuiteBase.scala
├── IssuesTest.scala
├── LoadDataBenchmark.scala
├── LoadDataTest.scala
├── LoadbalanceTest.scala
├── MaxErrorsTest.scala
├── OutputMetricsTest.scala
├── ReferenceTableTest.scala
├── SQLHelperTest.scala
├── SQLOverwriteTest.scala
├── SQLPermissionsTest.scala
├── SQLPushdownTest.scala
├── SanityTest.scala
├── SinglestoreConnectionPoolTest.scala
├── SinglestoreOptionsTest.scala
├── TestHelper.scala
└── VersionTest.scala
Condensed preview — 125 files, each showing path, character count, and a content snippet. Download the .json file or copy for the full structured content (1,379K chars).
[
{
"path": ".arcconfig",
"chars": 116,
"preview": "{\n \"project_id\" : \"memsql-spark-connector\",\n \"conduit_uri\" : \"https:\\/\\/grizzly.internal.memcompute.com\\/api\\/\"\n}\n"
},
{
"path": ".github/workflows/test-and-publish.yml",
"chars": 5212,
"preview": "name: Test and Publish\n\non:\n pull_request:\n types: [ opened, synchronize, reopened ]\n schedule:\n - cron: \"0 0 * "
},
{
"path": ".gitignore",
"chars": 558,
"preview": "# Covers JetBrains IDEs: IntelliJ, RubyMine, PhpStorm, AppCode, PyCharm, CLion, Android Studio and WebStorm\n# Reference:"
},
{
"path": ".java-version",
"chars": 4,
"preview": "1.8\n"
},
{
"path": ".scalafmt.conf",
"chars": 28,
"preview": "maxColumn = 100\nalign = more"
},
{
"path": "CHANGELOG",
"chars": 9840,
"preview": "2026-04-02 Version 5.0.1\n * Added customSchema option\n\n2026-04-02 Version 5.0.0\n * Removed setting of the default "
},
{
"path": "LICENSE",
"chars": 11362,
"preview": "\n Apache License\n Version 2.0, January 2004\n "
},
{
"path": "README.md",
"chars": 47567,
"preview": "# SingleStoreDB Spark Connector\n## Version: 5.0.1 ["
},
{
"path": "build.sbt",
"chars": 3706,
"preview": "import xerial.sbt.Sonatype._\n\n/*\n To run tests or publish with a specific spark version use this java option:\n -Dspa"
},
{
"path": "demo/Dockerfile",
"chars": 421,
"preview": "FROM apache/zeppelin:0.9.0\n\nENV SPARK_VERSION=4.0.0\n\nUSER root\n\nRUN wget https://apache.ip-connect.vn.ua/spark/spark-${S"
},
{
"path": "demo/README.md",
"chars": 2697,
"preview": "## singlestore-spark-connector demo\n\nThis is Dockerfile which uses the upstream [Zeppelin Image](https://hub.docker.com/"
},
{
"path": "demo/notebook/pyspark-singlestore-demo_2F8XQUKFG.zpln",
"chars": 11320,
"preview": "{\n \"paragraphs\": [\n {\n \"text\": \"%md\\n## This is a small demo that illustrates the usage of the SingleStore-Spar"
},
{
"path": "demo/notebook/scala-singlestore-demo_2F6Y3APTX.zpln",
"chars": 12089,
"preview": "{\n \"paragraphs\": [\n {\n \"text\": \"%md\\n## This is a small demo that illustrates the usage of the SingleStore-Spar"
},
{
"path": "demo/notebook/spark-sql-singlestore-demo_2F7PZ81H6.zpln",
"chars": 16337,
"preview": "{\n \"paragraphs\": [\n {\n \"text\": \"%md\\n## This is a small demo that illustrates the usage of the SingleStore-Spar"
},
{
"path": "project/build.properties",
"chars": 18,
"preview": "sbt.version=1.3.8\n"
},
{
"path": "project/plugins.sbt",
"chars": 238,
"preview": "addSbtPlugin(\"org.xerial.sbt\" % \"sbt-sonatype\" % \"3.12.2\")\naddSbtPlugin(\"com.jsuereth\" % \"sbt-pgp\" % \"2.0.1\")\na"
},
{
"path": "scripts/jwt/jwt_auth_config.json",
"chars": 943,
"preview": "{\n \"username_claim\": \"username\",\n \"methods\": [\n {\n \"algorithms\": [ \"RS384\" ],\n \"secret\": \"-----"
},
{
"path": "scripts/setup-cluster.sh",
"chars": 6356,
"preview": "set -eu\n\n# this script must be run from the top-level of the repo\ncd \"$(git rev-parse --show-toplevel)\"\n\n\nDEFAULT_SINGLE"
},
{
"path": "src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister",
"chars": 67,
"preview": "com.singlestore.spark.DefaultSource\ncom.memsql.spark.DefaultSource\n"
},
{
"path": "src/main/scala/com/memsql/spark/DefaultSource.scala",
"chars": 191,
"preview": "package com.memsql.spark\n\nimport com.singlestore.spark\n\nclass DefaultSource extends spark.DefaultSource {\n\n override de"
},
{
"path": "src/main/scala/com/singlestore/spark/AggregatorParallelReadListener.scala",
"chars": 6184,
"preview": "package com.singlestore.spark\n\nimport java.sql.{Connection, SQLException}\nimport java.util.Properties\nimport com.singles"
},
{
"path": "src/main/scala/com/singlestore/spark/AvroSchemaHelper.scala",
"chars": 617,
"preview": "package com.singlestore.spark\n\nimport org.apache.avro.Schema\nimport org.apache.avro.Schema.Type\nimport org.apache.avro.S"
},
{
"path": "src/main/scala/com/singlestore/spark/CompletionIterator.scala",
"chars": 707,
"preview": "package com.singlestore.spark\n\n// Copied from spark's CompletionIterator which is private even though it is generically "
},
{
"path": "src/main/scala/com/singlestore/spark/DefaultSource.scala",
"chars": 4265,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.SQLGenContext\nimport org.apache.spark.TaskContext\nimp"
},
{
"path": "src/main/scala/com/singlestore/spark/ExpressionGen.scala",
"chars": 37178,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.aggregate._\nimport org.apache.spark.sql."
},
{
"path": "src/main/scala/com/singlestore/spark/JdbcHelpers.scala",
"chars": 23949,
"preview": "package com.singlestore.spark\n\nimport java.sql.{\n Connection,\n PreparedStatement,\n ResultSet,\n SQLException,\n SQLIn"
},
{
"path": "src/main/scala/com/singlestore/spark/LazyLogging.scala",
"chars": 185,
"preview": "package com.singlestore.spark\n\nimport org.slf4j.{Logger, LoggerFactory}\n\ntrait LazyLogging {\n @transient\n protected la"
},
{
"path": "src/main/scala/com/singlestore/spark/Loan.scala",
"chars": 249,
"preview": "package com.singlestore.spark\n\nclass Loan[A <: AutoCloseable](resource: A) {\n def to[T](handle: A => T): T =\n try ha"
},
{
"path": "src/main/scala/com/singlestore/spark/MetricsHandler.scala",
"chars": 222,
"preview": "package org.apache.spark.metrics.source\n\nimport org.apache.spark.TaskContext\n\nobject MetricsHandler {\n def setRecordsWr"
},
{
"path": "src/main/scala/com/singlestore/spark/OverwriteBehavior.scala",
"chars": 579,
"preview": "package com.singlestore.spark\n\nsealed trait OverwriteBehavior\n\ncase object Truncate extends OverwriteBehavior\ncase "
},
{
"path": "src/main/scala/com/singlestore/spark/ParallelReadEnablement.scala",
"chars": 976,
"preview": "package com.singlestore.spark\n\nsealed trait ParallelReadEnablement\n\ncase object Disabled extends ParallelReadEnable"
},
{
"path": "src/main/scala/com/singlestore/spark/ParallelReadType.scala",
"chars": 879,
"preview": "package com.singlestore.spark\n\nsealed trait ParallelReadType\n\ncase object ReadFromLeaves extends Parall"
},
{
"path": "src/main/scala/com/singlestore/spark/SQLGen.scala",
"chars": 27793,
"preview": "package com.singlestore.spark\n\nimport java.sql.{Date, Timestamp}\nimport org.apache.spark.sql.catalyst.expressions._\nimpo"
},
{
"path": "src/main/scala/com/singlestore/spark/SQLHelper.scala",
"chars": 3208,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.JdbcHelpers.{executeQuery, getDDLConnProperties}\nimport org."
},
{
"path": "src/main/scala/com/singlestore/spark/SQLPushdownRule.scala",
"chars": 3130,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.{ExpressionExtractor, SQLGenContext}\nimport org.apach"
},
{
"path": "src/main/scala/com/singlestore/spark/SinglestoreBatchInsertWriter.scala",
"chars": 3327,
"preview": "package com.singlestore.spark\n\nimport java.sql.Connection\nimport java.util.Base64\n\nimport com.singlestore.spark.JdbcHelp"
},
{
"path": "src/main/scala/com/singlestore/spark/SinglestoreConnectionPool.scala",
"chars": 1349,
"preview": "package com.singlestore.spark\n\nimport java.sql.Connection\nimport java.util.Properties\nimport org.apache.commons.dbcp2.{B"
},
{
"path": "src/main/scala/com/singlestore/spark/SinglestoreConnectionPoolOptions.scala",
"chars": 500,
"preview": "package com.singlestore.spark\n\ncase class SinglestoreConnectionPoolOptions(enabled: Boolean,\n "
},
{
"path": "src/main/scala/com/singlestore/spark/SinglestoreDialect.scala",
"chars": 2787,
"preview": "package com.singlestore.spark\n\nimport java.sql.Types\n\nimport org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils\ni"
},
{
"path": "src/main/scala/com/singlestore/spark/SinglestoreLoadDataWriter.scala",
"chars": 12297,
"preview": "package com.singlestore.spark\n\nimport java.io.{IOException, InputStream, OutputStream, PipedInputStream, PipedOutputStre"
},
{
"path": "src/main/scala/com/singlestore/spark/SinglestoreOptions.scala",
"chars": 16939,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SinglestoreOptions.TableKey\nimport org.apache.spark.sql.cata"
},
{
"path": "src/main/scala/com/singlestore/spark/SinglestorePartitioner.scala",
"chars": 13545,
"preview": "package com.singlestore.spark\n\nimport java.sql.SQLException\nimport java.util.Properties\n\nimport com.singlestore.spark.Jd"
},
{
"path": "src/main/scala/com/singlestore/spark/SinglestoreRDD.scala",
"chars": 7582,
"preview": "package com.singlestore.spark\n\nimport java.sql.{Connection, Date, PreparedStatement, ResultSet}\nimport java.util.concurr"
},
{
"path": "src/main/scala/com/singlestore/spark/SinglestoreReader.scala",
"chars": 5754,
"preview": "package com.singlestore.spark\n\nimport java.sql.SQLSyntaxErrorException\n\nimport com.singlestore.spark.SQLGen.{ExpressionE"
},
{
"path": "src/main/scala/com/singlestore/spark/vendor/apache/SchemaConverters.scala",
"chars": 2564,
"preview": "package com.singlestore.spark.vendor.apache\n\nimport org.apache.avro._\nimport org.apache.spark.sql.types._\n\n/**\n * NOTE"
},
{
"path": "src/main/scala/com/singlestore/spark/vendor/apache/third_party_license",
"chars": 11355,
"preview": " Apache License\n Version 2.0, January 2004\n "
},
{
"path": "src/main/scala-sparkv3.1/spark/MaxNumConcurentTasks.scala",
"chars": 423,
"preview": "package org.apache.spark.scheduler\n\nimport org.apache.spark.rdd.RDD\n\nobject MaxNumConcurrentTasks {\n def get(rdd: RDD[_"
},
{
"path": "src/main/scala-sparkv3.1/spark/VersionSpecificAggregateExpressionExtractor.scala",
"chars": 4104,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.{ExpressionExtractor, SQLGenContext, Statement}\nimpor"
},
{
"path": "src/main/scala-sparkv3.1/spark/VersionSpecificAggregateExtractor.scala",
"chars": 516,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}\nimport org"
},
{
"path": "src/main/scala-sparkv3.1/spark/VersionSpecificExpressionGen.scala",
"chars": 10931,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.ExpressionGen._\nimport com.singlestore.spark.SQLGen.{Express"
},
{
"path": "src/main/scala-sparkv3.1/spark/VersionSpecificSortExtractor.scala",
"chars": 491,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}\nimport org.apach"
},
{
"path": "src/main/scala-sparkv3.1/spark/VersionSpecificUtil.scala",
"chars": 215,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.types.{CalendarIntervalType, DataType}\n\nobject VersionSpecifi"
},
{
"path": "src/main/scala-sparkv3.1/spark/VersionSpecificWindowBoundaryExpressionExtractor.scala",
"chars": 553,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.{ExpressionExtractor, SQLGenContext, Statement}\nimpor"
},
{
"path": "src/main/scala-sparkv3.1/spark/VersionSpecificWindowExtractor.scala",
"chars": 539,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression, SortOrder}"
},
{
"path": "src/main/scala-sparkv3.2/spark/MaxNumConcurrentTasks.scala",
"chars": 423,
"preview": "package org.apache.spark.scheduler\n\nimport org.apache.spark.rdd.RDD\n\nobject MaxNumConcurrentTasks {\n def get(rdd: RDD[_"
},
{
"path": "src/main/scala-sparkv3.2/spark/VersionSpecificAggregateExpressionExtractor.scala",
"chars": 4136,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.{ExpressionExtractor, SQLGenContext, Statement}\nimpor"
},
{
"path": "src/main/scala-sparkv3.2/spark/VersionSpecificAggregateExtractor.scala",
"chars": 516,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}\nimport org"
},
{
"path": "src/main/scala-sparkv3.2/spark/VersionSpecificExpressionGen.scala",
"chars": 12259,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.ExpressionGen._\nimport com.singlestore.spark.SQLGen.{\n Expr"
},
{
"path": "src/main/scala-sparkv3.2/spark/VersionSpecificSortExtractor.scala",
"chars": 491,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}\nimport org.apach"
},
{
"path": "src/main/scala-sparkv3.2/spark/VersionSpecificUtil.scala",
"chars": 357,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.types.{\n CalendarIntervalType,\n DataType,\n DayTimeInterval"
},
{
"path": "src/main/scala-sparkv3.2/spark/VersionSpecificWindowBoundaryExpressionExtractor.scala",
"chars": 496,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.{ExpressionExtractor, SQLGenContext, Statement}\nimpor"
},
{
"path": "src/main/scala-sparkv3.2/spark/VersionSpecificWindowExtractor.scala",
"chars": 539,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression, SortOrder}"
},
{
"path": "src/main/scala-sparkv3.3/spark/MaxNumConcurrentTasks.scala",
"chars": 423,
"preview": "package org.apache.spark.scheduler\n\nimport org.apache.spark.rdd.RDD\n\nobject MaxNumConcurrentTasks {\n def get(rdd: RDD[_"
},
{
"path": "src/main/scala-sparkv3.3/spark/VersionSpecificAggregateExpressionExtractor.scala",
"chars": 4136,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.{ExpressionExtractor, SQLGenContext, Statement}\nimpor"
},
{
"path": "src/main/scala-sparkv3.3/spark/VersionSpecificAggregateExtractor.scala",
"chars": 516,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}\nimport org"
},
{
"path": "src/main/scala-sparkv3.3/spark/VersionSpecificExpressionGen.scala",
"chars": 11887,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.ExpressionGen._\nimport com.singlestore.spark.SQLGen.{\n Expr"
},
{
"path": "src/main/scala-sparkv3.3/spark/VersionSpecificSortExtractor.scala",
"chars": 491,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}\nimport org.apach"
},
{
"path": "src/main/scala-sparkv3.3/spark/VersionSpecificUtil.scala",
"chars": 357,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.types.{\n CalendarIntervalType,\n DataType,\n DayTimeInterval"
},
{
"path": "src/main/scala-sparkv3.3/spark/VersionSpecificWindowBoundaryExpressionExtractor.scala",
"chars": 496,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.{ExpressionExtractor, SQLGenContext, Statement}\nimpor"
},
{
"path": "src/main/scala-sparkv3.3/spark/VersionSpecificWindowExtractor.scala",
"chars": 539,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression, SortOrder}"
},
{
"path": "src/main/scala-sparkv3.4/spark/MaxNumConcurrentTasks.scala",
"chars": 423,
"preview": "package org.apache.spark.scheduler\n\nimport org.apache.spark.rdd.RDD\n\nobject MaxNumConcurrentTasks {\n def get(rdd: RDD[_"
},
{
"path": "src/main/scala-sparkv3.4/spark/VersionSpecificAggregateExpressionExtractor.scala",
"chars": 4214,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.{ExpressionExtractor, SQLGenContext, Statement}\nimpor"
},
{
"path": "src/main/scala-sparkv3.4/spark/VersionSpecificAggregateExtractor.scala",
"chars": 516,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}\nimport org"
},
{
"path": "src/main/scala-sparkv3.4/spark/VersionSpecificExpressionGen.scala",
"chars": 11990,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.ExpressionGen._\nimport com.singlestore.spark.SQLGen.{\n Expr"
},
{
"path": "src/main/scala-sparkv3.4/spark/VersionSpecificSortExtractor.scala",
"chars": 491,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}\nimport org.apach"
},
{
"path": "src/main/scala-sparkv3.4/spark/VersionSpecificUtil.scala",
"chars": 357,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.types.{\n CalendarIntervalType,\n DataType,\n DayTimeInterval"
},
{
"path": "src/main/scala-sparkv3.4/spark/VersionSpecificWindowBoundaryExpressionExtractor.scala",
"chars": 496,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.{ExpressionExtractor, SQLGenContext, Statement}\nimpor"
},
{
"path": "src/main/scala-sparkv3.4/spark/VersionSpecificWindowExtractor.scala",
"chars": 539,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression, SortOrder}"
},
{
"path": "src/main/scala-sparkv3.5/spark/MaxNumConcurrentTasks.scala",
"chars": 423,
"preview": "package org.apache.spark.scheduler\n\nimport org.apache.spark.rdd.RDD\n\nobject MaxNumConcurrentTasks {\n def get(rdd: RDD[_"
},
{
"path": "src/main/scala-sparkv3.5/spark/VersionSpecificAggregateExpressionExtractor.scala",
"chars": 4214,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.{ExpressionExtractor, SQLGenContext, Statement}\nimpor"
},
{
"path": "src/main/scala-sparkv3.5/spark/VersionSpecificAggregateExtractor.scala",
"chars": 516,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}\nimport org"
},
{
"path": "src/main/scala-sparkv3.5/spark/VersionSpecificExpressionGen.scala",
"chars": 11990,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.ExpressionGen._\nimport com.singlestore.spark.SQLGen.{\n Expr"
},
{
"path": "src/main/scala-sparkv3.5/spark/VersionSpecificSortExtractor.scala",
"chars": 491,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}\nimport org.apach"
},
{
"path": "src/main/scala-sparkv3.5/spark/VersionSpecificUtil.scala",
"chars": 357,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.types.{\n CalendarIntervalType,\n DataType,\n DayTimeInterval"
},
{
"path": "src/main/scala-sparkv3.5/spark/VersionSpecificWindowBoundaryExpressionExtractor.scala",
"chars": 496,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.{ExpressionExtractor, SQLGenContext, Statement}\nimpor"
},
{
"path": "src/main/scala-sparkv3.5/spark/VersionSpecificWindowExtractor.scala",
"chars": 539,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression, SortOrder}"
},
{
"path": "src/main/scala-sparkv4.0/spark/MaxNumConcurrentTasks.scala",
"chars": 423,
"preview": "package org.apache.spark.scheduler\n\nimport org.apache.spark.rdd.RDD\n\nobject MaxNumConcurrentTasks {\n def get(rdd: RDD[_"
},
{
"path": "src/main/scala-sparkv4.0/spark/VersionSpecificAggregateExpressionExtractor.scala",
"chars": 4214,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.{ExpressionExtractor, SQLGenContext, Statement}\nimpor"
},
{
"path": "src/main/scala-sparkv4.0/spark/VersionSpecificAggregateExtractor.scala",
"chars": 519,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}\nimport org"
},
{
"path": "src/main/scala-sparkv4.0/spark/VersionSpecificExpressionGen.scala",
"chars": 12350,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.ExpressionGen._\nimport com.singlestore.spark.SQLGen.{\n Expr"
},
{
"path": "src/main/scala-sparkv4.0/spark/VersionSpecificSortExtractor.scala",
"chars": 414,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.SortOrder\nimport org.apache.spark.sql.ca"
},
{
"path": "src/main/scala-sparkv4.0/spark/VersionSpecificUtil.scala",
"chars": 357,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.types.{\n CalendarIntervalType,\n DataType,\n DayTimeInterval"
},
{
"path": "src/main/scala-sparkv4.0/spark/VersionSpecificWindowBoundaryExpressionExtractor.scala",
"chars": 496,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.{ExpressionExtractor, SQLGenContext, Statement}\nimpor"
},
{
"path": "src/main/scala-sparkv4.0/spark/VersionSpecificWindowExtractor.scala",
"chars": 542,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression, SortOrder}"
},
{
"path": "src/test/resources/data/movies.json",
"chars": 29682,
"preview": "{\"id\":1,\"title\":\"Vampire in Venice (Nosferatu a Venezia) (Nosferatu in Venice)\",\"genre\":\"Horror\",\"critic_review\":\"Intege"
},
{
"path": "src/test/resources/data/movies_rating.json",
"chars": 29758,
"preview": "{\"id\":1,\"movie_rating\": '{ \\\"movie_id\\\" : 1, \\\"title\\\" : \\\"Vampire in Venice (Nosferatu a Venezia) (Nosferatu in Venice)"
},
{
"path": "src/test/resources/data/reviews.json",
"chars": 338411,
"preview": "{\"user_id\":255,\"movie_id\":44,\"rating\":3.0,\"review\":\"Fusce consequat. Nulla nisl. Nunc nisl.\",\"created\":\"1996-01-21T17:25"
},
{
"path": "src/test/resources/data/users.json",
"chars": 165714,
"preview": "{\"id\":1,\"first_name\":\"Di\",\"last_name\":\"Gother\",\"email\":\"dgother0@adobe.com\",\"owns_house\":false,\"favorite_color\":\"Fuscia\""
},
{
"path": "src/test/resources/log4j.properties",
"chars": 686,
"preview": "# Set everything to be logged to the console\nlog4j.rootCategory=ERROR, console\nlog4j.appender.console=org.apache.log4j.C"
},
{
"path": "src/test/resources/log4j2.properties",
"chars": 584,
"preview": "# Create STDOUT appender that writes data to the console\nappenders = console\nappender.console.type = Console\nappender.co"
},
{
"path": "src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker",
"chars": 17,
"preview": "mock-maker-inline"
},
{
"path": "src/test/scala/com/singlestore/spark/BatchInsertBenchmark.scala",
"chars": 2538,
"preview": "package com.singlestore.spark\n\nimport java.sql.{Connection, Date, DriverManager}\nimport java.time.LocalDate\nimport java."
},
{
"path": "src/test/scala/com/singlestore/spark/BatchInsertTest.scala",
"chars": 5487,
"preview": "package com.singlestore.spark\n\nimport com.github.mrpowers.spark.daria.sql.SparkSessionExt._\nimport org.apache.spark.sql."
},
{
"path": "src/test/scala/com/singlestore/spark/BenchmarkSerializingTest.scala",
"chars": 5341,
"preview": "package com.singlestore.spark\n\nimport com.github.mrpowers.spark.daria.sql.SparkSessionExt._\nimport org.apache.spark.sql."
},
{
"path": "src/test/scala/com/singlestore/spark/BinaryTypeBenchmark.scala",
"chars": 2886,
"preview": "package com.singlestore.spark\n\nimport java.sql.{Connection, DriverManager}\nimport java.util.Properties\n\nimport com.githu"
},
{
"path": "src/test/scala/com/singlestore/spark/CustomDatatypesTest.scala",
"chars": 24701,
"preview": "package com.singlestore.spark\n\nimport java.sql.{Date, Timestamp}\n\nimport com.github.mrpowers.spark.daria.sql.SparkSessio"
},
{
"path": "src/test/scala/com/singlestore/spark/ExternalHostTest.scala",
"chars": 8183,
"preview": "package com.singlestore.spark\n\nimport java.sql.PreparedStatement\nimport java.util.Properties\n\nimport com.github.mrpowers"
},
{
"path": "src/test/scala/com/singlestore/spark/IntegrationSuiteBase.scala",
"chars": 7104,
"preview": "package com.singlestore.spark\n\nimport java.sql.{Connection, DriverManager}\nimport java.util.{Properties, TimeZone}\n\nimpo"
},
{
"path": "src/test/scala/com/singlestore/spark/IssuesTest.scala",
"chars": 3602,
"preview": "package com.singlestore.spark\n\nimport com.github.mrpowers.spark.daria.sql.SparkSessionExt._\nimport org.apache.spark.sql."
},
{
"path": "src/test/scala/com/singlestore/spark/LoadDataBenchmark.scala",
"chars": 2670,
"preview": "package com.singlestore.spark\n\nimport java.sql.{Connection, Date, DriverManager}\nimport java.time.{Instant, LocalDate}\ni"
},
{
"path": "src/test/scala/com/singlestore/spark/LoadDataTest.scala",
"chars": 9811,
"preview": "package com.singlestore.spark\n\nimport com.github.mrpowers.spark.daria.sql.SparkSessionExt._\nimport org.apache.spark.sql."
},
{
"path": "src/test/scala/com/singlestore/spark/LoadbalanceTest.scala",
"chars": 2618,
"preview": "package com.singlestore.spark\n\nimport com.github.mrpowers.spark.daria.sql.SparkSessionExt._\nimport com.singlestore.spark"
},
{
"path": "src/test/scala/com/singlestore/spark/MaxErrorsTest.scala",
"chars": 2532,
"preview": "package com.singlestore.spark\n\nimport java.sql.SQLTransientConnectionException\n\nimport com.github.mrpowers.spark.daria.s"
},
{
"path": "src/test/scala/com/singlestore/spark/OutputMetricsTest.scala",
"chars": 1671,
"preview": "package com.singlestore.spark\n\nimport java.util.concurrent.CountDownLatch\nimport com.github.mrpowers.spark.daria.sql.Spa"
},
{
"path": "src/test/scala/com/singlestore/spark/ReferenceTableTest.scala",
"chars": 2867,
"preview": "package com.singlestore.spark\n\nimport com.github.mrpowers.spark.daria.sql.SparkSessionExt._\nimport org.apache.spark.sql."
},
{
"path": "src/test/scala/com/singlestore/spark/SQLHelperTest.scala",
"chars": 6962,
"preview": "package com.singlestore.spark\n\nimport java.sql.{Date, Timestamp}\n\nimport com.github.mrpowers.spark.daria.sql.SparkSessio"
},
{
"path": "src/test/scala/com/singlestore/spark/SQLOverwriteTest.scala",
"chars": 12576,
"preview": "package com.singlestore.spark\n\nimport com.github.mrpowers.spark.daria.sql.SparkSessionExt._\nimport com.singlestore.spark"
},
{
"path": "src/test/scala/com/singlestore/spark/SQLPermissionsTest.scala",
"chars": 5094,
"preview": "package com.singlestore.spark\n\nimport java.util.UUID\nimport com.github.mrpowers.spark.daria.sql.SparkSessionExt._\nimport"
},
{
"path": "src/test/scala/com/singlestore/spark/SQLPushdownTest.scala",
"chars": 158839,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.{Relation, SinglestoreVersion}\nimport com.singlestore"
},
{
"path": "src/test/scala/com/singlestore/spark/SanityTest.scala",
"chars": 12998,
"preview": "package com.singlestore.spark\n\nimport java.sql.SQLSyntaxErrorException\nimport com.github.mrpowers.spark.daria.sql.SparkS"
},
{
"path": "src/test/scala/com/singlestore/spark/SinglestoreConnectionPoolTest.scala",
"chars": 4617,
"preview": "package com.singlestore.spark\n\nimport java.sql.Connection\nimport java.time.Duration\nimport java.util.Properties\n\nimport "
},
{
"path": "src/test/scala/com/singlestore/spark/SinglestoreOptionsTest.scala",
"chars": 2795,
"preview": "package com.singlestore.spark\n\nimport org.apache.spark.sql.catalyst.util.CaseInsensitiveMap\n\nclass SinglestoreOptionsTes"
},
{
"path": "src/test/scala/com/singlestore/spark/TestHelper.scala",
"chars": 441,
"preview": "package com.singlestore.spark\n\nimport java.sql.SQLException\n\nimport scala.annotation.tailrec\n\nobject TestHelper {\n\n @ta"
},
{
"path": "src/test/scala/com/singlestore/spark/VersionTest.scala",
"chars": 561,
"preview": "package com.singlestore.spark\n\nimport com.singlestore.spark.SQLGen.SinglestoreVersion\nimport org.scalatest.funspec.AnyFu"
}
]
// ... and 1 more files (download for full content)
About this extraction
This page contains the full source code of the memsql/memsql-spark-connector GitHub repository, extracted and formatted as plain text for AI agents and large language models (LLMs). The extraction includes 125 files (1.2 MB), approximately 347.0k tokens. Use this with OpenClaw, Claude, ChatGPT, Cursor, Windsurf, or any other AI tool that accepts text input. You can copy the full output to your clipboard or download it as a .txt file.
Extracted by GitExtract — free GitHub repo to text converter for AI. Built by Nikandr Surkov.