├── .github ├── ISSUE_TEMPLATE │ ├── bug_report.md │ ├── enhancement.md │ ├── feature_request.md │ ├── general-question.md │ └── proposal.md ├── license-checker-config.yml ├── pull_request_template.md └── workflows │ ├── TLS-test.yml │ ├── alter-primary-key-false-test.yml │ ├── changelog.yml │ ├── codeql-analysis.yml │ ├── followerread.yml │ ├── issue-check.yaml │ ├── license-check.yml │ └── verify.yml ├── .gitignore ├── .gitmodules ├── .lift.toml ├── .travis.yml ├── CHANGELOG.md ├── LICENSE ├── OWNERS ├── README.md ├── assembly ├── pom.xml └── src │ └── main │ └── assembly │ └── assembly.xml ├── config ├── cert │ ├── jks │ │ ├── client-keystore │ │ └── server-cert-store │ └── pem │ │ ├── client-pkcs8.key │ │ ├── client.key │ │ ├── client.pem │ │ ├── openssl.cnf │ │ ├── pd.key │ │ ├── pd.pem │ │ ├── root.key │ │ ├── root.pem │ │ ├── root.srl │ │ ├── tidb.key │ │ ├── tidb.pem │ │ ├── tikv.key │ │ └── tikv.pem ├── followerread │ ├── pd.toml │ ├── tidb.toml │ └── tikv.toml ├── hive-site.xml.template ├── pd-4.0.toml ├── pd-TLS.toml ├── pd.toml ├── tidb-4.0.toml ├── tidb-TLS.toml ├── tidb.toml ├── tiflash-daily-test.toml ├── tiflash-learner-daily-test.toml ├── tiflash-learner.toml ├── tiflash.toml ├── tikv-4.0.toml ├── tikv-TLS.toml └── tikv.toml ├── core-test ├── pom.xml └── src │ └── KEEPME ├── core ├── pom.xml ├── scripts │ ├── DumpHiveMetastore.sql │ ├── fetch-test-data.sh │ └── version.sh └── src │ ├── main │ ├── java │ │ └── com │ │ │ └── pingcap │ │ │ └── tikv │ │ │ ├── columnar │ │ │ ├── TiColumnVectorAdapter.java │ │ │ └── TiColumnarBatchHelper.java │ │ │ ├── datatype │ │ │ └── TypeMapping.java │ │ │ └── hostmap │ │ │ └── UriHostMapping.java │ ├── resources │ │ ├── META-INF │ │ │ └── services │ │ │ │ └── org.apache.spark.sql.sources.DataSourceRegister │ │ └── log4j.properties.template │ └── scala │ │ ├── com │ │ └── pingcap │ │ │ └── tispark │ │ │ ├── DefaultSource.scala │ │ │ ├── MetaManager.scala │ │ │ ├── TiConfigConst.scala │ │ │ ├── TiDBRelation.scala │ │ │ ├── TiDBUtils.scala │ │ │ ├── TiPartition.scala │ │ │ ├── TiSparkInfo.scala │ │ │ ├── TiTableReference.scala │ │ │ ├── accumulator │ │ │ └── CacheInvalidateAccumulator.scala │ │ │ ├── auth │ │ │ ├── MySQLPriv.scala │ │ │ └── TiAuthorization.scala │ │ │ ├── handler │ │ │ └── CacheInvalidateEventHandler.scala │ │ │ ├── listener │ │ │ ├── CacheInvalidateListener.scala │ │ │ └── PDCacheInvalidateListener.scala │ │ │ ├── safepoint │ │ │ └── ServiceSafePoint.scala │ │ │ ├── statistics │ │ │ ├── StatisticsHelper.scala │ │ │ ├── StatisticsManager.scala │ │ │ └── estimate │ │ │ │ └── TableSizeEstimator.scala │ │ │ ├── telemetry │ │ │ ├── TeleMsg.scala │ │ │ ├── Telemetry.scala │ │ │ ├── TelemetryRule.scala │ │ │ ├── TiSparkTeleConf.scala │ │ │ └── TiSparkTeleInfo.scala │ │ │ ├── utils │ │ │ ├── HttpClientUtil.scala │ │ │ ├── ReflectionUtil.scala │ │ │ ├── SystemInfoUtil.scala │ │ │ ├── TiUtil.scala │ │ │ ├── TwoPhaseCommitHepler.scala │ │ │ └── WriteUtil.scala │ │ │ ├── v2 │ │ │ ├── TiDBTable.scala │ │ │ ├── TiDBTableProvider.scala │ │ │ ├── TiDBTableScan.scala │ │ │ ├── TiDBTableScanBuilder.scala │ │ │ └── sink │ │ │ │ ├── TiDBBatchWrite.scala │ │ │ │ ├── TiDBDataWrite.scala │ │ │ │ └── TiDBDataWriterFactory.scala │ │ │ └── write │ │ │ ├── DBTable.scala │ │ │ ├── EncodedKVPair.scala │ │ │ ├── SerializableKey.scala │ │ │ ├── TiBatchWrite.scala │ │ │ ├── TiBatchWriteTable.scala │ │ │ ├── TiDBDelete.scala │ │ │ ├── TiDBOptions.scala │ │ │ ├── TiDBWriter.scala │ │ │ ├── TiReginSplitPartitioner.scala │ │ │ ├── WrappedEncodedRow.scala │ │ │ └── WrappedRow.scala │ │ └── org │ │ └── apache │ │ └── spark │ │ └── sql │ │ ├── TiContext.scala │ │ ├── TiExtensions.scala │ │ ├── catalyst │ │ ├── catalog │ │ │ └── TiCatalog.scala │ │ ├── expressions │ │ │ ├── BasicExpression.scala │ │ │ ├── TiExprUtils.scala │ │ │ └── aggregate │ │ │ │ └── SpecialSum.scala │ │ ├── parser │ │ │ ├── TiParser.scala │ │ │ └── TiParserFactory.scala │ │ ├── planner │ │ │ ├── TiAggregation.scala │ │ │ ├── TiAggregationImpl.scala │ │ │ └── TiStrategyFactory.scala │ │ ├── plans │ │ │ └── logical │ │ │ │ └── BasicLogicalPlan.scala │ │ └── rule │ │ │ ├── TiAuthRuleFactory.scala │ │ │ ├── TiAuthorizationRule.scala │ │ │ └── TiStatisticsRuleFactory.scala │ │ ├── execution │ │ ├── CoprocessorRDD.scala │ │ └── TiConverter.scala │ │ └── tispark │ │ ├── TiHandleRDD.scala │ │ ├── TiRDD.scala │ │ └── TiRowRDD.scala │ └── test │ ├── Readme.md │ ├── resources │ ├── issue │ │ └── LargeColumn.sql │ ├── prefix-index │ │ ├── PrefixTest.sql │ │ └── UTF8Test.sql │ ├── resolveLock-test │ │ ├── 1_account.sql │ │ ├── 1_give.sql │ │ ├── 2_get.sql │ │ ├── ddl.sql │ │ ├── q1.sql │ │ ├── q2.sql │ │ └── sum_account.sql │ ├── tidb_config.properties.TLS.template │ ├── tidb_config.properties.template │ ├── tidb_config.properties.withoutdataload.template │ ├── tpcds-sql │ │ ├── q1.sql │ │ ├── q10.sql │ │ ├── q11.sql │ │ ├── q12.sql │ │ ├── q13.sql │ │ ├── q14a.sql │ │ ├── q14b.sql │ │ ├── q15.sql │ │ ├── q16.sql │ │ ├── q17.sql │ │ ├── q18.sql │ │ ├── q19.sql │ │ ├── q2.sql │ │ ├── q20.sql │ │ ├── q21.sql │ │ ├── q22.sql │ │ ├── q23a.sql │ │ ├── q23b.sql │ │ ├── q24a.sql │ │ ├── q24b.sql │ │ ├── q25.sql │ │ ├── q26.sql │ │ ├── q27.sql │ │ ├── q28.sql │ │ ├── q29.sql │ │ ├── q3.sql │ │ ├── q30.sql │ │ ├── q31.sql │ │ ├── q32.sql │ │ ├── q33.sql │ │ ├── q34.sql │ │ ├── q35.sql │ │ ├── q36.sql │ │ ├── q37.sql │ │ ├── q38.sql │ │ ├── q39a.sql │ │ ├── q39b.sql │ │ ├── q4.sql │ │ ├── q40.sql │ │ ├── q41.sql │ │ ├── q42.sql │ │ ├── q43.sql │ │ ├── q44.sql │ │ ├── q45.sql │ │ ├── q46.sql │ │ ├── q47.sql │ │ ├── q48.sql │ │ ├── q49.sql │ │ ├── q5.sql │ │ ├── q50.sql │ │ ├── q51.sql │ │ ├── q52.sql │ │ ├── q53.sql │ │ ├── q54.sql │ │ ├── q55.sql │ │ ├── q56.sql │ │ ├── q57.sql │ │ ├── q58.sql │ │ ├── q59.sql │ │ ├── q6.sql │ │ ├── q60.sql │ │ ├── q61.sql │ │ ├── q62.sql │ │ ├── q63.sql │ │ ├── q64.sql │ │ ├── q65.sql │ │ ├── q66.sql │ │ ├── q67.sql │ │ ├── q68.sql │ │ ├── q69.sql │ │ ├── q7.sql │ │ ├── q70.sql │ │ ├── q71.sql │ │ ├── q72.sql │ │ ├── q73.sql │ │ ├── q74.sql │ │ ├── q75.sql │ │ ├── q76.sql │ │ ├── q77.sql │ │ ├── q78.sql │ │ ├── q79.sql │ │ ├── q8.sql │ │ ├── q80.sql │ │ ├── q81.sql │ │ ├── q82.sql │ │ ├── q83.sql │ │ ├── q84.sql │ │ ├── q85.sql │ │ ├── q86.sql │ │ ├── q87.sql │ │ ├── q88.sql │ │ ├── q89.sql │ │ ├── q9.sql │ │ ├── q90.sql │ │ ├── q91.sql │ │ ├── q92.sql │ │ ├── q93.sql │ │ ├── q94.sql │ │ ├── q95.sql │ │ ├── q96.sql │ │ ├── q97.sql │ │ ├── q98.sql │ │ ├── q99.sql │ │ ├── ss_max.sql │ │ └── ss_maxb.sql │ └── tpch-sql │ │ ├── q1.sql │ │ ├── q10.sql │ │ ├── q11.sql │ │ ├── q12.sql │ │ ├── q13.sql │ │ ├── q14.sql │ │ ├── q15.sql │ │ ├── q16.sql │ │ ├── q17.sql │ │ ├── q18.sql │ │ ├── q19.sql │ │ ├── q2.sql │ │ ├── q20.sql │ │ ├── q21.sql │ │ ├── q22.sql │ │ ├── q3.sql │ │ ├── q4.sql │ │ ├── q5.sql │ │ ├── q6.sql │ │ ├── q7.sql │ │ ├── q8.sql │ │ └── q9.sql │ └── scala │ ├── com │ └── pingcap │ │ └── tispark │ │ ├── BatchWriteIssueSuite.scala │ │ ├── TiBatchWriteSuite.scala │ │ ├── UnitSuite.scala │ │ ├── auth │ │ ├── TiAuthIntegrationSuite.scala │ │ └── TiAuthSuite.scala │ │ ├── collation │ │ └── CollationSuite.scala │ │ ├── concurrency │ │ ├── ConcurrencyTest.scala │ │ ├── WriteDDLConflictSuite.scala │ │ ├── WriteDDLNotConflictSuite.scala │ │ ├── WriteReadSuite.scala │ │ └── WriteWriteConflictSuite.scala │ │ ├── convert │ │ ├── ToBitSuite.scala │ │ ├── ToBytesSuite.scala │ │ ├── ToDateSuite.scala │ │ ├── ToDateTimeSuite.scala │ │ ├── ToDecimalSuite.scala │ │ ├── ToEnumSuite.scala │ │ ├── ToRealSuite.scala │ │ ├── ToSignedSuite.scala │ │ ├── ToStringSuite.scala │ │ ├── ToTimestampSuite.scala │ │ └── ToUnsignedSuite.scala │ │ ├── datasource │ │ ├── AddingIndexInsertSuite.scala │ │ ├── AddingIndexReplaceSuite.scala │ │ ├── AutoIncrementSuite.scala │ │ ├── AutoRandomSuite.scala │ │ ├── BaseBatchWriteTest.scala │ │ ├── BaseBatchWriteWithoutDropTableTest.scala │ │ ├── BaseDataSourceTest.scala │ │ ├── BasicBatchWriteSuite.scala │ │ ├── BasicSQLSuite.scala │ │ ├── CheckUnsupportedSuite.scala │ │ ├── ColumnMappingSuite.scala │ │ ├── EdgeConditionSuite.scala │ │ ├── ExceptionTestSuite.scala │ │ ├── FilterPushdownSuite.scala │ │ ├── InsertSuite.scala │ │ ├── MissingParameterSuite.scala │ │ ├── OnlyOnePkSuite.scala │ │ ├── RowIDAllocatorSuite.scala │ │ ├── ShardRowIDBitsSuite.scala │ │ ├── TiSparkTypeSuite.scala │ │ └── UpperCaseColumnNameSuite.scala │ │ ├── datatype │ │ ├── BatchWriteDataTypeSuite.scala │ │ └── DecimalTypeSuite.scala │ │ ├── delete │ │ ├── DeleteCompatibilitySuite.scala │ │ ├── DeleteConcurrencyTestSuite.scala │ │ ├── DeleteNotSupportSuite.scala │ │ └── DeleteWhereClauseSuite.scala │ │ ├── hostmap │ │ └── HostMappingSuite.scala │ │ ├── index │ │ └── LineItemSuite.scala │ │ ├── insert │ │ └── InsertSuite.scala │ │ ├── multitable │ │ └── MultiTableWriteSuite.scala │ │ ├── overflow │ │ ├── BitOverflowSuite.scala │ │ ├── BytesOverflowSuite.scala │ │ ├── DateOverflowSuite.scala │ │ ├── DateTimeOverflowSuite.scala │ │ ├── DecimalOverflowSuite.scala │ │ ├── EnumOverflowSuite.scala │ │ ├── SignedOverflowSuite.scala │ │ ├── StringOverflowSuite.scala │ │ └── UnsignedOverflowSuite.scala │ │ ├── partition │ │ ├── PartitionBaseSuite.scala │ │ ├── PartitionNewCollationSuite.scala │ │ └── PartitionWriteSuite.scala │ │ ├── telemetry │ │ └── TelemetrySuite.scala │ │ ├── tiflash │ │ └── TiFlashSuite.scala │ │ ├── tls │ │ ├── CheckTLSEnable.scala │ │ ├── JDBCTLSSuite.scala │ │ ├── TiKVClientTLSSuite.scala │ │ └── TiSparkTLSSuite.scala │ │ └── ttl │ │ ├── InitialTTLExpiredSuite.scala │ │ └── LockTimeoutSuite.scala │ └── org │ └── apache │ └── spark │ ├── SharedSparkContext.scala │ ├── SparkFunSuite.scala │ └── sql │ ├── AlterTableTestSuite.scala │ ├── BaseInitialOnceTest.scala │ ├── BaseTestGenerationSpec.scala │ ├── BaseTiSparkEnableBatchWriteTest.scala │ ├── BaseTiSparkTest.scala │ ├── ExpressionIndexSuite.scala │ ├── InvisibleIndexSuite.scala │ ├── IssueTestSuite.scala │ ├── MultipleSparkSessionTestSuite.scala │ ├── MysqlDataTypeSuite.scala │ ├── OutputOffsetsSuite.scala │ ├── PartitionTableSuite.scala │ ├── QueryTest.scala │ ├── SequenceTestSuite.scala │ ├── SparkDataTypeTestSuite.scala │ ├── TiDBMapDatabaseSuite.scala │ ├── TiSparkTestSpec.scala │ ├── TimezoneTestSuite.scala │ ├── ViewTestSuite.scala │ ├── benchmark │ ├── TPCDSQuerySuite.scala │ └── TPCHQuerySuite.scala │ ├── catalyst │ ├── catalog │ │ └── CatalogTestSuite.scala │ └── plans │ │ ├── BasePlanTest.scala │ │ ├── logical │ │ └── LogicalPlanTestSuite.scala │ │ └── statistics │ │ └── StatisticsTestSuite.scala │ ├── clustered │ ├── ClusteredIndexTest.scala │ ├── IndexScan0Suite.scala │ ├── IndexScan1Suite.scala │ ├── TableScan0Suite.scala │ └── TableScan1Suite.scala │ ├── expression │ ├── Aggregate0Suite.scala │ ├── ArithmeticAgg0Suite.scala │ ├── ArithmeticTest0Suite.scala │ ├── ArithmeticTest1Suite.scala │ ├── ArithmeticTest2Suite.scala │ ├── Between0Suite.scala │ ├── ComplexAggregateSuite.scala │ ├── ComplexGroupBySuite.scala │ ├── Count0Suite.scala │ ├── Distinct0Suite.scala │ ├── FirstLast0Suite.scala │ ├── Having0Suite.scala │ ├── InTest0Suite.scala │ ├── LikeTestSuite.scala │ ├── OtherTestSuite.scala │ ├── Union0Suite.scala │ └── index │ │ ├── Between0Suite.scala │ │ ├── ComprehensiveSuite.scala │ │ ├── CoveringIndex0Suite.scala │ │ ├── InTest0Suite.scala │ │ ├── Join0Suite.scala │ │ ├── PrefixIndexTestSuite.scala │ │ ├── Special0Suite.scala │ │ └── UnsignedTestSuite.scala │ ├── followerread │ └── FollowerReadSuite.scala │ ├── insertion │ ├── BatchWritePKAndIndexSuite.scala │ ├── BatchWritePkSuite.scala │ └── BatchWriteUniqueIndexSuite.scala │ ├── pushdown │ ├── AvgPushDownSuite.scala │ ├── BasePushDownSuite.scala │ ├── CountPushDownSuite.scala │ └── SumPushDownSuite.scala │ ├── staleread │ └── StaleReadSuite.scala │ ├── statistics │ └── StatisticsManagerSuite.scala │ ├── test │ ├── SharedSQLContext.scala │ ├── TestConstants.scala │ ├── TestSparkSession.scala │ └── Utils.scala │ ├── txn │ └── TxnTestSuite.scala │ └── types │ ├── BaseRandomDataTypeTest.scala │ ├── DataTypeNormalSuite.scala │ ├── MultiColumnDataTypeSuite.scala │ ├── SpecialTiDBTypeTestSuite.scala │ └── pk │ ├── ClusterIndexScanSuites.scala │ ├── DataTypePKSuite.scala │ ├── MultiColumnClusterIndexDataSuite00.scala │ ├── MultiColumnPKDataTypeSuite00.scala │ ├── MultiColumnPKDataTypeSuite01.scala │ ├── MultiColumnPKDataTypeSuite02.scala │ ├── MultiColumnPKDataTypeSuite03.scala │ ├── MultiColumnPKDataTypeSuite04.scala │ ├── MultiColumnPKDataTypeSuite05.scala │ ├── MultiColumnPKDataTypeSuite06.scala │ ├── MultiColumnPKDataTypeSuite07.scala │ ├── MultiColumnPKDataTypeSuite08.scala │ ├── MultiColumnPKDataTypeSuite09.scala │ ├── MultiColumnPKDataTypeSuite10.scala │ ├── MultiColumnPKDataTypeSuite11.scala │ ├── MultiColumnPKDataTypeSuite12.scala │ ├── MultiColumnPKDataTypeSuite13.scala │ ├── MultiColumnPKDataTypeSuite14.scala │ ├── MultiColumnPKDataTypeSuite15.scala │ ├── MultiColumnPKDataTypeSuite16.scala │ ├── MultiColumnPKDataTypeSuite17.scala │ ├── MultiColumnPKDataTypeSuite18.scala │ ├── MultiColumnPKDataTypeSuite19.scala │ ├── MultiColumnPKDataTypeSuite20.scala │ ├── MultiColumnPKDataTypeSuite21.scala │ ├── MultiColumnPKDataTypeSuite22.scala │ ├── MultiColumnPKDataTypeSuite23.scala │ ├── MultiColumnPKDataTypeSuite24.scala │ ├── MultiColumnPKDataTypeSuite25.scala │ ├── MultiColumnPKDataTypeSuite26.scala │ ├── MultiColumnPKDataTypeSuite27.scala │ ├── MultiColumnPKDataTypeSuite28.scala │ ├── MultiColumnPKDataTypeSuite29.scala │ ├── MultiColumnPKDataTypeSuite30.scala │ ├── MultiColumnPKDataTypeSuite31.scala │ ├── MultiColumnPKDataTypeSuite32.scala │ ├── MultiColumnPKDataTypeSuite33.scala │ ├── MultiColumnPKDataTypeSuite34.scala │ ├── MultiColumnPKDataTypeSuite35.scala │ └── MultiColumnPKDataTypeSuites.scala ├── db-random-test ├── pom.xml └── src │ ├── main │ └── scala │ │ └── com │ │ └── pingcap │ │ └── tispark │ │ └── test │ │ ├── RandomTest.scala │ │ └── generator │ │ ├── ColumnValueGenerator.scala │ │ ├── DataGenerator.scala │ │ ├── DataType.scala │ │ ├── GeneratorUtils.scala │ │ ├── Index.scala │ │ ├── IndexColumn.scala │ │ ├── NullableType.scala │ │ ├── Schema.scala │ │ └── SchemaAndData.scala │ └── test │ └── scala │ └── com │ └── pingcap │ └── tispark │ └── test │ ├── RandomTestSuite.scala │ └── generator │ └── DataGeneratorSuite.scala ├── dev ├── .scalafmt.conf ├── README.md ├── intellij-java-google-style.xml ├── javafmt └── scalafmt ├── docker-compose-4.0.yaml ├── docker-compose-TiDB-TLS.yaml ├── docker-compose.yaml ├── docs ├── architecture.png ├── design │ ├── 2022-02-28-dsv2-support.md │ ├── 2022-03-21-delete-support.md │ ├── 2022-05-09-staleread-support.md │ ├── 2022-06-15-phyical_plan_explain.md │ ├── 2022-06-30-partition-table-batchwrite-suppport.md │ ├── 2022-07-20-insert-support.md │ ├── 2022-08-11-normalize-client.md │ ├── 2022-09-16-follower-read.md │ ├── TEMPLATE.md │ └── imgs │ │ ├── delete-support │ │ ├── delete_step │ │ ├── delete_step.png │ │ ├── rowid │ │ └── rowid.png │ │ ├── dsv2-support │ │ ├── dsv2.png │ │ ├── new_write │ │ ├── new_write.png │ │ └── write.png │ │ ├── phyical_plan_explain │ │ └── code_flow_chart.png │ │ └── stale-read │ │ ├── meta │ │ └── meta.png ├── dev-guide │ ├── build │ │ ├── get_the_code_build_and_test.md │ │ ├── install_scala_java_spark_maven.md │ │ └── start_tidb_cluster.md │ ├── contribute │ │ ├── code_style.md │ │ ├── commit_the_code_and_submit_a_pull_request.md │ │ └── report_an_issue.md │ ├── pics │ │ ├── conflict.png │ │ ├── fork.png │ │ ├── issue_choose.png │ │ ├── issue_template.png │ │ ├── java_format.png │ │ ├── overview.png │ │ ├── pr.png │ │ ├── pr_templete.png │ │ └── scala_format.png │ ├── tispark_development_guide.md │ └── understand │ │ ├── overview.md │ │ └── write.md ├── features │ ├── authorization_userguide.md │ ├── datasource_api_userguide.md │ ├── delete_userguide.md │ ├── follower_read.md │ ├── insert_sql_userguide.md │ ├── partition_table.md │ ├── push_down.md │ ├── query_execution_plan_in_TiSpark.md │ ├── service_safe_point.md │ ├── stale_read.md │ └── telemetry.md ├── how_to_use_tidb_as_metastore_db.md ├── userguide_2.0.md └── userguide_3.0.md ├── pom.xml ├── spark-wrapper ├── spark-3.0 │ ├── pom.xml │ └── src │ │ └── main │ │ └── scala │ │ ├── com │ │ └── pingcap │ │ │ └── tispark │ │ │ └── SparkWrapper.scala │ │ └── org │ │ └── apache │ │ └── spark │ │ └── sql │ │ ├── catalyst │ │ ├── expressions │ │ │ └── TiBasicExpression.scala │ │ └── plans │ │ │ └── logical │ │ │ └── TiBasicLogicalPlan.scala │ │ ├── connector │ │ └── write │ │ │ └── TiDBWriteBuilder.scala │ │ └── extensions │ │ ├── TiAggregationProjection.scala │ │ └── TiStrategy.scala ├── spark-3.1 │ ├── pom.xml │ └── src │ │ └── main │ │ └── scala │ │ ├── com │ │ └── pingcap │ │ │ └── tispark │ │ │ └── SparkWrapper.scala │ │ └── org │ │ └── apache │ │ └── spark │ │ └── sql │ │ ├── catalyst │ │ ├── expressions │ │ │ └── TiBasicExpression.scala │ │ └── plans │ │ │ └── logical │ │ │ └── TiBasicLogicalPlan.scala │ │ ├── connector │ │ └── write │ │ │ └── TiDBWriteBuilder.scala │ │ └── extensions │ │ ├── TiAggregationProjectionV2.scala │ │ └── TiStrategy.scala ├── spark-3.2 │ ├── pom.xml │ └── src │ │ └── main │ │ └── scala │ │ ├── com │ │ └── pingcap │ │ │ └── tispark │ │ │ └── SparkWrapper.scala │ │ └── org │ │ └── apache │ │ └── spark │ │ └── sql │ │ ├── catalyst │ │ ├── expressions │ │ │ └── TiBasicExpression.scala │ │ └── plans │ │ │ └── logical │ │ │ └── TiBasicLogicalPlan.scala │ │ ├── connector │ │ └── write │ │ │ └── TiDBWriteBuilder.scala │ │ └── extensions │ │ ├── TiAggregationProjectionV2.scala │ │ └── TiStrategy.scala ├── spark-3.3 │ ├── pom.xml │ └── src │ │ └── main │ │ └── scala │ │ ├── com │ │ └── pingcap │ │ │ └── tispark │ │ │ └── SparkWrapper.scala │ │ └── org │ │ └── apache │ │ └── spark │ │ └── sql │ │ ├── catalyst │ │ ├── expressions │ │ │ └── TiBasicExpression.scala │ │ └── plans │ │ │ └── logical │ │ │ └── TiBasicLogicalPlan.scala │ │ ├── connector │ │ └── write │ │ │ └── TiDBWriteBuilder.scala │ │ └── extensions │ │ ├── TiAggregationProjectionV2.scala │ │ └── TiStrategy.scala └── spark-3.4 │ ├── pom.xml │ └── src │ └── main │ └── scala │ ├── com │ └── pingcap │ │ └── tispark │ │ └── SparkWrapper.scala │ └── org │ └── apache │ └── spark │ └── sql │ ├── catalyst │ ├── expressions │ │ └── TiBasicExpression.scala │ └── plans │ │ └── logical │ │ └── TiBasicLogicalPlan.scala │ ├── connector │ └── write │ │ └── TiDBWriteBuilder.scala │ └── extensions │ ├── TiAggregationProjectionV2.scala │ └── TiStrategy.scala └── tikv-client ├── BUILD ├── LICENSE ├── Makefile ├── README.md ├── WORKSPACE ├── pom.xml ├── scripts └── proto.sh ├── shading_rule └── src ├── main ├── java │ └── com │ │ └── pingcap │ │ └── tikv │ │ ├── BUILD │ │ ├── ClientSession.java │ │ ├── Main.java │ │ ├── ReplicaReadPolicy.java │ │ ├── Snapshot.java │ │ ├── TiConfiguration.java │ │ ├── TiDBJDBCClient.java │ │ ├── TiFlashClient.java │ │ ├── allocator │ │ └── RowIDAllocator.java │ │ ├── catalog │ │ ├── Catalog.java │ │ └── CatalogTransaction.java │ │ ├── codec │ │ ├── Codec.java │ │ ├── CodecDataInput.java │ │ ├── CodecDataInputLittleEndian.java │ │ ├── CodecDataOutput.java │ │ ├── CodecDataOutputLittleEndian.java │ │ ├── ExtendedDateTime.java │ │ ├── KeyUtils.java │ │ ├── MetaCodec.java │ │ ├── MyDecimal.java │ │ ├── RowDecoderV2.java │ │ ├── RowEncoderV2.java │ │ ├── RowV2.java │ │ ├── TableCodec.java │ │ ├── TableCodecV1.java │ │ └── TableCodecV2.java │ │ ├── columnar │ │ ├── BatchedTiChunkColumnVector.java │ │ ├── TiBlockColumnVector.java │ │ ├── TiChunk.java │ │ ├── TiChunkColumnVector.java │ │ ├── TiColumnVector.java │ │ ├── TiCoreTime.java │ │ ├── TiRowColumnVector.java │ │ └── datatypes │ │ │ ├── AutoGrowByteBuffer.java │ │ │ ├── CHType.java │ │ │ ├── CHTypeDate.java │ │ │ ├── CHTypeDateTime.java │ │ │ ├── CHTypeDecimal.java │ │ │ ├── CHTypeFixedString.java │ │ │ ├── CHTypeMyDate.java │ │ │ ├── CHTypeMyDateTime.java │ │ │ ├── CHTypeNumber.java │ │ │ └── CHTypeString.java │ │ ├── expression │ │ ├── AggregateFunction.java │ │ ├── ArithmeticBinaryExpression.java │ │ ├── Blocklist.java │ │ ├── ByItem.java │ │ ├── ColumnRef.java │ │ ├── ComparisonBinaryExpression.java │ │ ├── Constant.java │ │ ├── Expression.java │ │ ├── ExpressionBlocklist.java │ │ ├── FuncCallExpr.java │ │ ├── FuncCallExprEval.java │ │ ├── IsNull.java │ │ ├── LogicalBinaryExpression.java │ │ ├── Not.java │ │ ├── PartitionPruner.java │ │ ├── RangeColumnPartitionPruner.java │ │ ├── RangePartitionPruner.java │ │ ├── StringRegExpression.java │ │ ├── TypeBlocklist.java │ │ ├── Visitor.java │ │ └── visitor │ │ │ ├── ColumnMatcher.java │ │ │ ├── DefaultVisitor.java │ │ │ ├── IndexMatcher.java │ │ │ ├── IndexRangeSetBuilder.java │ │ │ ├── MetaResolver.java │ │ │ ├── PartAndFilterExprRewriter.java │ │ │ ├── ProtoConverter.java │ │ │ ├── PrunedPartitionBuilder.java │ │ │ ├── PseudoCostCalculator.java │ │ │ ├── RangePartitionLocator.java │ │ │ ├── RangeSetBuilder.java │ │ │ └── SupportedExpressionValidator.java │ │ ├── handle │ │ ├── CommonHandle.java │ │ ├── Handle.java │ │ └── IntHandle.java │ │ ├── jdbc │ │ ├── JDBCClient.java │ │ └── RowMapper.java │ │ ├── key │ │ ├── CommonHandleScanKeyRangeBuilder.java │ │ ├── CompoundKey.java │ │ ├── IndexKey.java │ │ ├── IndexScanKeyRangeBuilder.java │ │ ├── Key.java │ │ ├── KeyRangeBuilder.java │ │ ├── RowKey.java │ │ ├── StatisticsKeyRangeBuilder.java │ │ └── TypedKey.java │ │ ├── meta │ │ ├── CIStr.java │ │ ├── Collation.java │ │ ├── IndexType.java │ │ ├── SchemaState.java │ │ ├── TiColumnInfo.java │ │ ├── TiDAGRequest.java │ │ ├── TiDBInfo.java │ │ ├── TiFlashReplicaInfo.java │ │ ├── TiIndexColumn.java │ │ ├── TiIndexInfo.java │ │ ├── TiPartitionDef.java │ │ ├── TiPartitionInfo.java │ │ ├── TiSequenceInfo.java │ │ ├── TiTableInfo.java │ │ ├── TiUserIdentity.java │ │ ├── TiViewInfo.java │ │ └── collate │ │ │ ├── BinPaddingCollator.java │ │ │ ├── GeneralCICollator.java │ │ │ └── UnicodeCICollator.java │ │ ├── operation │ │ ├── SchemaInfer.java │ │ └── iterator │ │ │ ├── ChunkIterator.java │ │ │ ├── ConcreteScanIterator.java │ │ │ ├── CoprocessorIterator.java │ │ │ ├── DAGIterator.java │ │ │ ├── IndexScanIterator.java │ │ │ └── ScanIterator.java │ │ ├── parser │ │ ├── AstBuilder.java │ │ ├── CaseChangingCharStream.java │ │ ├── MySqlLexer.g4 │ │ ├── MySqlParser.g4 │ │ ├── PartitionWriteAstBuilder.java │ │ └── TiParser.java │ │ ├── partition │ │ ├── PartitionExpression.java │ │ ├── PartitionedTable.java │ │ └── TableCommon.java │ │ ├── predicates │ │ ├── IndexRange.java │ │ ├── PredicateUtils.java │ │ ├── ScanSpec.java │ │ ├── SelectivityCalculator.java │ │ └── TiKVScanAnalyzer.java │ │ ├── row │ │ ├── DefaultRowReader.java │ │ ├── ObjectRowImpl.java │ │ ├── Row.java │ │ ├── RowReader.java │ │ └── RowReaderFactory.java │ │ ├── statistics │ │ ├── Bucket.java │ │ ├── CMSketch.java │ │ ├── ColumnStatistics.java │ │ ├── Histogram.java │ │ ├── IndexStatistics.java │ │ └── TableStatistics.java │ │ ├── types │ │ ├── AbstractDateTimeType.java │ │ ├── BitType.java │ │ ├── BytesType.java │ │ ├── Charset.java │ │ ├── Converter.java │ │ ├── DataType.java │ │ ├── DataTypeFactory.java │ │ ├── DateTimeType.java │ │ ├── DateType.java │ │ ├── DecimalType.java │ │ ├── EnumType.java │ │ ├── IntegerType.java │ │ ├── JsonType.java │ │ ├── MySQLType.java │ │ ├── RealType.java │ │ ├── SetType.java │ │ ├── StringType.java │ │ ├── TimeType.java │ │ ├── TimestampType.java │ │ └── UninitializedType.java │ │ └── util │ │ ├── CHTypeMapping.java │ │ ├── ConvertUpstreamUtils.java │ │ ├── LogDesensitization.java │ │ ├── MemoryUtil.java │ │ ├── RangeSplitter.java │ │ └── RegionUtils.java └── resources │ └── unicode_map_table └── test ├── java └── com │ └── pingcap │ └── tikv │ ├── BUILD │ ├── TiConfigurationTest.java │ ├── allocator │ └── RowIDAllocatorTest.java │ ├── codec │ ├── CodecTest.java │ ├── KeyUtilsTest.java │ ├── MyDecimalTest.java │ ├── TableCodecTest.java │ ├── TableCodecV1Test.java │ └── TableCodecV2Test.java │ ├── collation │ └── CollationTest.java │ ├── expression │ ├── ComparisonBinaryExpressionTest.java │ └── visitor │ │ ├── PartAndFilterExprRewriterTest.java │ │ └── PartitionLocatorTest.java │ ├── key │ ├── CompoundKeyTest.java │ ├── IndexKeyTest.java │ └── KeyTest.java │ ├── meta │ ├── DBInfoTest.java │ ├── MetaUtils.java │ ├── TiDAGRequestTest.java │ └── TiTableInfoTest.java │ ├── operation │ ├── ChunkIteratorTest.java │ └── SchemaInferTest.java │ ├── parser │ └── TiParserTest.java │ ├── predicates │ ├── IndexMatcherTest.java │ ├── PredicateUtilsTest.java │ └── TiKVScanAnalyzerTest.java │ ├── rule.bzl │ ├── types │ ├── ConverterTest.java │ ├── DataTypeFactoryTest.java │ ├── DecimalTypeTest.java │ ├── IntegerTypeTest.java │ └── RealTypeTest.java │ └── util │ └── ConcreteBackOffTest.java └── resources └── log4j.properties /.github/ISSUE_TEMPLATE/bug_report.md: -------------------------------------------------------------------------------- 1 | --- 2 | name: "\U0001F41B Bug Report" 3 | about: Create a report to help us improve 4 | title: "[BUG] Title of Bug Report" 5 | labels: type/bug 6 | assignees: '' 7 | 8 | --- 9 | 10 | **Describe the bug** 11 | 12 | 13 | **What did you do** 14 | 30 | 31 | **What do you expect** 32 | 33 | 34 | **What happens instead** 35 | 36 | 37 | 41 | 42 | **Spark and TiSpark version info** 43 | 44 | 45 | 50 | -------------------------------------------------------------------------------- /.github/ISSUE_TEMPLATE/enhancement.md: -------------------------------------------------------------------------------- 1 | --- 2 | name: "\U0001F680 Enhancement" 3 | about: As a Tispark developer, I want to make an enhancement. 4 | labels: type/enhancement 5 | --- 6 | 7 | ## Enhancement -------------------------------------------------------------------------------- /.github/ISSUE_TEMPLATE/feature_request.md: -------------------------------------------------------------------------------- 1 | --- 2 | name: "\U0001F947 Feature Request" 3 | about: Suggest an idea for this project 4 | title: '' 5 | labels: type/feature-request 6 | assignees: '' 7 | 8 | --- 9 | 10 | **Is your feature request related to a problem? Please describe.** 11 | A clear and concise description of what the problem is. Ex. I'm always frustrated when [...] 12 | 13 | **Describe the solution you'd like** 14 | A clear and concise description of what you want to happen. 15 | 16 | **Describe alternatives you've considered** 17 | A clear and concise description of any alternative solutions or features you've considered. 18 | 19 | **Additional context** 20 | Add any other context or screenshots about the feature request here. 21 | -------------------------------------------------------------------------------- /.github/ISSUE_TEMPLATE/general-question.md: -------------------------------------------------------------------------------- 1 | --- 2 | name: "\U0001F914 Ask a Question or Get Support" 3 | about: I want to ask a question. 4 | labels: type/question 5 | --- 6 | 7 | ## General Question -------------------------------------------------------------------------------- /.github/ISSUE_TEMPLATE/proposal.md: -------------------------------------------------------------------------------- 1 | --- 2 | name: "Proposal" 3 | about: Propose a solution for a feature, architecture change, etc. 4 | labels: type/proposal 5 | --- 6 | 7 | 8 | 9 | ## Problem: 10 | 11 | 12 | ## Goals 13 | 14 | 15 | ## Solutions 16 | 20 | 21 | ## Design 22 | 25 | 26 | ## Implementation 27 | 31 | 32 | ## Test 33 | 37 | 38 | ## Doc 39 | -------------------------------------------------------------------------------- /.github/license-checker-config.yml: -------------------------------------------------------------------------------- 1 | header: 2 | license: 3 | spdx-id: Apache-2.0 4 | copyright-owner: PingCAP, Inc. 5 | paths: 6 | - '**/*.java' 7 | - '**/*.scala' 8 | - '**/*.py' 9 | comment: on-failure -------------------------------------------------------------------------------- /.github/pull_request_template.md: -------------------------------------------------------------------------------- 1 | ### What problem does this PR solve? 2 | 3 | 4 | ### What is changed and how it works? 5 | 6 | 7 | ### Check List 8 | 9 | Tests 10 | 11 | - Unit test 12 | - Integration test 13 | - Manual test (add detailed scripts or steps below) 14 | - No code 15 | 16 | Code changes 17 | 18 | - Has exported function/method change 19 | - Has exported variable/fields change 20 | - Has interface methods change 21 | - Has persistent data change 22 | 23 | Side effects 24 | 25 | - Possible performance regression 26 | - Increased code complexity 27 | - Breaking backward compatibility 28 | 29 | Related changes 30 | 31 | - Need to cherry-pick to the release branch 32 | - Need to update the documentation 33 | - Need to update the `tidb-ansible` repository 34 | - Need to be included in the release note 35 | -------------------------------------------------------------------------------- /.github/workflows/changelog.yml: -------------------------------------------------------------------------------- 1 | name: Update changelog manually 2 | on: 3 | workflow_dispatch: 4 | inputs: 5 | branch: 6 | description: 'branch name' 7 | required: true 8 | default: 'master' 9 | 10 | jobs: 11 | changelog: 12 | name: update changelog 13 | runs-on: ubuntu-latest 14 | steps: 15 | - uses: actions/checkout@v2 16 | with: 17 | ref: ${{ inputs.branch }} 18 | - uses: rhysd/changelog-from-release/action@v2 19 | with: 20 | file: CHANGELOG.md 21 | github_token: ${{ secrets.GITHUB_TOKEN }} 22 | 23 | -------------------------------------------------------------------------------- /.github/workflows/followerread.yml: -------------------------------------------------------------------------------- 1 | name: Follower Read test 2 | 3 | on: 4 | push: 5 | branches: [master] 6 | pull_request: 7 | branches: [master] 8 | 9 | jobs: 10 | followerread-test: 11 | runs-on: ubuntu-latest 12 | steps: 13 | - name: checkout 14 | uses: actions/checkout@v2 15 | - name: set up JDK 16 | uses: actions/setup-java@v3 17 | with: 18 | java-version: '8' 19 | distribution: 'adopt' 20 | cache: maven 21 | 22 | - name: deploy TiUP 23 | run: curl --proto '=https' --tlsv1.2 -sSf https://tiup-mirrors.pingcap.com/install.sh | sh 24 | - name: deploy TiDB 25 | run: /home/runner/.tiup/bin/tiup playground --db.config config/followerread/tidb.toml --kv.config config/followerread/tikv.toml --pd.config config/followerread/pd.toml --kv 3 --without-monitor & 26 | 27 | - name: copy properties 28 | run: cp core/src/test/resources/tidb_config.properties.withoutdataload.template core/src/test/resources/tidb_config.properties 29 | 30 | - name: build 31 | run: mvn clean package -Dmaven.test.skip=true -B 32 | 33 | - name: test follower read 34 | run: mvn test -e -am -pl core -Dtest=moo -DwildcardSuites=org.apache.spark.sql.followerread.FollowerReadSuite -DfailIfNoTests=false -------------------------------------------------------------------------------- /.github/workflows/issue-check.yaml: -------------------------------------------------------------------------------- 1 | name: Close inactive issues 2 | on: 3 | schedule: 4 | - cron: "30 1 * * *" 5 | 6 | jobs: 7 | close-issues: 8 | runs-on: ubuntu-latest 9 | permissions: 10 | issues: write 11 | pull-requests: write 12 | steps: 13 | - uses: actions/stale@v3 14 | with: 15 | days-before-issue-stale: 30 16 | days-before-issue-close: 14 17 | exempt-issue-labels: "type/feature-request,type/enhancement,lifecycle/frozen" 18 | stale-issue-label: "stale" 19 | close-issue-label: "close for stale" 20 | stale-issue-message: "This issue is stale because it has been open for 30 days with no activity." 21 | close-issue-message: "This issue was closed because it has been inactive for 14 days since being marked as stale." 22 | days-before-pr-stale: -1 23 | days-before-pr-close: -1 24 | repo-token: ${{ secrets.GITHUB_TOKEN }} -------------------------------------------------------------------------------- /.github/workflows/license-check.yml: -------------------------------------------------------------------------------- 1 | name: License checker 2 | 3 | on: 4 | push: 5 | branches: 6 | - master 7 | pull_request: 8 | branches: 9 | - master 10 | 11 | jobs: 12 | check-license: 13 | runs-on: ubuntu-latest 14 | 15 | steps: 16 | - uses: actions/checkout@v2 17 | - name: Check License Header 18 | uses: apache/skywalking-eyes@v0.3.0 19 | env: 20 | GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} 21 | with: 22 | log: info 23 | config: .github/license-checker-config.yml 24 | -------------------------------------------------------------------------------- /.gitmodules: -------------------------------------------------------------------------------- 1 | [submodule "core/src/test/resources/tispark-test"] 2 | path = core/src/test/resources/tispark-test 3 | url = https://github.com/pingcap/tispark-test-data.git 4 | -------------------------------------------------------------------------------- /.lift.toml: -------------------------------------------------------------------------------- 1 | build = "mvn" 2 | 3 | ignoreFiles = """ 4 | assembly/** 5 | db-random-test/** 6 | spark-wrapper/** 7 | tikv-client/target/** 8 | core/target/** 9 | **/build/** 10 | """ 11 | 12 | tools = [ "infer", "findsecbugs","errorprone" ] 13 | 14 | jdk11 = false -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | language: java 2 | script: 3 | - "cd tikv-client && mvn clean verify" 4 | after_success: 5 | - bash <(curl -s https://codecov.io/bash) -t f499cf7f-6b6c-4f02-8af8-d83d9811f18a 6 | -------------------------------------------------------------------------------- /OWNERS: -------------------------------------------------------------------------------- 1 | # See the OWNERS docs at https://go.k8s.io/owners 2 | approvers: 3 | - birdstorm 4 | - Coderlxl 5 | - humengyu2012 6 | - ilovesoup 7 | - LittleFall 8 | - marsishandsome 9 | - shiyuhang0 10 | - sunxiaoguang 11 | - windtalker 12 | - xuanyu66 13 | reviewers: 14 | - Daemonxiao 15 | - qidi1 16 | - zhangyangyu 17 | -------------------------------------------------------------------------------- /config/cert/jks/client-keystore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/config/cert/jks/client-keystore -------------------------------------------------------------------------------- /config/cert/jks/server-cert-store: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/config/cert/jks/server-cert-store -------------------------------------------------------------------------------- /config/cert/pem/root.srl: -------------------------------------------------------------------------------- 1 | 42E6FBF8D35C1FE8AF3B84831DD93185C484A407 2 | -------------------------------------------------------------------------------- /config/followerread/pd.toml: -------------------------------------------------------------------------------- 1 | [replication] 2 | ## Doesn't work when deploy with tiup playground 3 | max-replicas = 3 4 | -------------------------------------------------------------------------------- /config/followerread/tidb.toml: -------------------------------------------------------------------------------- 1 | index-limit = 512 -------------------------------------------------------------------------------- /config/followerread/tikv.toml: -------------------------------------------------------------------------------- 1 | [server] 2 | labels = {key1="value1", key2="value2"} 3 | [raftdb] 4 | max-open-files = 256 5 | [rocksdb] 6 | max-open-files = 256 -------------------------------------------------------------------------------- /config/pd-4.0.toml: -------------------------------------------------------------------------------- 1 | [pd-server] 2 | ## Disable dashboard 3 | dashboard-address = "none" 4 | 5 | [replication] 6 | enable-placement-rules = true 7 | max-replicas = 1 8 | 9 | [dashboard] 10 | ## When enabled, usage data will be sent to PingCAP for improving user experience. 11 | enable-telemetry = false 12 | -------------------------------------------------------------------------------- /config/pd-TLS.toml: -------------------------------------------------------------------------------- 1 | [security] 2 | cacert-path = "/config/cert/pem/root.pem" 3 | cert-path = "/config/cert/pem/pd.pem" 4 | key-path = "/config/cert/pem/pd.key" 5 | 6 | 7 | -------------------------------------------------------------------------------- /config/tidb-4.0.toml: -------------------------------------------------------------------------------- 1 | # TiDB Configuration. 2 | 3 | # When create table, split a separated region for it. It is recommended to 4 | # turn off this option if there will be a large number of tables created. 5 | split-table = true 6 | 7 | # delay-clean-table-lock is used to control whether delayed-release the table lock in the abnormal situation. (Milliseconds) 8 | delay-clean-table-lock = 60000 9 | 10 | # enable-table-lock is used to control table lock feature. Default is false, indicate the table lock feature is disabled. 11 | enable-table-lock = true 12 | 13 | # alter-primary-key is used to control alter primary key feature. Default is false, indicate the alter primary key feature is disabled. 14 | # If it is true, we can add the primary key by "alter table", but we may not be able to drop the primary key. 15 | # In order to support "drop primary key" operation , this flag must be true and the table does not have the pkIsHandle flag. 16 | alter-primary-key = true 17 | 18 | # index-limit is used to deal with compatibility issues. It can only be in [64, 64*8]. 19 | index-limit = 512 -------------------------------------------------------------------------------- /config/tidb-TLS.toml: -------------------------------------------------------------------------------- 1 | index-limit = 512 2 | host = "0.0.0.0" 3 | store = "tikv" 4 | 5 | [security] 6 | auto-tls = true 7 | ssl-ca = "/config/cert/pem/root.pem" 8 | ssl-cert = "/config/cert/pem/tidb.pem" 9 | ssl-key = "/config/cert/pem/tidb.key" 10 | 11 | cluster-ssl-ca = "/config/cert/pem/root.pem" 12 | cluster-ssl-cert = "/config/cert/pem/tidb.pem" 13 | cluster-ssl-key = "/config/cert/pem/tidb.key" -------------------------------------------------------------------------------- /config/tiflash-learner-daily-test.toml: -------------------------------------------------------------------------------- 1 | log-file = "/home/jenkins/agent/workspace/tispark_regression_test_daily/tiflash/tiflash_tikv.log" 2 | 3 | [rocksdb] 4 | wal-dir = "" 5 | 6 | [security] 7 | ca-path = "" 8 | cert-path = "" 9 | key-path = "" 10 | 11 | [server] 12 | addr = "0.0.0.0:20170" 13 | advertise-addr = "127.0.0.1:20170" 14 | engine-addr = "127.0.0.1:3930" 15 | status-addr = "127.0.0.1:20292" 16 | 17 | [storage] 18 | data-dir = "/home/jenkins/agent/workspace/tispark_regression_test_daily/tiflash/flash" -------------------------------------------------------------------------------- /config/tiflash-learner.toml: -------------------------------------------------------------------------------- 1 | log-file = "/home/jenkins/agent/workspace/tispark_ghpr_integration_test/tiflash/tiflash_tikv.log" 2 | 3 | [rocksdb] 4 | wal-dir = "" 5 | 6 | [security] 7 | ca-path = "" 8 | cert-path = "" 9 | key-path = "" 10 | 11 | [server] 12 | addr = "0.0.0.0:20170" 13 | advertise-addr = "127.0.0.1:20170" 14 | engine-addr = "127.0.0.1:3930" 15 | status-addr = "127.0.0.1:20292" 16 | 17 | [storage] 18 | data-dir = "/home/jenkins/agent/workspace/tispark_ghpr_integration_test/tiflash/flash" -------------------------------------------------------------------------------- /config/tikv-4.0.toml: -------------------------------------------------------------------------------- 1 | # TiKV config template 2 | 3 | [raftstore] 4 | # set store capacity, if no set, use disk capacity. 5 | capacity = "8G" 6 | 7 | [coprocessor] 8 | # When it is true, it will try to split a region with table prefix if 9 | # that region crosses tables. 10 | # split-region-on-table = false 11 | # When the region's size exceeds region-max-size, we will split the region 12 | # into two which the left region's size will be region-split-size or a little 13 | # bit smaller. 14 | # region-max-size = "144MB" 15 | # region-split-size = "96MB" 16 | # region-max-size = "2MB" 17 | # region-split-size = "1MB" -------------------------------------------------------------------------------- /config/tikv-TLS.toml: -------------------------------------------------------------------------------- 1 | [security] 2 | ca-path = "/config/cert/pem/root.pem" 3 | cert-path = "/config/cert/pem/tikv.pem" 4 | key-path = "/config/cert/pem/tikv.key" 5 | -------------------------------------------------------------------------------- /core-test/src/KEEPME: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/core-test/src/KEEPME -------------------------------------------------------------------------------- /core/scripts/fetch-test-data.sh: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | CURRENT_DIR=`pwd` 4 | TISPARK_HOME="$(cd "`dirname "$0"`"/../..; pwd)" 5 | cd $TISPARK_HOME/core 6 | git submodule update --init --recursive 7 | cd $CURRENT_DIR -------------------------------------------------------------------------------- /core/scripts/version.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | # 3 | # Copyright 2017 PingCAP, Inc. 4 | # 5 | # Licensed under the Apache License, Version 2.0 (the "License"); 6 | # you may not use this file except in compliance with the License. 7 | # You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | # 16 | 17 | TISPARK_HOME="$(cd "`dirname "$0"`"/../..; pwd)" 18 | 19 | TiSparkReleaseVersion=$(mvn help:evaluate -Dexpression=project.version -q -DforceStdout) 20 | TiSparkBuildTS=`date -u '+%Y-%m-%d %I:%M:%S'` 21 | TiSparkGitHash=`git rev-parse HEAD` 22 | TiSparkGitBranch=`git rev-parse --abbrev-ref HEAD` 23 | echo ' 24 | package com.pingcap.tispark 25 | 26 | object TiSparkVersion { 27 | val version: String = 28 | "Release Version: '${TiSparkReleaseVersion}'\\nGit Commit Hash: '${TiSparkGitHash}'\\nGit Branch: '${TiSparkGitBranch}'\\nUTC Build Time: '${TiSparkBuildTS}'" 29 | }' > ${TISPARK_HOME}/core/src/main/scala/com/pingcap/tispark/TiSparkVersion.scala 30 | -------------------------------------------------------------------------------- /core/src/main/java/com/pingcap/tikv/columnar/TiColumnarBatchHelper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tikv.columnar; 18 | 19 | import org.apache.spark.sql.vectorized.ColumnarBatch; 20 | 21 | /** A helper class to create {@link ColumnarBatch} from {@link TiChunk} */ 22 | public final class TiColumnarBatchHelper { 23 | public static ColumnarBatch createColumnarBatch(TiChunk chunk) { 24 | int colLen = chunk.numOfCols(); 25 | TiColumnVectorAdapter[] columns = new TiColumnVectorAdapter[colLen]; 26 | for (int i = 0; i < colLen; i++) { 27 | columns[i] = new TiColumnVectorAdapter(chunk.column(i)); 28 | } 29 | ColumnarBatch batch = new ColumnarBatch(columns); 30 | batch.setNumRows(chunk.numOfRows()); 31 | return batch; 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister: -------------------------------------------------------------------------------- 1 | com.pingcap.tispark.v2.TiDBTableProvider -------------------------------------------------------------------------------- /core/src/main/resources/log4j.properties.template: -------------------------------------------------------------------------------- 1 | # Set everything to be logged to the console 2 | log4j.rootCategory=INFO, console 3 | log4j.appender.console=org.apache.log4j.ConsoleAppender 4 | log4j.appender.console.target=System.err 5 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 6 | log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n 7 | 8 | # Set the default spark-shell log level to WARN. When running the spark-shell, the 9 | # log level for this class is used to overwrite the root logger's log level, so that 10 | # the user can have different defaults for the shell and regular Spark apps. 11 | log4j.logger.org.apache.spark.repl.Main=WARN 12 | 13 | # Settings to quiet third party logs that are too verbose 14 | log4j.logger.org.spark_project.jetty=WARN 15 | log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR 16 | log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO 17 | log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO 18 | log4j.logger.org.apache.parquet=ERROR 19 | log4j.logger.parquet=ERROR 20 | 21 | # SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support 22 | log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL 23 | log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR 24 | 25 | # tispark 26 | log4j.logger.com.pingcap.tispark=DEBUG 27 | -------------------------------------------------------------------------------- /core/src/main/scala/com/pingcap/tispark/DefaultSource.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tispark 18 | 19 | import com.pingcap.tispark.v2.TiDBTableProvider 20 | 21 | /** 22 | * Spark Data Source API requires that the class name must be `DefaultSource` 23 | */ 24 | class DefaultSource extends TiDBTableProvider {} 25 | -------------------------------------------------------------------------------- /core/src/main/scala/com/pingcap/tispark/TiDBRelation.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tispark 18 | 19 | import org.apache.spark.sql.SQLContext 20 | import org.apache.spark.sql.sources.BaseRelation 21 | import org.apache.spark.sql.types.StructType 22 | 23 | // Just for the build of v1 path 24 | case class TiDBRelation(sqlContext: SQLContext) extends BaseRelation { 25 | override def schema: StructType = ??? 26 | } 27 | -------------------------------------------------------------------------------- /core/src/main/scala/com/pingcap/tispark/TiPartition.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tispark 18 | 19 | import org.tikv.common.util.RangeSplitter.RegionTask 20 | import org.apache.spark.Partition 21 | 22 | class TiPartition(val idx: Int, val tasks: Seq[RegionTask], val appId: String) extends Partition { 23 | override def index: Int = idx 24 | } 25 | -------------------------------------------------------------------------------- /core/src/main/scala/com/pingcap/tispark/TiTableReference.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tispark 18 | 19 | case class TiTableReference( 20 | databaseName: String, 21 | tableName: String, 22 | var sizeInBytes: Long = Long.MaxValue, 23 | var numRows: Long = Long.MaxValue) 24 | -------------------------------------------------------------------------------- /core/src/main/scala/com/pingcap/tispark/v2/TiDBTableScanBuilder.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2023 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tispark.v2 18 | 19 | import com.pingcap.tispark.TiTableReference 20 | import org.apache.spark.sql.connector.read.{Scan, ScanBuilder} 21 | import org.apache.spark.sql.types.StructType 22 | 23 | case class TiDBTableScanBuilder(tableRef: TiTableReference, schema: StructType) 24 | extends ScanBuilder { 25 | override def build(): Scan = { 26 | TiDBTableScan(tableRef, schema) 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /core/src/main/scala/com/pingcap/tispark/v2/sink/TiDBDataWriterFactory.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2021 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tispark.v2.sink 18 | 19 | import com.pingcap.tikv.TiConfiguration 20 | import com.pingcap.tispark.write.TiDBOptions 21 | import org.apache.spark.sql.catalyst.InternalRow 22 | import org.apache.spark.sql.connector.write.{DataWriter, DataWriterFactory} 23 | import org.apache.spark.sql.types.StructType 24 | 25 | /** 26 | * Use V1WriteBuilder before turn to v2 27 | */ 28 | case class TiDBDataWriterFactory( 29 | schema: StructType, 30 | tiDBOptions: TiDBOptions, 31 | ticonf: TiConfiguration) 32 | extends DataWriterFactory { 33 | 34 | override def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = 35 | TiDBDataWrite(partitionId, taskId, schema, tiDBOptions, ticonf) 36 | } 37 | -------------------------------------------------------------------------------- /core/src/main/scala/com/pingcap/tispark/write/DBTable.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tispark.write 18 | 19 | case class DBTable(database: String, table: String) {} 20 | -------------------------------------------------------------------------------- /core/src/main/scala/com/pingcap/tispark/write/EncodedKVPair.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tispark.write 18 | 19 | /** 20 | * EncodedKVPair is used at last stage if write process. 21 | * At this stage, sorting should not be happened anymore. 22 | */ 23 | case class EncodedKVPair(encodedKey: SerializableKey, encodedValue: Array[Byte]) { 24 | override def hashCode(): Int = encodedKey.hashCode() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/main/scala/com/pingcap/tispark/write/WrappedEncodedRow.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tispark.write 18 | 19 | import com.pingcap.tikv.handle.Handle 20 | import com.pingcap.tispark.write.TiBatchWrite.TiRow 21 | 22 | case class WrappedEncodedRow( 23 | row: TiRow, 24 | handle: Handle, 25 | encodedKey: SerializableKey, 26 | encodedValue: Array[Byte], 27 | isIndex: Boolean, 28 | indexId: Long, 29 | remove: Boolean) 30 | extends Ordered[WrappedEncodedRow] { 31 | override def compare(that: WrappedEncodedRow): Int = this.handle.compare(that.handle) 32 | 33 | override def hashCode(): Int = encodedKey.hashCode() 34 | } 35 | -------------------------------------------------------------------------------- /core/src/main/scala/com/pingcap/tispark/write/WrappedRow.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tispark.write 18 | 19 | import com.pingcap.tikv.handle.Handle 20 | import com.pingcap.tispark.write.TiBatchWrite.TiRow 21 | 22 | case class WrappedRow(row: TiRow, handle: Handle) 23 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/sql/catalyst/parser/TiParserFactory.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2022 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.catalyst.parser 18 | 19 | import org.apache.spark.sql.{SparkSession, TiContext} 20 | 21 | case class TiParserFactory(getOrCreateTiContext: SparkSession => TiContext) 22 | extends ((SparkSession, ParserInterface) => ParserInterface) { 23 | override def apply( 24 | sparkSession: SparkSession, 25 | parserInterface: ParserInterface): ParserInterface = { 26 | TiParser(getOrCreateTiContext, sparkSession, parserInterface) 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/sql/catalyst/planner/TiAggregation.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2022 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.catalyst.planner 18 | 19 | import org.apache.spark.sql.catalyst.expressions.NamedExpression 20 | import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression 21 | import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan 22 | 23 | object TiAggregation { 24 | type ReturnType = 25 | (Seq[NamedExpression], Seq[AggregateExpression], Seq[NamedExpression], LogicalPlan) 26 | 27 | def unapply(plan: LogicalPlan): Option[ReturnType] = TiAggregationImpl.unapply(plan) 28 | } 29 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/sql/catalyst/planner/TiStrategyFactory.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2022 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.catalyst.planner 18 | 19 | import com.pingcap.tispark.utils.ReflectionUtil 20 | import org.apache.spark.sql.{SparkSession, Strategy, TiContext, TiExtensions} 21 | 22 | /** 23 | * The logical plan DataSourceV2ScanRelation is different in spark 3.0 and 3.1 24 | * @param getOrCreateTiContext 25 | */ 26 | class TiStrategyFactory(getOrCreateTiContext: SparkSession => TiContext) 27 | extends (SparkSession => Strategy) { 28 | override def apply(sparkSession: SparkSession): Strategy = { 29 | TiExtensions.validateCatalog(sparkSession) 30 | ReflectionUtil.newTiStrategy(getOrCreateTiContext, sparkSession) 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BasicLogicalPlan.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2022 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.catalyst.plans.logical 20 | 21 | import com.pingcap.tispark.auth.TiAuthorization 22 | import com.pingcap.tispark.utils.ReflectionUtil 23 | 24 | object BasicLogicalPlan { 25 | def verifyAuthorizationRule( 26 | logicalPlan: LogicalPlan, 27 | tiAuthorization: Option[TiAuthorization]): LogicalPlan = 28 | ReflectionUtil.callTiBasicLogicalPlanVerifyAuthorizationRule(logicalPlan, tiAuthorization) 29 | } 30 | -------------------------------------------------------------------------------- /core/src/test/resources/prefix-index/PrefixTest.sql: -------------------------------------------------------------------------------- 1 | DROP TABLE IF EXISTS `prefix`; 2 | CREATE TABLE `prefix` ( 3 | `a` int(11) NOT NULL, 4 | `b` varchar(55) DEFAULT NULL, 5 | `c` int(11) DEFAULT NULL, 6 | PRIMARY KEY (`a`), 7 | KEY `prefix_index` (`b`(2)), 8 | KEY `prefix_complex` (`a`, `b`(2)) 9 | ) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin; 10 | INSERT INTO `prefix` VALUES(0, "b", 2), (1, "bbb", 3), (2, "bbc", 4), (3, "bbb", 5), (4, "abc", 6), (5, "abc", 7), (6, "abc", 7), (7, "ÿÿ", 8), (8, "ÿÿ0", 9), (9, "ÿÿÿ", 10); 11 | ANALYZE TABLE `prefix`; -------------------------------------------------------------------------------- /core/src/test/resources/prefix-index/UTF8Test.sql: -------------------------------------------------------------------------------- 1 | DROP TABLE IF EXISTS `t1`; 2 | CREATE TABLE `t1` ( 3 | `name` varchar(12) DEFAULT NULL, 4 | KEY `pname` (`name`(12)) 5 | ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_unicode_ci; 6 | insert into t1 values('借款策略集_网页'); -------------------------------------------------------------------------------- /core/src/test/resources/resolveLock-test/1_account.sql: -------------------------------------------------------------------------------- 1 | select 2 | C_ACCTBAL 3 | from 4 | resolveLock_test.CUSTOMER 5 | where 6 | C_CUSTKEY 7 | = $1 -------------------------------------------------------------------------------- /core/src/test/resources/resolveLock-test/1_give.sql: -------------------------------------------------------------------------------- 1 | update 2 | resolveLock_test.CUSTOMER 3 | set 4 | C_ACCTBAL 5 | = C_ACCTBAL - $1 6 | where 7 | C_CUSTKEY 8 | = $2 -------------------------------------------------------------------------------- /core/src/test/resources/resolveLock-test/2_get.sql: -------------------------------------------------------------------------------- 1 | update 2 | resolveLock_test.CUSTOMER 3 | set 4 | C_ACCTBAL 5 | = C_ACCTBAL + $1 6 | where 7 | C_CUSTKEY 8 | = $2 -------------------------------------------------------------------------------- /core/src/test/resources/resolveLock-test/q1.sql: -------------------------------------------------------------------------------- 1 | select 2 | sum(C_ACCTBAL) 3 | from 4 | CUSTOMER 5 | where 6 | C_ACCTBAL % 2 == 0 -------------------------------------------------------------------------------- /core/src/test/resources/resolveLock-test/q2.sql: -------------------------------------------------------------------------------- 1 | select 2 | sum(C_ACCTBAL) 3 | from 4 | CUSTOMER 5 | 6 | UNION 7 | 8 | select 9 | avg(C_ACCTBAL) 10 | from 11 | CUSTOMER 12 | 13 | UNION 14 | 15 | select 16 | count(*) 17 | from 18 | CUSTOMER 19 | GROUP BY 20 | C_NATIONKEY 21 | 22 | UNION 23 | 24 | select 25 | C_CUSTKEY + C_NATIONKEY 26 | from 27 | CUSTOMER 28 | 29 | UNION 30 | 31 | select 32 | count(*) 33 | from 34 | CUSTOMER 35 | 36 | UNION 37 | 38 | select 39 | count(*) 40 | from 41 | CUSTOMER 42 | GROUP BY 43 | C_NATIONKEY -------------------------------------------------------------------------------- /core/src/test/resources/resolveLock-test/sum_account.sql: -------------------------------------------------------------------------------- 1 | select 2 | sum(C_ACCTBAL) 3 | from 4 | CUSTOMER -------------------------------------------------------------------------------- /core/src/test/resources/tidb_config.properties.TLS.template: -------------------------------------------------------------------------------- 1 | spark.tispark.pd.addresses pd:2379 2 | spark.sql.catalog.tidb_catalog org.apache.spark.sql.catalyst.catalog.TiCatalog 3 | spark.sql.catalog.tidb_catalog.pd.addresses pd:2379 4 | 5 | spark.tispark.tikv.tls_enable=true 6 | spark.tispark.tikv.trust_cert_collection=/config/cert/pem/root.pem 7 | spark.tispark.tikv.key_cert_chain=/config/cert/pem/client.pem 8 | spark.tispark.tikv.key_file=/config/cert/pem/client-pkcs8.key 9 | 10 | #spark.tispark.tikv.jks_enable=true 11 | #spark.tispark.tikv.jks_key_path=/config/cert/jks/client-keystore 12 | #spark.tispark.tikv.jks_key_password=123456 13 | #spark.tispark.tikv.jks_trust_path=/config/cert/jks/server-cert-store 14 | #spark.tispark.tikv.jks_trust_password=12345678 15 | 16 | spark.tispark.jdbc.tls_enable=true 17 | spark.tispark.jdbc.server_cert_store = /config/cert/jks/server-cert-store 18 | spark.tispark.jdbc.server_cert_password = 12345678 19 | spark.tispark.jdbc.client_cert_store = /config/cert/jks/client-keystore 20 | spark.tispark.jdbc.client_cert_password = 123456 21 | -------------------------------------------------------------------------------- /core/src/test/resources/tidb_config.properties.withoutdataload.template: -------------------------------------------------------------------------------- 1 | test.data.load=false 2 | spark.sql.catalog.tidb_catalog org.apache.spark.sql.catalyst.catalog.TiCatalog 3 | 4 | 5 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q1.sql: -------------------------------------------------------------------------------- 1 | --q1.sql-- 2 | 3 | WITH customer_total_return AS 4 | (SELECT sr_customer_sk AS ctr_customer_sk, sr_store_sk AS ctr_store_sk, 5 | sum(sr_return_amt) AS ctr_total_return 6 | FROM store_returns, date_dim 7 | WHERE sr_returned_date_sk = d_date_sk AND d_year = 2000 8 | GROUP BY sr_customer_sk, sr_store_sk) 9 | SELECT c_customer_id 10 | FROM customer_total_return ctr1, store, customer 11 | WHERE ctr1.ctr_total_return > 12 | (SELECT avg(ctr_total_return)*1.2 13 | FROM customer_total_return ctr2 14 | WHERE ctr1.ctr_store_sk = ctr2.ctr_store_sk) 15 | AND s_store_sk = ctr1.ctr_store_sk 16 | AND s_state = 'TN' 17 | AND ctr1.ctr_customer_sk = c_customer_sk 18 | ORDER BY c_customer_id LIMIT 100 19 | 20 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q12.sql: -------------------------------------------------------------------------------- 1 | --q12.sql-- 2 | 3 | select i_item_id, 4 | i_item_desc, i_category, i_class, i_current_price, 5 | sum(ws_ext_sales_price) as itemrevenue, 6 | sum(ws_ext_sales_price)*100/sum(sum(ws_ext_sales_price)) over 7 | (partition by i_class) as revenueratio 8 | from 9 | web_sales, item, date_dim 10 | where 11 | ws_item_sk = i_item_sk 12 | and i_category in ('Sports', 'Books', 'Home') 13 | and ws_sold_date_sk = d_date_sk 14 | and d_date between cast('1999-02-22' as date) 15 | and (cast('1999-02-22' as date) + interval '30' day) 16 | group by 17 | i_item_id, i_item_desc, i_category, i_class, i_current_price 18 | order by 19 | i_category, i_class, i_item_id, i_item_desc, revenueratio 20 | LIMIT 100 21 | 22 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q15.sql: -------------------------------------------------------------------------------- 1 | --q15.sql-- 2 | 3 | select ca_zip, sum(cs_sales_price) 4 | from catalog_sales, customer, customer_address, date_dim 5 | where cs_bill_customer_sk = c_customer_sk 6 | and c_current_addr_sk = ca_address_sk 7 | and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', 8 | '85392', '85460', '80348', '81792') 9 | or ca_state in ('CA','WA','GA') 10 | or cs_sales_price > 500) 11 | and cs_sold_date_sk = d_date_sk 12 | and d_qoy = 2 and d_year = 2001 13 | group by ca_zip 14 | order by ca_zip 15 | limit 100 16 | 17 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q16.sql: -------------------------------------------------------------------------------- 1 | --q16.sql-- 2 | 3 | select 4 | count(distinct cs_order_number) as `order count`, 5 | sum(cs_ext_ship_cost) as `total shipping cost`, 6 | sum(cs_net_profit) as `total net profit` 7 | from 8 | catalog_sales cs1, date_dim, customer_address, call_center 9 | where 10 | d_date between cast ('2002-02-01' as date) and (cast('2002-02-01' as date) + interval '60' day) 11 | and cs1.cs_ship_date_sk = d_date_sk 12 | and cs1.cs_ship_addr_sk = ca_address_sk 13 | and ca_state = 'GA' 14 | and cs1.cs_call_center_sk = cc_call_center_sk 15 | and cc_county in ('Williamson County','Williamson County','Williamson County','Williamson County', 'Williamson County') 16 | and exists (select * 17 | from catalog_sales cs2 18 | where cs1.cs_order_number = cs2.cs_order_number 19 | and cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk) 20 | and not exists(select * 21 | from catalog_returns cr1 22 | where cs1.cs_order_number = cr1.cr_order_number) 23 | order by count(distinct cs_order_number) 24 | limit 100 25 | 26 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q18.sql: -------------------------------------------------------------------------------- 1 | --q18.sql-- 2 | 3 | select i_item_id, 4 | ca_country, 5 | ca_state, 6 | ca_county, 7 | avg( cast(cs_quantity as decimal(12,2))) agg1, 8 | avg( cast(cs_list_price as decimal(12,2))) agg2, 9 | avg( cast(cs_coupon_amt as decimal(12,2))) agg3, 10 | avg( cast(cs_sales_price as decimal(12,2))) agg4, 11 | avg( cast(cs_net_profit as decimal(12,2))) agg5, 12 | avg( cast(c_birth_year as decimal(12,2))) agg6, 13 | avg( cast(cd1.cd_dep_count as decimal(12,2))) agg7 14 | from catalog_sales, customer_demographics cd1, 15 | customer_demographics cd2, customer, customer_address, date_dim, item 16 | where cs_sold_date_sk = d_date_sk and 17 | cs_item_sk = i_item_sk and 18 | cs_bill_cdemo_sk = cd1.cd_demo_sk and 19 | cs_bill_customer_sk = c_customer_sk and 20 | cd1.cd_gender = 'F' and 21 | cd1.cd_education_status = 'Unknown' and 22 | c_current_cdemo_sk = cd2.cd_demo_sk and 23 | c_current_addr_sk = ca_address_sk and 24 | c_birth_month in (1,6,8,9,12,2) and 25 | d_year = 1998 and 26 | ca_state in ('MS','IN','ND','OK','NM','VA','MS') 27 | group by rollup (i_item_id, ca_country, ca_state, ca_county) 28 | order by ca_country, ca_state, ca_county, i_item_id 29 | LIMIT 100 30 | 31 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q19.sql: -------------------------------------------------------------------------------- 1 | --q19.sql-- 2 | 3 | select i_brand_id brand_id, i_brand brand, i_manufact_id, i_manufact, 4 | sum(ss_ext_sales_price) ext_price 5 | from date_dim, store_sales, item,customer,customer_address,store 6 | where d_date_sk = ss_sold_date_sk 7 | and ss_item_sk = i_item_sk 8 | and i_manager_id = 8 9 | and d_moy = 11 10 | and d_year = 1998 11 | and ss_customer_sk = c_customer_sk 12 | and c_current_addr_sk = ca_address_sk 13 | and substr(ca_zip,1,5) <> substr(s_zip,1,5) 14 | and ss_store_sk = s_store_sk 15 | group by i_brand, i_brand_id, i_manufact_id, i_manufact 16 | order by ext_price desc, brand, brand_id, i_manufact_id, i_manufact 17 | limit 100 18 | 19 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q20.sql: -------------------------------------------------------------------------------- 1 | --q20.sql-- 2 | 3 | select i_item_id, i_item_desc 4 | ,i_category 5 | ,i_class 6 | ,i_current_price 7 | ,sum(cs_ext_sales_price) as itemrevenue 8 | ,sum(cs_ext_sales_price)*100/sum(sum(cs_ext_sales_price)) over 9 | (partition by i_class) as revenueratio 10 | from catalog_sales, item, date_dim 11 | where cs_item_sk = i_item_sk 12 | and i_category in ('Sports', 'Books', 'Home') 13 | and cs_sold_date_sk = d_date_sk 14 | and d_date between cast('1999-02-22' as date) 15 | and (cast('1999-02-22' as date) + interval '30' day) 16 | group by i_item_id, i_item_desc, i_category, i_class, i_current_price 17 | order by i_category, i_class, i_item_id, i_item_desc, revenueratio 18 | limit 100 19 | 20 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q21.sql: -------------------------------------------------------------------------------- 1 | --q21.sql-- 2 | 3 | select * from( 4 | select w_warehouse_name, i_item_id, 5 | sum(case when (cast(d_date as date) < cast ('2000-03-11' as date)) 6 | then inv_quantity_on_hand 7 | else 0 end) as inv_before, 8 | sum(case when (cast(d_date as date) >= cast ('2000-03-11' as date)) 9 | then inv_quantity_on_hand 10 | else 0 end) as inv_after 11 | from inventory, warehouse, item, date_dim 12 | where i_current_price between 0.99 and 1.49 13 | and i_item_sk = inv_item_sk 14 | and inv_warehouse_sk = w_warehouse_sk 15 | and inv_date_sk = d_date_sk 16 | and d_date between (cast('2000-03-11' as date) - interval '30' day) 17 | and (cast('2000-03-11' as date) + interval '30' day) 18 | group by w_warehouse_name, i_item_id) x 19 | where (case when inv_before > 0 20 | then inv_after / inv_before 21 | else null 22 | end) between 2.0/3.0 and 3.0/2.0 23 | order by w_warehouse_name, i_item_id 24 | limit 100 25 | 26 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q22.sql: -------------------------------------------------------------------------------- 1 | --q22.sql-- 2 | 3 | select i_product_name, i_brand, i_class, i_category, avg(inv_quantity_on_hand) qoh 4 | from inventory, date_dim, item, warehouse 5 | where inv_date_sk=d_date_sk 6 | and inv_item_sk=i_item_sk 7 | and inv_warehouse_sk = w_warehouse_sk 8 | and d_month_seq between 1200 and 1200 + 11 9 | group by rollup(i_product_name, i_brand, i_class, i_category) 10 | order by qoh, i_product_name, i_brand, i_class, i_category 11 | limit 100 12 | 13 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q24a.sql: -------------------------------------------------------------------------------- 1 | --q24a.sql-- 2 | 3 | with ssales as 4 | (select c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, 5 | i_current_price, i_manager_id, i_units, i_size, sum(ss_net_paid) netpaid 6 | from store_sales, store_returns, store, item, customer, customer_address 7 | where ss_ticket_number = sr_ticket_number 8 | and ss_item_sk = sr_item_sk 9 | and ss_customer_sk = c_customer_sk 10 | and ss_item_sk = i_item_sk 11 | and ss_store_sk = s_store_sk 12 | and c_birth_country = upper(ca_country) 13 | and s_zip = ca_zip 14 | and s_market_id = 8 15 | group by c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, 16 | i_current_price, i_manager_id, i_units, i_size) 17 | select c_last_name, c_first_name, s_store_name, sum(netpaid) paid 18 | from ssales 19 | where i_color = 'pale' 20 | group by c_last_name, c_first_name, s_store_name 21 | having sum(netpaid) > (select 0.05*avg(netpaid) from ssales) 22 | 23 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q24b.sql: -------------------------------------------------------------------------------- 1 | --q24b.sql-- 2 | 3 | with ssales as 4 | (select c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, 5 | i_current_price, i_manager_id, i_units, i_size, sum(ss_net_paid) netpaid 6 | from store_sales, store_returns, store, item, customer, customer_address 7 | where ss_ticket_number = sr_ticket_number 8 | and ss_item_sk = sr_item_sk 9 | and ss_customer_sk = c_customer_sk 10 | and ss_item_sk = i_item_sk 11 | and ss_store_sk = s_store_sk 12 | and c_birth_country = upper(ca_country) 13 | and s_zip = ca_zip 14 | and s_market_id = 8 15 | group by c_last_name, c_first_name, s_store_name, ca_state, s_state, 16 | i_color, i_current_price, i_manager_id, i_units, i_size) 17 | select c_last_name, c_first_name, s_store_name, sum(netpaid) paid 18 | from ssales 19 | where i_color = 'chiffon' 20 | group by c_last_name, c_first_name, s_store_name 21 | having sum(netpaid) > (select 0.05*avg(netpaid) from ssales) 22 | 23 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q25.sql: -------------------------------------------------------------------------------- 1 | --q25.sql-- 2 | 3 | select i_item_id, i_item_desc, s_store_id, s_store_name, 4 | sum(ss_net_profit) as store_sales_profit, 5 | sum(sr_net_loss) as store_returns_loss, 6 | sum(cs_net_profit) as catalog_sales_profit 7 | from 8 | store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, date_dim d3, 9 | store, item 10 | where 11 | d1.d_moy = 4 12 | and d1.d_year = 2001 13 | and d1.d_date_sk = ss_sold_date_sk 14 | and i_item_sk = ss_item_sk 15 | and s_store_sk = ss_store_sk 16 | and ss_customer_sk = sr_customer_sk 17 | and ss_item_sk = sr_item_sk 18 | and ss_ticket_number = sr_ticket_number 19 | and sr_returned_date_sk = d2.d_date_sk 20 | and d2.d_moy between 4 and 10 21 | and d2.d_year = 2001 22 | and sr_customer_sk = cs_bill_customer_sk 23 | and sr_item_sk = cs_item_sk 24 | and cs_sold_date_sk = d3.d_date_sk 25 | and d3.d_moy between 4 and 10 26 | and d3.d_year = 2001 27 | group by 28 | i_item_id, i_item_desc, s_store_id, s_store_name 29 | order by 30 | i_item_id, i_item_desc, s_store_id, s_store_name 31 | limit 100 32 | 33 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q26.sql: -------------------------------------------------------------------------------- 1 | --q26.sql-- 2 | 3 | select i_item_id, 4 | avg(cs_quantity) agg1, 5 | avg(cs_list_price) agg2, 6 | avg(cs_coupon_amt) agg3, 7 | avg(cs_sales_price) agg4 8 | from catalog_sales, customer_demographics, date_dim, item, promotion 9 | where cs_sold_date_sk = d_date_sk and 10 | cs_item_sk = i_item_sk and 11 | cs_bill_cdemo_sk = cd_demo_sk and 12 | cs_promo_sk = p_promo_sk and 13 | cd_gender = 'M' and 14 | cd_marital_status = 'S' and 15 | cd_education_status = 'College' and 16 | (p_channel_email = 'N' or p_channel_event = 'N') and 17 | d_year = 2000 18 | group by i_item_id 19 | order by i_item_id 20 | limit 100 21 | 22 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q27.sql: -------------------------------------------------------------------------------- 1 | --q27.sql-- 2 | 3 | select i_item_id, 4 | s_state, grouping(s_state) g_state, 5 | avg(ss_quantity) agg1, 6 | avg(ss_list_price) agg2, 7 | avg(ss_coupon_amt) agg3, 8 | avg(ss_sales_price) agg4 9 | from store_sales, customer_demographics, date_dim, store, item 10 | where ss_sold_date_sk = d_date_sk and 11 | ss_item_sk = i_item_sk and 12 | ss_store_sk = s_store_sk and 13 | ss_cdemo_sk = cd_demo_sk and 14 | cd_gender = 'M' and 15 | cd_marital_status = 'S' and 16 | cd_education_status = 'College' and 17 | d_year = 2002 and 18 | s_state in ('TN','TN', 'TN', 'TN', 'TN', 'TN') 19 | group by rollup (i_item_id, s_state) 20 | order by i_item_id, s_state 21 | limit 100 22 | 23 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q29.sql: -------------------------------------------------------------------------------- 1 | --q29.sql-- 2 | 3 | select 4 | i_item_id 5 | ,i_item_desc 6 | ,s_store_id 7 | ,s_store_name 8 | ,sum(ss_quantity) as store_sales_quantity 9 | ,sum(sr_return_quantity) as store_returns_quantity 10 | ,sum(cs_quantity) as catalog_sales_quantity 11 | from 12 | store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, 13 | date_dim d3, store, item 14 | where 15 | d1.d_moy = 9 16 | and d1.d_year = 1999 17 | and d1.d_date_sk = ss_sold_date_sk 18 | and i_item_sk = ss_item_sk 19 | and s_store_sk = ss_store_sk 20 | and ss_customer_sk = sr_customer_sk 21 | and ss_item_sk = sr_item_sk 22 | and ss_ticket_number = sr_ticket_number 23 | and sr_returned_date_sk = d2.d_date_sk 24 | and d2.d_moy between 9 and 9 + 3 25 | and d2.d_year = 1999 26 | and sr_customer_sk = cs_bill_customer_sk 27 | and sr_item_sk = cs_item_sk 28 | and cs_sold_date_sk = d3.d_date_sk 29 | and d3.d_year in (1999,1999+1,1999+2) 30 | group by 31 | i_item_id, i_item_desc, s_store_id, s_store_name 32 | order by 33 | i_item_id, i_item_desc, s_store_id, s_store_name 34 | limit 100 35 | 36 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q3.sql: -------------------------------------------------------------------------------- 1 | --q3.sql-- 2 | 3 | SELECT dt.d_year, item.i_brand_id brand_id, item.i_brand brand,SUM(ss_ext_sales_price) sum_agg 4 | FROM date_dim dt, store_sales, item 5 | WHERE dt.d_date_sk = store_sales.ss_sold_date_sk 6 | AND store_sales.ss_item_sk = item.i_item_sk 7 | AND item.i_manufact_id = 128 8 | AND dt.d_moy=11 9 | GROUP BY dt.d_year, item.i_brand, item.i_brand_id 10 | ORDER BY dt.d_year, sum_agg desc, brand_id 11 | LIMIT 100 12 | 13 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q30.sql: -------------------------------------------------------------------------------- 1 | --q30.sql-- 2 | 3 | with customer_total_return as 4 | (select wr_returning_customer_sk as ctr_customer_sk 5 | ,ca_state as ctr_state, 6 | sum(wr_return_amt) as ctr_total_return 7 | from web_returns, date_dim, customer_address 8 | where wr_returned_date_sk = d_date_sk 9 | and d_year = 2002 10 | and wr_returning_addr_sk = ca_address_sk 11 | group by wr_returning_customer_sk,ca_state) 12 | select c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag 13 | ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address 14 | ,c_last_review_date,ctr_total_return 15 | from customer_total_return ctr1, customer_address, customer 16 | where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 17 | from customer_total_return ctr2 18 | where ctr1.ctr_state = ctr2.ctr_state) 19 | and ca_address_sk = c_current_addr_sk 20 | and ca_state = 'GA' 21 | and ctr1.ctr_customer_sk = c_customer_sk 22 | order by c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag 23 | ,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address 24 | ,c_last_review_date,ctr_total_return 25 | limit 100 26 | 27 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q32.sql: -------------------------------------------------------------------------------- 1 | --q32.sql-- 2 | 3 | select sum(cs_ext_discount_amt) as `excess discount amount` 4 | from 5 | catalog_sales, item, date_dim 6 | where 7 | i_manufact_id = 977 8 | and i_item_sk = cs_item_sk 9 | and d_date between cast ('2000-01-27' as date) and (cast('2000-01-27' as date) + interval '90' day) 10 | and d_date_sk = cs_sold_date_sk 11 | and cs_ext_discount_amt > ( 12 | select 1.3 * avg(cs_ext_discount_amt) 13 | from catalog_sales, date_dim 14 | where cs_item_sk = i_item_sk 15 | and d_date between cast ('2000-01-27' as date) and (cast('2000-01-27' as date) + interval '90' day) 16 | and d_date_sk = cs_sold_date_sk) 17 | limit 100 18 | 19 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q36.sql: -------------------------------------------------------------------------------- 1 | --q36.sql-- 2 | 3 | select 4 | sum(ss_net_profit)/sum(ss_ext_sales_price) as gross_margin 5 | ,i_category 6 | ,i_class 7 | ,grouping(i_category)+grouping(i_class) as lochierarchy 8 | ,rank() over ( 9 | partition by grouping(i_category)+grouping(i_class), 10 | case when grouping(i_class) = 0 then i_category end 11 | order by sum(ss_net_profit)/sum(ss_ext_sales_price) asc) as rank_within_parent 12 | from 13 | store_sales, date_dim d1, item, store 14 | where 15 | d1.d_year = 2001 16 | and d1.d_date_sk = ss_sold_date_sk 17 | and i_item_sk = ss_item_sk 18 | and s_store_sk = ss_store_sk 19 | and s_state in ('TN','TN','TN','TN','TN','TN','TN','TN') 20 | group by rollup(i_category,i_class) 21 | order by 22 | lochierarchy desc 23 | ,case when lochierarchy = 0 then i_category end 24 | ,rank_within_parent 25 | limit 100 26 | 27 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q37.sql: -------------------------------------------------------------------------------- 1 | --q37.sql-- 2 | 3 | select i_item_id, i_item_desc, i_current_price 4 | from item, inventory, date_dim, catalog_sales 5 | where i_current_price between 68 and 68 + 30 6 | and inv_item_sk = i_item_sk 7 | and d_date_sk=inv_date_sk 8 | and d_date between cast('2000-02-01' as date) and (cast('2000-02-01' as date) + interval '60' day) 9 | and i_manufact_id in (677,940,694,808) 10 | and inv_quantity_on_hand between 100 and 500 11 | and cs_item_sk = i_item_sk 12 | group by i_item_id,i_item_desc,i_current_price 13 | order by i_item_id 14 | limit 100 15 | 16 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q38.sql: -------------------------------------------------------------------------------- 1 | --q38.sql-- 2 | 3 | select count(*) from ( 4 | select distinct c_last_name, c_first_name, d_date 5 | from store_sales, date_dim, customer 6 | where store_sales.ss_sold_date_sk = date_dim.d_date_sk 7 | and store_sales.ss_customer_sk = customer.c_customer_sk 8 | and d_month_seq between 1200 and 1200 + 11 9 | intersect 10 | select distinct c_last_name, c_first_name, d_date 11 | from catalog_sales, date_dim, customer 12 | where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk 13 | and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk 14 | and d_month_seq between 1200 and 1200 + 11 15 | intersect 16 | select distinct c_last_name, c_first_name, d_date 17 | from web_sales, date_dim, customer 18 | where web_sales.ws_sold_date_sk = date_dim.d_date_sk 19 | and web_sales.ws_bill_customer_sk = customer.c_customer_sk 20 | and d_month_seq between 1200 and 1200 + 11 21 | ) hot_cust 22 | limit 100 23 | 24 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q39a.sql: -------------------------------------------------------------------------------- 1 | --q39a.sql-- 2 | 3 | with inv as 4 | (select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy 5 | ,stdev,mean, case mean when 0 then null else stdev/mean end cov 6 | from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy 7 | ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean 8 | from inventory, item, warehouse, date_dim 9 | where inv_item_sk = i_item_sk 10 | and inv_warehouse_sk = w_warehouse_sk 11 | and inv_date_sk = d_date_sk 12 | and d_year = 2001 13 | group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo 14 | where case mean when 0 then 0 else stdev/mean end > 1) 15 | select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov 16 | ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov 17 | from inv inv1,inv inv2 18 | where inv1.i_item_sk = inv2.i_item_sk 19 | and inv1.w_warehouse_sk = inv2.w_warehouse_sk 20 | and inv1.d_moy=1 21 | and inv2.d_moy=1+1 22 | order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov 23 | ,inv2.d_moy,inv2.mean, inv2.cov 24 | 25 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q39b.sql: -------------------------------------------------------------------------------- 1 | --q39b.sql-- 2 | 3 | with inv as 4 | (select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy 5 | ,stdev,mean, case mean when 0 then null else stdev/mean end cov 6 | from(select w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy 7 | ,stddev_samp(inv_quantity_on_hand) stdev,avg(inv_quantity_on_hand) mean 8 | from inventory, item, warehouse, date_dim 9 | where inv_item_sk = i_item_sk 10 | and inv_warehouse_sk = w_warehouse_sk 11 | and inv_date_sk = d_date_sk 12 | and d_year = 2001 13 | group by w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy) foo 14 | where case mean when 0 then 0 else stdev/mean end > 1) 15 | select inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean, inv1.cov 16 | ,inv2.w_warehouse_sk,inv2.i_item_sk,inv2.d_moy,inv2.mean, inv2.cov 17 | from inv inv1,inv inv2 18 | where inv1.i_item_sk = inv2.i_item_sk 19 | and inv1.w_warehouse_sk = inv2.w_warehouse_sk 20 | and inv1.d_moy=1 21 | and inv2.d_moy=1+1 22 | and inv1.cov > 1.5 23 | order by inv1.w_warehouse_sk,inv1.i_item_sk,inv1.d_moy,inv1.mean,inv1.cov 24 | ,inv2.d_moy,inv2.mean, inv2.cov 25 | 26 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q40.sql: -------------------------------------------------------------------------------- 1 | --q40.sql-- 2 | 3 | select 4 | w_state 5 | ,i_item_id 6 | ,sum(case when (cast(d_date as date) < cast('2000-03-11' as date)) 7 | then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_before 8 | ,sum(case when (cast(d_date as date) >= cast('2000-03-11' as date)) 9 | then cs_sales_price - coalesce(cr_refunded_cash,0) else 0 end) as sales_after 10 | from 11 | catalog_sales left outer join catalog_returns on 12 | (cs_order_number = cr_order_number 13 | and cs_item_sk = cr_item_sk) 14 | ,warehouse, item, date_dim 15 | where 16 | i_current_price between 0.99 and 1.49 17 | and i_item_sk = cs_item_sk 18 | and cs_warehouse_sk = w_warehouse_sk 19 | and cs_sold_date_sk = d_date_sk 20 | and d_date between (cast('2000-03-11' as date) - interval '30' day) 21 | and (cast('2000-03-11' as date) + interval '30' day) 22 | group by w_state,i_item_id 23 | order by w_state,i_item_id 24 | limit 100 25 | 26 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q42.sql: -------------------------------------------------------------------------------- 1 | --q42.sql-- 2 | 3 | select dt.d_year, item.i_category_id, item.i_category, sum(ss_ext_sales_price) 4 | from date_dim dt, store_sales, item 5 | where dt.d_date_sk = store_sales.ss_sold_date_sk 6 | and store_sales.ss_item_sk = item.i_item_sk 7 | and item.i_manager_id = 1 8 | and dt.d_moy=11 9 | and dt.d_year=2000 10 | group by dt.d_year 11 | ,item.i_category_id 12 | ,item.i_category 13 | order by sum(ss_ext_sales_price) desc,dt.d_year 14 | ,item.i_category_id 15 | ,item.i_category 16 | limit 100 17 | 18 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q43.sql: -------------------------------------------------------------------------------- 1 | --q43.sql-- 2 | 3 | select s_store_name, s_store_id, 4 | sum(case when (d_day_name='Sunday') then ss_sales_price else null end) sun_sales, 5 | sum(case when (d_day_name='Monday') then ss_sales_price else null end) mon_sales, 6 | sum(case when (d_day_name='Tuesday') then ss_sales_price else null end) tue_sales, 7 | sum(case when (d_day_name='Wednesday') then ss_sales_price else null end) wed_sales, 8 | sum(case when (d_day_name='Thursday') then ss_sales_price else null end) thu_sales, 9 | sum(case when (d_day_name='Friday') then ss_sales_price else null end) fri_sales, 10 | sum(case when (d_day_name='Saturday') then ss_sales_price else null end) sat_sales 11 | from date_dim, store_sales, store 12 | where d_date_sk = ss_sold_date_sk and 13 | s_store_sk = ss_store_sk and 14 | s_gmt_offset = -5 and 15 | d_year = 2000 16 | group by s_store_name, s_store_id 17 | order by s_store_name, s_store_id,sun_sales,mon_sales,tue_sales,wed_sales, 18 | thu_sales,fri_sales,sat_sales 19 | limit 100 20 | 21 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q45.sql: -------------------------------------------------------------------------------- 1 | --q45.sql-- 2 | 3 | select ca_zip, ca_city, sum(ws_sales_price) 4 | from web_sales, customer, customer_address, date_dim, item 5 | where ws_bill_customer_sk = c_customer_sk 6 | and c_current_addr_sk = ca_address_sk 7 | and ws_item_sk = i_item_sk 8 | and ( substr(ca_zip,1,5) in ('85669', '86197','88274','83405','86475', '85392', '85460', '80348', '81792') 9 | or 10 | i_item_id in (select i_item_id 11 | from item 12 | where i_item_sk in (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) 13 | ) 14 | ) 15 | and ws_sold_date_sk = d_date_sk 16 | and d_qoy = 2 and d_year = 2001 17 | group by ca_zip, ca_city 18 | order by ca_zip, ca_city 19 | limit 100 20 | 21 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q46.sql: -------------------------------------------------------------------------------- 1 | --q46.sql-- 2 | 3 | select c_last_name, c_first_name, ca_city, bought_city, ss_ticket_number, amt,profit 4 | from 5 | (select ss_ticket_number 6 | ,ss_customer_sk 7 | ,ca_city bought_city 8 | ,sum(ss_coupon_amt) amt 9 | ,sum(ss_net_profit) profit 10 | from store_sales, date_dim, store, household_demographics, customer_address 11 | where store_sales.ss_sold_date_sk = date_dim.d_date_sk 12 | and store_sales.ss_store_sk = store.s_store_sk 13 | and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk 14 | and store_sales.ss_addr_sk = customer_address.ca_address_sk 15 | and (household_demographics.hd_dep_count = 4 or 16 | household_demographics.hd_vehicle_count= 3) 17 | and date_dim.d_dow in (6,0) 18 | and date_dim.d_year in (1999,1999+1,1999+2) 19 | and store.s_city in ('Fairview','Midway','Fairview','Fairview','Fairview') 20 | group by ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city) dn,customer,customer_address current_addr 21 | where ss_customer_sk = c_customer_sk 22 | and customer.c_current_addr_sk = current_addr.ca_address_sk 23 | and current_addr.ca_city <> bought_city 24 | order by c_last_name, c_first_name, ca_city, bought_city, ss_ticket_number 25 | limit 100 26 | 27 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q52.sql: -------------------------------------------------------------------------------- 1 | --q52.sql-- 2 | 3 | select dt.d_year 4 | ,item.i_brand_id brand_id 5 | ,item.i_brand brand 6 | ,sum(ss_ext_sales_price) ext_price 7 | from date_dim dt, store_sales, item 8 | where dt.d_date_sk = store_sales.ss_sold_date_sk 9 | and store_sales.ss_item_sk = item.i_item_sk 10 | and item.i_manager_id = 1 11 | and dt.d_moy=11 12 | and dt.d_year=2000 13 | group by dt.d_year, item.i_brand, item.i_brand_id 14 | order by dt.d_year, ext_price desc, brand_id 15 | limit 100 16 | 17 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q53.sql: -------------------------------------------------------------------------------- 1 | --q53.sql-- 2 | 3 | select * from 4 | (select i_manufact_id, 5 | sum(ss_sales_price) sum_sales, 6 | avg(sum(ss_sales_price)) over (partition by i_manufact_id) avg_quarterly_sales 7 | from item, store_sales, date_dim, store 8 | where ss_item_sk = i_item_sk and 9 | ss_sold_date_sk = d_date_sk and 10 | ss_store_sk = s_store_sk and 11 | d_month_seq in (1200,1200+1,1200+2,1200+3,1200+4,1200+5,1200+6, 12 | 1200+7,1200+8,1200+9,1200+10,1200+11) and 13 | ((i_category in ('Books','Children','Electronics') and 14 | i_class in ('personal','portable','reference','self-help') and 15 | i_brand in ('scholaramalgamalg #14','scholaramalgamalg #7', 16 | 'exportiunivamalg #9','scholaramalgamalg #9')) 17 | or 18 | (i_category in ('Women','Music','Men') and 19 | i_class in ('accessories','classical','fragrances','pants') and 20 | i_brand in ('amalgimporto #1','edu packscholar #1','exportiimporto #1', 21 | 'importoamalg #1'))) 22 | group by i_manufact_id, d_qoy ) tmp1 23 | where case when avg_quarterly_sales > 0 24 | then abs (sum_sales - avg_quarterly_sales)/ avg_quarterly_sales 25 | else null end > 0.1 26 | order by avg_quarterly_sales, 27 | sum_sales, 28 | i_manufact_id 29 | limit 100 30 | 31 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q55.sql: -------------------------------------------------------------------------------- 1 | --q55.sql-- 2 | 3 | select i_brand_id brand_id, i_brand brand, 4 | sum(ss_ext_sales_price) ext_price 5 | from date_dim, store_sales, item 6 | where d_date_sk = ss_sold_date_sk 7 | and ss_item_sk = i_item_sk 8 | and i_manager_id=28 9 | and d_moy=11 10 | and d_year=1999 11 | group by i_brand, i_brand_id 12 | order by ext_price desc, brand_id 13 | limit 100 14 | 15 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q6.sql: -------------------------------------------------------------------------------- 1 | --q6.sql-- 2 | 3 | SELECT state, cnt FROM ( 4 | SELECT a.ca_state state, count(*) cnt 5 | FROM 6 | customer_address a, customer c, store_sales s, date_dim d, item i 7 | WHERE a.ca_address_sk = c.c_current_addr_sk 8 | AND c.c_customer_sk = s.ss_customer_sk 9 | AND s.ss_sold_date_sk = d.d_date_sk 10 | AND s.ss_item_sk = i.i_item_sk 11 | AND d.d_month_seq = 12 | (SELECT distinct (d_month_seq) FROM date_dim 13 | WHERE d_year = 2001 AND d_moy = 1) 14 | AND i.i_current_price > 1.2 * 15 | (SELECT avg(j.i_current_price) FROM item j 16 | WHERE j.i_category = i.i_category) 17 | GROUP BY a.ca_state 18 | ) x 19 | WHERE cnt >= 10 20 | ORDER BY cnt LIMIT 100 21 | 22 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q61.sql: -------------------------------------------------------------------------------- 1 | --q61.sql-- 2 | 3 | select promotions,total,cast(promotions as decimal(15,4))/cast(total as decimal(15,4))*100 4 | from 5 | (select sum(ss_ext_sales_price) promotions 6 | from store_sales, store, promotion, date_dim, customer, customer_address, item 7 | where ss_sold_date_sk = d_date_sk 8 | and ss_store_sk = s_store_sk 9 | and ss_promo_sk = p_promo_sk 10 | and ss_customer_sk= c_customer_sk 11 | and ca_address_sk = c_current_addr_sk 12 | and ss_item_sk = i_item_sk 13 | and ca_gmt_offset = -5 14 | and i_category = 'Jewelry' 15 | and (p_channel_dmail = 'Y' or p_channel_email = 'Y' or p_channel_tv = 'Y') 16 | and s_gmt_offset = -5 17 | and d_year = 1998 18 | and d_moy = 11) promotional_sales cross join 19 | (select sum(ss_ext_sales_price) total 20 | from store_sales, store, date_dim, customer, customer_address, item 21 | where ss_sold_date_sk = d_date_sk 22 | and ss_store_sk = s_store_sk 23 | and ss_customer_sk= c_customer_sk 24 | and ca_address_sk = c_current_addr_sk 25 | and ss_item_sk = i_item_sk 26 | and ca_gmt_offset = -5 27 | and i_category = 'Jewelry' 28 | and s_gmt_offset = -5 29 | and d_year = 1998 30 | and d_moy = 11) all_sales 31 | order by promotions, total 32 | limit 100 33 | 34 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q62.sql: -------------------------------------------------------------------------------- 1 | --q62.sql-- 2 | 3 | select 4 | substr(w_warehouse_name,1,20) 5 | ,sm_type 6 | ,web_name 7 | ,sum(case when (ws_ship_date_sk - ws_sold_date_sk <= 30 ) then 1 else 0 end) as `30 days` 8 | ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 30) and 9 | (ws_ship_date_sk - ws_sold_date_sk <= 60) then 1 else 0 end ) as `31-60 days` 10 | ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 60) and 11 | (ws_ship_date_sk - ws_sold_date_sk <= 90) then 1 else 0 end) as `61-90 days` 12 | ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 90) and 13 | (ws_ship_date_sk - ws_sold_date_sk <= 120) then 1 else 0 end) as `91-120 days` 14 | ,sum(case when (ws_ship_date_sk - ws_sold_date_sk > 120) then 1 else 0 end) as `>120 days` 15 | from 16 | web_sales, warehouse, ship_mode, web_site, date_dim 17 | where 18 | d_month_seq between 1200 and 1200 + 11 19 | and ws_ship_date_sk = d_date_sk 20 | and ws_warehouse_sk = w_warehouse_sk 21 | and ws_ship_mode_sk = sm_ship_mode_sk 22 | and ws_web_site_sk = web_site_sk 23 | group by 24 | substr(w_warehouse_name,1,20), sm_type, web_name 25 | order by 26 | substr(w_warehouse_name,1,20), sm_type, web_name 27 | limit 100 28 | 29 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q65.sql: -------------------------------------------------------------------------------- 1 | --q65.sql-- 2 | 3 | select 4 | s_store_name, i_item_desc, sc.revenue, i_current_price, i_wholesale_cost, i_brand 5 | from store, item, 6 | (select ss_store_sk, avg(revenue) as ave 7 | from 8 | (select ss_store_sk, ss_item_sk, 9 | sum(ss_sales_price) as revenue 10 | from store_sales, date_dim 11 | where ss_sold_date_sk = d_date_sk and d_month_seq between 1176 and 1176+11 12 | group by ss_store_sk, ss_item_sk) sa 13 | group by ss_store_sk) sb, 14 | (select ss_store_sk, ss_item_sk, sum(ss_sales_price) as revenue 15 | from store_sales, date_dim 16 | where ss_sold_date_sk = d_date_sk and d_month_seq between 1176 and 1176+11 17 | group by ss_store_sk, ss_item_sk) sc 18 | where sb.ss_store_sk = sc.ss_store_sk and 19 | sc.revenue <= 0.1 * sb.ave and 20 | s_store_sk = sc.ss_store_sk and 21 | i_item_sk = sc.ss_item_sk 22 | order by s_store_name, i_item_desc 23 | limit 100 24 | 25 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q67.sql: -------------------------------------------------------------------------------- 1 | --q67.sql-- 2 | 3 | select * from 4 | (select i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy, s_store_id, 5 | sumsales, rank() over (partition by i_category order by sumsales desc) rk 6 | from 7 | (select i_category, i_class, i_brand, i_product_name, d_year, d_qoy, d_moy, 8 | s_store_id, sum(coalesce(ss_sales_price*ss_quantity,0)) sumsales 9 | from store_sales, date_dim, store, item 10 | where ss_sold_date_sk=d_date_sk 11 | and ss_item_sk=i_item_sk 12 | and ss_store_sk = s_store_sk 13 | and d_month_seq between 1200 and 1200+11 14 | group by rollup(i_category, i_class, i_brand, i_product_name, d_year, d_qoy, 15 | d_moy,s_store_id))dw1) dw2 16 | where rk <= 100 17 | order by 18 | i_category, i_class, i_brand, i_product_name, d_year, 19 | d_qoy, d_moy, s_store_id, sumsales, rk 20 | limit 100 21 | 22 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q68.sql: -------------------------------------------------------------------------------- 1 | --q68.sql-- 2 | 3 | select 4 | c_last_name, c_first_name, ca_city, bought_city, ss_ticket_number, extended_price, 5 | extended_tax, list_price 6 | from (select 7 | ss_ticket_number, ss_customer_sk, ca_city bought_city, 8 | sum(ss_ext_sales_price) extended_price, 9 | sum(ss_ext_list_price) list_price, 10 | sum(ss_ext_tax) extended_tax 11 | from store_sales, date_dim, store, household_demographics, customer_address 12 | where store_sales.ss_sold_date_sk = date_dim.d_date_sk 13 | and store_sales.ss_store_sk = store.s_store_sk 14 | and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk 15 | and store_sales.ss_addr_sk = customer_address.ca_address_sk 16 | and date_dim.d_dom between 1 and 2 17 | and (household_demographics.hd_dep_count = 4 or 18 | household_demographics.hd_vehicle_count = 3) 19 | and date_dim.d_year in (1999,1999+1,1999+2) 20 | and store.s_city in ('Midway','Fairview') 21 | group by ss_ticket_number, ss_customer_sk, ss_addr_sk,ca_city) dn, 22 | customer, 23 | customer_address current_addr 24 | where ss_customer_sk = c_customer_sk 25 | and customer.c_current_addr_sk = current_addr.ca_address_sk 26 | and current_addr.ca_city <> bought_city 27 | order by c_last_name, ss_ticket_number 28 | limit 100 29 | 30 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q7.sql: -------------------------------------------------------------------------------- 1 | --q7.sql-- 2 | 3 | SELECT i_item_id, 4 | avg(ss_quantity) agg1, 5 | avg(ss_list_price) agg2, 6 | avg(ss_coupon_amt) agg3, 7 | avg(ss_sales_price) agg4 8 | FROM store_sales, customer_demographics, date_dim, item, promotion 9 | WHERE ss_sold_date_sk = d_date_sk AND 10 | ss_item_sk = i_item_sk AND 11 | ss_cdemo_sk = cd_demo_sk AND 12 | ss_promo_sk = p_promo_sk AND 13 | cd_gender = 'M' AND 14 | cd_marital_status = 'S' AND 15 | cd_education_status = 'College' AND 16 | (p_channel_email = 'N' or p_channel_event = 'N') AND 17 | d_year = 2000 18 | GROUP BY i_item_id 19 | ORDER BY i_item_id LIMIT 100 20 | 21 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q70.sql: -------------------------------------------------------------------------------- 1 | --q70.sql-- 2 | 3 | select 4 | sum(ss_net_profit) as total_sum, s_state, s_county 5 | ,grouping(s_state)+grouping(s_county) as lochierarchy 6 | ,rank() over ( 7 | partition by grouping(s_state)+grouping(s_county), 8 | case when grouping(s_county) = 0 then s_state end 9 | order by sum(ss_net_profit) desc) as rank_within_parent 10 | from 11 | store_sales, date_dim d1, store 12 | where 13 | d1.d_month_seq between 1200 and 1200+11 14 | and d1.d_date_sk = ss_sold_date_sk 15 | and s_store_sk = ss_store_sk 16 | and s_state in 17 | (select s_state from 18 | (select s_state as s_state, 19 | rank() over ( partition by s_state order by sum(ss_net_profit) desc) as ranking 20 | from store_sales, store, date_dim 21 | where d_month_seq between 1200 and 1200+11 22 | and d_date_sk = ss_sold_date_sk 23 | and s_store_sk = ss_store_sk 24 | group by s_state) tmp1 25 | where ranking <= 5) 26 | group by rollup(s_state,s_county) 27 | order by 28 | lochierarchy desc 29 | ,case when lochierarchy = 0 then s_state end 30 | ,rank_within_parent 31 | limit 100 32 | 33 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q72.sql: -------------------------------------------------------------------------------- 1 | --q72.sql-- 2 | 3 | select i_item_desc 4 | ,w_warehouse_name 5 | ,d1.d_week_seq 6 | ,sum(case when p_promo_sk is null then 1 else 0 end) no_promo 7 | ,sum(case when p_promo_sk is not null then 1 else 0 end) promo 8 | ,count(*) total_cnt 9 | from catalog_sales 10 | join inventory on (cs_item_sk = inv_item_sk) 11 | join warehouse on (w_warehouse_sk=inv_warehouse_sk) 12 | join item on (i_item_sk = cs_item_sk) 13 | join customer_demographics on (cs_bill_cdemo_sk = cd_demo_sk) 14 | join household_demographics on (cs_bill_hdemo_sk = hd_demo_sk) 15 | join date_dim d1 on (cs_sold_date_sk = d1.d_date_sk) 16 | join date_dim d2 on (inv_date_sk = d2.d_date_sk) 17 | join date_dim d3 on (cs_ship_date_sk = d3.d_date_sk) 18 | left outer join promotion on (cs_promo_sk=p_promo_sk) 19 | left outer join catalog_returns on (cr_item_sk = cs_item_sk and cr_order_number = cs_order_number) 20 | where d1.d_week_seq = d2.d_week_seq 21 | and inv_quantity_on_hand < cs_quantity 22 | and d3.d_date > (cast(d1.d_date AS DATE) + interval '5' day) 23 | and hd_buy_potential = '>10000' 24 | and d1.d_year = 1999 25 | and cd_marital_status = 'D' 26 | group by i_item_desc,w_warehouse_name,d1.d_week_seq 27 | order by total_cnt desc, i_item_desc, w_warehouse_name, d1.d_week_seq 28 | limit 100 29 | 30 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q73.sql: -------------------------------------------------------------------------------- 1 | --q73.sql-- 2 | 3 | select 4 | c_last_name, c_first_name, c_salutation, c_preferred_cust_flag, 5 | ss_ticket_number, cnt from 6 | (select ss_ticket_number, ss_customer_sk, count(*) cnt 7 | from store_sales,date_dim,store,household_demographics 8 | where store_sales.ss_sold_date_sk = date_dim.d_date_sk 9 | and store_sales.ss_store_sk = store.s_store_sk 10 | and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk 11 | and date_dim.d_dom between 1 and 2 12 | and (household_demographics.hd_buy_potential = '>10000' or 13 | household_demographics.hd_buy_potential = 'unknown') 14 | and household_demographics.hd_vehicle_count > 0 15 | and case when household_demographics.hd_vehicle_count > 0 then 16 | household_demographics.hd_dep_count/ household_demographics.hd_vehicle_count else null end > 1 17 | and date_dim.d_year in (1999,1999+1,1999+2) 18 | and store.s_county in ('Williamson County','Franklin Parish','Bronx County','Orange County') 19 | group by ss_ticket_number,ss_customer_sk) dj,customer 20 | where ss_customer_sk = c_customer_sk 21 | and cnt between 1 and 5 22 | order by cnt desc, c_last_name asc 23 | 24 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q76.sql: -------------------------------------------------------------------------------- 1 | --q76.sql-- 2 | 3 | SELECT 4 | channel, col_name, d_year, d_qoy, i_category, COUNT(*) sales_cnt, 5 | SUM(ext_sales_price) sales_amt 6 | FROM( 7 | SELECT 8 | 'store' as channel, ss_store_sk col_name, d_year, d_qoy, i_category, 9 | ss_ext_sales_price ext_sales_price 10 | FROM store_sales, item, date_dim 11 | WHERE ss_store_sk IS NULL 12 | AND ss_sold_date_sk=d_date_sk 13 | AND ss_item_sk=i_item_sk 14 | UNION ALL 15 | SELECT 16 | 'web' as channel, ws_ship_customer_sk col_name, d_year, d_qoy, i_category, 17 | ws_ext_sales_price ext_sales_price 18 | FROM web_sales, item, date_dim 19 | WHERE ws_ship_customer_sk IS NULL 20 | AND ws_sold_date_sk=d_date_sk 21 | AND ws_item_sk=i_item_sk 22 | UNION ALL 23 | SELECT 24 | 'catalog' as channel, cs_ship_addr_sk col_name, d_year, d_qoy, i_category, 25 | cs_ext_sales_price ext_sales_price 26 | FROM catalog_sales, item, date_dim 27 | WHERE cs_ship_addr_sk IS NULL 28 | AND cs_sold_date_sk=d_date_sk 29 | AND cs_item_sk=i_item_sk) foo 30 | GROUP BY channel, col_name, d_year, d_qoy, i_category 31 | ORDER BY channel, col_name, d_year, d_qoy, i_category 32 | limit 100 33 | 34 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q79.sql: -------------------------------------------------------------------------------- 1 | --q79.sql-- 2 | 3 | select 4 | c_last_name,c_first_name,substr(s_city,1,30),ss_ticket_number,amt,profit 5 | from 6 | (select ss_ticket_number 7 | ,ss_customer_sk 8 | ,store.s_city 9 | ,sum(ss_coupon_amt) amt 10 | ,sum(ss_net_profit) profit 11 | from store_sales,date_dim,store,household_demographics 12 | where store_sales.ss_sold_date_sk = date_dim.d_date_sk 13 | and store_sales.ss_store_sk = store.s_store_sk 14 | and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk 15 | and (household_demographics.hd_dep_count = 6 or 16 | household_demographics.hd_vehicle_count > 2) 17 | and date_dim.d_dow = 1 18 | and date_dim.d_year in (1999,1999+1,1999+2) 19 | and store.s_number_employees between 200 and 295 20 | group by ss_ticket_number,ss_customer_sk,ss_addr_sk,store.s_city) ms,customer 21 | where ss_customer_sk = c_customer_sk 22 | order by c_last_name,c_first_name,substr(s_city,1,30), profit 23 | limit 100 24 | 25 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q81.sql: -------------------------------------------------------------------------------- 1 | --q81.sql-- 2 | 3 | with customer_total_return as 4 | (select 5 | cr_returning_customer_sk as ctr_customer_sk, ca_state as ctr_state, 6 | sum(cr_return_amt_inc_tax) as ctr_total_return 7 | from catalog_returns, date_dim, customer_address 8 | where cr_returned_date_sk = d_date_sk 9 | and d_year = 2000 10 | and cr_returning_addr_sk = ca_address_sk 11 | group by cr_returning_customer_sk, ca_state ) 12 | select 13 | c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name, 14 | ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country, 15 | ca_gmt_offset,ca_location_type,ctr_total_return 16 | from customer_total_return ctr1, customer_address, customer 17 | where ctr1.ctr_total_return > (select avg(ctr_total_return)*1.2 18 | from customer_total_return ctr2 19 | where ctr1.ctr_state = ctr2.ctr_state) 20 | and ca_address_sk = c_current_addr_sk 21 | and ca_state = 'GA' 22 | and ctr1.ctr_customer_sk = c_customer_sk 23 | order by c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name 24 | ,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset 25 | ,ca_location_type,ctr_total_return 26 | limit 100 27 | 28 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q82.sql: -------------------------------------------------------------------------------- 1 | --q82.sql-- 2 | 3 | select i_item_id, i_item_desc, i_current_price 4 | from item, inventory, date_dim, store_sales 5 | where i_current_price between 62 and 62+30 6 | and inv_item_sk = i_item_sk 7 | and d_date_sk=inv_date_sk 8 | and d_date between cast('2000-05-25' as date) and (cast('2000-05-25' as date) + interval '60' day) 9 | and i_manufact_id in (129, 270, 821, 423) 10 | and inv_quantity_on_hand between 100 and 500 11 | and ss_item_sk = i_item_sk 12 | group by i_item_id,i_item_desc,i_current_price 13 | order by i_item_id 14 | limit 100 15 | 16 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q84.sql: -------------------------------------------------------------------------------- 1 | --q84.sql-- 2 | 3 | select c_customer_id as customer_id 4 | ,coalesce(c_last_name,'') + ', ' + coalesce(c_first_name,'') as customername 5 | from customer 6 | ,customer_address 7 | ,customer_demographics 8 | ,household_demographics 9 | ,income_band 10 | ,store_returns 11 | where ca_city = 'Edgewood' 12 | and c_current_addr_sk = ca_address_sk 13 | and ib_lower_bound >= 38128 14 | and ib_upper_bound <= 38128 + 50000 15 | and ib_income_band_sk = hd_income_band_sk 16 | and cd_demo_sk = c_current_cdemo_sk 17 | and hd_demo_sk = c_current_hdemo_sk 18 | and sr_cdemo_sk = cd_demo_sk 19 | order by c_customer_id 20 | limit 100 21 | 22 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q86.sql: -------------------------------------------------------------------------------- 1 | --q86.sql-- 2 | 3 | select sum(ws_net_paid) as total_sum, i_category, i_class, 4 | grouping(i_category)+grouping(i_class) as lochierarchy, 5 | rank() over ( 6 | partition by grouping(i_category)+grouping(i_class), 7 | case when grouping(i_class) = 0 then i_category end 8 | order by sum(ws_net_paid) desc) as rank_within_parent 9 | from 10 | web_sales, date_dim d1, item 11 | where 12 | d1.d_month_seq between 1200 and 1200+11 13 | and d1.d_date_sk = ws_sold_date_sk 14 | and i_item_sk = ws_item_sk 15 | group by rollup(i_category,i_class) 16 | order by 17 | lochierarchy desc, 18 | case when lochierarchy = 0 then i_category end, 19 | rank_within_parent 20 | limit 100 21 | 22 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q87.sql: -------------------------------------------------------------------------------- 1 | --q87.sql-- 2 | 3 | select count(*) 4 | from ((select distinct c_last_name, c_first_name, d_date 5 | from store_sales, date_dim, customer 6 | where store_sales.ss_sold_date_sk = date_dim.d_date_sk 7 | and store_sales.ss_customer_sk = customer.c_customer_sk 8 | and d_month_seq between 1200 and 1200+11) 9 | except 10 | (select distinct c_last_name, c_first_name, d_date 11 | from catalog_sales, date_dim, customer 12 | where catalog_sales.cs_sold_date_sk = date_dim.d_date_sk 13 | and catalog_sales.cs_bill_customer_sk = customer.c_customer_sk 14 | and d_month_seq between 1200 and 1200+11) 15 | except 16 | (select distinct c_last_name, c_first_name, d_date 17 | from web_sales, date_dim, customer 18 | where web_sales.ws_sold_date_sk = date_dim.d_date_sk 19 | and web_sales.ws_bill_customer_sk = customer.c_customer_sk 20 | and d_month_seq between 1200 and 1200+11) 21 | ) cool_cust 22 | 23 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q89.sql: -------------------------------------------------------------------------------- 1 | --q89.sql-- 2 | 3 | select * 4 | from( 5 | select i_category, i_class, i_brand, 6 | s_store_name, s_company_name, 7 | d_moy, 8 | sum(ss_sales_price) sum_sales, 9 | avg(sum(ss_sales_price)) over 10 | (partition by i_category, i_brand, s_store_name, s_company_name) 11 | avg_monthly_sales 12 | from item, store_sales, date_dim, store 13 | where ss_item_sk = i_item_sk and 14 | ss_sold_date_sk = d_date_sk and 15 | ss_store_sk = s_store_sk and 16 | d_year in (1999) and 17 | ((i_category in ('Books','Electronics','Sports') and 18 | i_class in ('computers','stereo','football')) 19 | or (i_category in ('Men','Jewelry','Women') and 20 | i_class in ('shirts','birdal','dresses'))) 21 | group by i_category, i_class, i_brand, 22 | s_store_name, s_company_name, d_moy) tmp1 23 | where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) else null end > 0.1 24 | order by sum_sales - avg_monthly_sales, s_store_name 25 | limit 100 26 | 27 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q90.sql: -------------------------------------------------------------------------------- 1 | --q90.sql-- 2 | 3 | select cast(amc as decimal(15,4))/cast(pmc as decimal(15,4)) am_pm_ratio 4 | from ( select count(*) amc 5 | from web_sales, household_demographics , time_dim, web_page 6 | where ws_sold_time_sk = time_dim.t_time_sk 7 | and ws_ship_hdemo_sk = household_demographics.hd_demo_sk 8 | and ws_web_page_sk = web_page.wp_web_page_sk 9 | and time_dim.t_hour between 8 and 8+1 10 | and household_demographics.hd_dep_count = 6 11 | and web_page.wp_char_count between 5000 and 5200) at cross join 12 | ( select count(*) pmc 13 | from web_sales, household_demographics , time_dim, web_page 14 | where ws_sold_time_sk = time_dim.t_time_sk 15 | and ws_ship_hdemo_sk = household_demographics.hd_demo_sk 16 | and ws_web_page_sk = web_page.wp_web_page_sk 17 | and time_dim.t_hour between 19 and 19+1 18 | and household_demographics.hd_dep_count = 6 19 | and web_page.wp_char_count between 5000 and 5200) pt 20 | order by am_pm_ratio 21 | limit 100 22 | 23 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q91.sql: -------------------------------------------------------------------------------- 1 | --q91.sql-- 2 | 3 | select 4 | cc_call_center_id Call_Center, cc_name Call_Center_Name, cc_manager Manager, 5 | sum(cr_net_loss) Returns_Loss 6 | from 7 | call_center, catalog_returns, date_dim, customer, customer_address, 8 | customer_demographics, household_demographics 9 | where 10 | cr_call_center_sk = cc_call_center_sk 11 | and cr_returned_date_sk = d_date_sk 12 | and cr_returning_customer_sk = c_customer_sk 13 | and cd_demo_sk = c_current_cdemo_sk 14 | and hd_demo_sk = c_current_hdemo_sk 15 | and ca_address_sk = c_current_addr_sk 16 | and d_year = 1998 17 | and d_moy = 11 18 | and ( (cd_marital_status = 'M' and cd_education_status = 'Unknown') 19 | or(cd_marital_status = 'W' and cd_education_status = 'Advanced Degree')) 20 | and hd_buy_potential like 'Unknown%' 21 | and ca_gmt_offset = -7 22 | group by cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status 23 | order by sum(cr_net_loss) desc 24 | 25 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q92.sql: -------------------------------------------------------------------------------- 1 | --q92.sql-- 2 | 3 | select sum(ws_ext_discount_amt) as `Excess Discount Amount` 4 | from web_sales, item, date_dim 5 | where i_manufact_id = 350 6 | and i_item_sk = ws_item_sk 7 | and d_date between cast ('2000-01-27' as date) and (cast('2000-01-27' as date) + interval '90' day) 8 | and d_date_sk = ws_sold_date_sk 9 | and ws_ext_discount_amt > 10 | ( 11 | SELECT 1.3 * avg(ws_ext_discount_amt) 12 | FROM web_sales, date_dim 13 | WHERE ws_item_sk = i_item_sk 14 | and d_date between cast ('2000-01-27' as date) and (cast('2000-01-27' as date) + interval '90' day) 15 | and d_date_sk = ws_sold_date_sk 16 | ) 17 | order by sum(ws_ext_discount_amt) 18 | limit 100 19 | 20 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q93.sql: -------------------------------------------------------------------------------- 1 | --q93.sql-- 2 | 3 | select ss_customer_sk, sum(act_sales) sumsales 4 | from (select 5 | ss_item_sk, ss_ticket_number, ss_customer_sk, 6 | case when sr_return_quantity is not null then (ss_quantity-sr_return_quantity)*ss_sales_price 7 | else (ss_quantity*ss_sales_price) end act_sales 8 | from store_sales 9 | left outer join store_returns 10 | on (sr_item_sk = ss_item_sk and sr_ticket_number = ss_ticket_number), 11 | reason 12 | where sr_reason_sk = r_reason_sk and r_reason_desc = 'reason 28') t 13 | group by ss_customer_sk 14 | order by sumsales, ss_customer_sk 15 | limit 100 16 | 17 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q94.sql: -------------------------------------------------------------------------------- 1 | --q94.sql-- 2 | 3 | select 4 | count(distinct ws_order_number) as `order count` 5 | ,sum(ws_ext_ship_cost) as `total shipping cost` 6 | ,sum(ws_net_profit) as `total net profit` 7 | from 8 | web_sales ws1, date_dim, customer_address, web_site 9 | where 10 | d_date between cast('1999-02-01' as date) and 11 | (cast('1999-02-01' as date) + interval '60' day) 12 | and ws1.ws_ship_date_sk = d_date_sk 13 | and ws1.ws_ship_addr_sk = ca_address_sk 14 | and ca_state = 'IL' 15 | and ws1.ws_web_site_sk = web_site_sk 16 | and web_company_name = 'pri' 17 | and exists (select * 18 | from web_sales ws2 19 | where ws1.ws_order_number = ws2.ws_order_number 20 | and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) 21 | and not exists(select * 22 | from web_returns wr1 23 | where ws1.ws_order_number = wr1.wr_order_number) 24 | order by count(distinct ws_order_number) 25 | limit 100 26 | 27 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q95.sql: -------------------------------------------------------------------------------- 1 | --q95.sql-- 2 | 3 | with ws_wh as 4 | (select ws1.ws_order_number,ws1.ws_warehouse_sk wh1,ws2.ws_warehouse_sk wh2 5 | from web_sales ws1,web_sales ws2 6 | where ws1.ws_order_number = ws2.ws_order_number 7 | and ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) 8 | select 9 | count(distinct ws_order_number) as `order count` 10 | ,sum(ws_ext_ship_cost) as `total shipping cost` 11 | ,sum(ws_net_profit) as `total net profit` 12 | from 13 | web_sales ws1, date_dim, customer_address, web_site 14 | where 15 | d_date between cast ('1999-02-01' as date) and 16 | (cast('1999-02-01' as date) + interval '60' day) 17 | and ws1.ws_ship_date_sk = d_date_sk 18 | and ws1.ws_ship_addr_sk = ca_address_sk 19 | and ca_state = 'IL' 20 | and ws1.ws_web_site_sk = web_site_sk 21 | and web_company_name = 'pri' 22 | and ws1.ws_order_number in (select ws_order_number 23 | from ws_wh) 24 | and ws1.ws_order_number in (select wr_order_number 25 | from web_returns,ws_wh 26 | where wr_order_number = ws_wh.ws_order_number) 27 | order by count(distinct ws_order_number) 28 | limit 100 29 | 30 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q96.sql: -------------------------------------------------------------------------------- 1 | --q96.sql-- 2 | 3 | select count(*) 4 | from store_sales, household_demographics, time_dim, store 5 | where ss_sold_time_sk = time_dim.t_time_sk 6 | and ss_hdemo_sk = household_demographics.hd_demo_sk 7 | and ss_store_sk = s_store_sk 8 | and time_dim.t_hour = 20 9 | and time_dim.t_minute >= 30 10 | and household_demographics.hd_dep_count = 7 11 | and store.s_store_name = 'ese' 12 | order by count(*) 13 | limit 100 14 | 15 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q97.sql: -------------------------------------------------------------------------------- 1 | --q97.sql-- 2 | 3 | with ssci as ( 4 | select ss_customer_sk customer_sk, ss_item_sk item_sk 5 | from store_sales,date_dim 6 | where ss_sold_date_sk = d_date_sk 7 | and d_month_seq between 1200 and 1200 + 11 8 | group by ss_customer_sk, ss_item_sk), 9 | csci as( 10 | select cs_bill_customer_sk customer_sk, cs_item_sk item_sk 11 | from catalog_sales,date_dim 12 | where cs_sold_date_sk = d_date_sk 13 | and d_month_seq between 1200 and 1200 + 11 14 | group by cs_bill_customer_sk, cs_item_sk) 15 | select sum(case when ssci.customer_sk is not null and csci.customer_sk is null then 1 else 0 end) store_only 16 | ,sum(case when ssci.customer_sk is null and csci.customer_sk is not null then 1 else 0 end) catalog_only 17 | ,sum(case when ssci.customer_sk is not null and csci.customer_sk is not null then 1 else 0 end) store_and_catalog 18 | from ssci full outer join csci on (ssci.customer_sk=csci.customer_sk 19 | and ssci.item_sk = csci.item_sk) 20 | limit 100 21 | 22 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q98.sql: -------------------------------------------------------------------------------- 1 | --q98.sql-- 2 | 3 | select i_item_desc, i_category, i_class, i_current_price 4 | ,sum(ss_ext_sales_price) as itemrevenue 5 | ,sum(ss_ext_sales_price)*100/sum(sum(ss_ext_sales_price)) over 6 | (partition by i_class) as revenueratio 7 | from 8 | store_sales, item, date_dim 9 | where 10 | ss_item_sk = i_item_sk 11 | and i_category in ('Sports', 'Books', 'Home') 12 | and ss_sold_date_sk = d_date_sk 13 | and d_date between cast('1999-02-22' as date) 14 | and (cast('1999-02-22' as date) + interval '30' day) 15 | group by 16 | i_item_id, i_item_desc, i_category, i_class, i_current_price 17 | order by 18 | i_category, i_class, i_item_id, i_item_desc, revenueratio 19 | 20 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/q99.sql: -------------------------------------------------------------------------------- 1 | --q99.sql-- 2 | 3 | select 4 | substr(w_warehouse_name,1,20), sm_type, cc_name 5 | ,sum(case when (cs_ship_date_sk - cs_sold_date_sk <= 30 ) then 1 else 0 end) as `30 days` 6 | ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 30) and 7 | (cs_ship_date_sk - cs_sold_date_sk <= 60) then 1 else 0 end ) as `31-60 days` 8 | ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 60) and 9 | (cs_ship_date_sk - cs_sold_date_sk <= 90) then 1 else 0 end) as `61-90 days` 10 | ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 90) and 11 | (cs_ship_date_sk - cs_sold_date_sk <= 120) then 1 else 0 end) as `91-120 days` 12 | ,sum(case when (cs_ship_date_sk - cs_sold_date_sk > 120) then 1 else 0 end) as `>120 days` 13 | from 14 | catalog_sales, warehouse, ship_mode, call_center, date_dim 15 | where 16 | d_month_seq between 1200 and 1200 + 11 17 | and cs_ship_date_sk = d_date_sk 18 | and cs_warehouse_sk = w_warehouse_sk 19 | and cs_ship_mode_sk = sm_ship_mode_sk 20 | and cs_call_center_sk = cc_call_center_sk 21 | group by 22 | substr(w_warehouse_name,1,20), sm_type, cc_name 23 | order by substr(w_warehouse_name,1,20), sm_type, cc_name 24 | limit 100 25 | 26 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/ss_max.sql: -------------------------------------------------------------------------------- 1 | select 2 | count(*) as total, 3 | count(ss_sold_date_sk) as not_null_total, 4 | count(distinct ss_sold_date_sk) as unique_days, 5 | max(ss_sold_date_sk) as max_ss_sold_date_sk, 6 | max(ss_sold_time_sk) as max_ss_sold_time_sk, 7 | max(ss_item_sk) as max_ss_item_sk, 8 | max(ss_customer_sk) as max_ss_customer_sk, 9 | max(ss_cdemo_sk) as max_ss_cdemo_sk, 10 | max(ss_hdemo_sk) as max_ss_hdemo_sk, 11 | max(ss_addr_sk) as max_ss_addr_sk, 12 | max(ss_store_sk) as max_ss_store_sk, 13 | max(ss_promo_sk) as max_ss_promo_sk 14 | from store_sales 15 | -------------------------------------------------------------------------------- /core/src/test/resources/tpcds-sql/ss_maxb.sql: -------------------------------------------------------------------------------- 1 | select 2 | count(*) as total, 3 | count(ss_sold_date_sk) as not_null_total, 4 | --count(distinct ss_sold_date_sk) as unique_days, 5 | max(ss_sold_date_sk) as max_ss_sold_date_sk, 6 | max(ss_sold_time_sk) as max_ss_sold_time_sk, 7 | max(ss_item_sk) as max_ss_item_sk, 8 | max(ss_customer_sk) as max_ss_customer_sk, 9 | max(ss_cdemo_sk) as max_ss_cdemo_sk, 10 | max(ss_hdemo_sk) as max_ss_hdemo_sk, 11 | max(ss_addr_sk) as max_ss_addr_sk, 12 | max(ss_store_sk) as max_ss_store_sk, 13 | max(ss_promo_sk) as max_ss_promo_sk 14 | from store_sales -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q1.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | l_returnflag, 5 | l_linestatus, 6 | sum(l_quantity) as sum_qty, 7 | sum(l_extendedprice) as sum_base_price, 8 | sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, 9 | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, 10 | avg(l_quantity) as avg_qty, 11 | avg(l_extendedprice) as avg_price, 12 | avg(l_discount) as avg_disc, 13 | count(*) as count_order 14 | from 15 | lineitem 16 | where 17 | l_shipdate <= date '1998-12-01' - interval '90' day 18 | group by 19 | l_returnflag, 20 | l_linestatus 21 | order by 22 | l_returnflag, 23 | l_linestatus 24 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q10.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | c_custkey, 5 | c_name, 6 | sum(l_extendedprice * (1 - l_discount)) as revenue, 7 | c_acctbal, 8 | n_name, 9 | c_address, 10 | c_phone, 11 | c_comment 12 | from 13 | customer, 14 | orders, 15 | lineitem, 16 | nation 17 | where 18 | c_custkey = o_custkey 19 | and l_orderkey = o_orderkey 20 | and o_orderdate >= date '1993-10-01' 21 | and o_orderdate < date '1993-10-01' + interval '3' month 22 | and l_returnflag = 'R' 23 | and c_nationkey = n_nationkey 24 | group by 25 | c_custkey, 26 | c_name, 27 | c_acctbal, 28 | c_phone, 29 | n_name, 30 | c_address, 31 | c_comment 32 | order by 33 | revenue desc 34 | limit 20 35 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q11.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | ps_partkey, 5 | sum(ps_supplycost * ps_availqty) as value 6 | from 7 | partsupp, 8 | supplier, 9 | nation 10 | where 11 | ps_suppkey = s_suppkey 12 | and s_nationkey = n_nationkey 13 | and n_name = 'GERMANY' 14 | group by 15 | ps_partkey having 16 | sum(ps_supplycost * ps_availqty) > ( 17 | select 18 | sum(ps_supplycost * ps_availqty) * 0.0001000000 19 | from 20 | partsupp, 21 | supplier, 22 | nation 23 | where 24 | ps_suppkey = s_suppkey 25 | and s_nationkey = n_nationkey 26 | and n_name = 'GERMANY' 27 | ) 28 | order by 29 | value desc 30 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q12.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | l_shipmode, 5 | sum(case 6 | when o_orderpriority = '1-URGENT' 7 | or o_orderpriority = '2-HIGH' 8 | then 1 9 | else 0 10 | end) as high_line_count, 11 | sum(case 12 | when o_orderpriority <> '1-URGENT' 13 | and o_orderpriority <> '2-HIGH' 14 | then 1 15 | else 0 16 | end) as low_line_count 17 | from 18 | orders, 19 | lineitem 20 | where 21 | o_orderkey = l_orderkey 22 | and l_shipmode in ('MAIL', 'SHIP') 23 | and l_commitdate < l_receiptdate 24 | and l_shipdate < l_commitdate 25 | and l_receiptdate >= date '1994-01-01' 26 | and l_receiptdate < date '1994-01-01' + interval '1' year 27 | group by 28 | l_shipmode 29 | order by 30 | l_shipmode 31 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q13.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | c_count, 5 | count(*) as custdist 6 | from 7 | ( 8 | select 9 | c_custkey, 10 | count(o_orderkey) as c_count 11 | from 12 | customer left outer join orders on 13 | c_custkey = o_custkey 14 | and o_comment not like '%special%requests%' 15 | group by 16 | c_custkey 17 | ) as c_orders 18 | group by 19 | c_count 20 | order by 21 | custdist desc, 22 | c_count desc 23 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q14.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | 100.00 * sum(case 5 | when p_type like 'PROMO%' 6 | then l_extendedprice * (1 - l_discount) 7 | else 0 8 | end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue 9 | from 10 | lineitem, 11 | part 12 | where 13 | l_partkey = p_partkey 14 | and l_shipdate >= date '1995-09-01' 15 | and l_shipdate < date '1995-09-01' + interval '1' month 16 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q15.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | with revenue0 as 4 | (select 5 | l_suppkey as supplier_no, 6 | sum(l_extendedprice * (1 - l_discount)) as total_revenue 7 | from 8 | lineitem 9 | where 10 | l_shipdate >= date '1996-01-01' 11 | and l_shipdate < date '1996-01-01' + interval '3' month 12 | group by 13 | l_suppkey) 14 | select 15 | s_suppkey, 16 | s_name, 17 | s_address, 18 | s_phone, 19 | total_revenue 20 | from 21 | supplier, 22 | revenue0 23 | where 24 | s_suppkey = supplier_no 25 | and total_revenue = ( 26 | select 27 | max(total_revenue) 28 | from 29 | revenue0 30 | ) 31 | order by 32 | s_suppkey 33 | 34 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q16.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | p_brand, 5 | p_type, 6 | p_size, 7 | count(distinct ps_suppkey) as supplier_cnt 8 | from 9 | partsupp, 10 | part 11 | where 12 | p_partkey = ps_partkey 13 | and p_brand <> 'Brand#45' 14 | and p_type not like 'MEDIUM POLISHED%' 15 | and p_size in (49, 14, 23, 45, 19, 3, 36, 9) 16 | and ps_suppkey not in ( 17 | select 18 | s_suppkey 19 | from 20 | supplier 21 | where 22 | s_comment like '%Customer%Complaints%' 23 | ) 24 | group by 25 | p_brand, 26 | p_type, 27 | p_size 28 | order by 29 | supplier_cnt desc, 30 | p_brand, 31 | p_type, 32 | p_size 33 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q17.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | sum(l_extendedprice) / 7.0 as avg_yearly 5 | from 6 | lineitem, 7 | part 8 | where 9 | p_partkey = l_partkey 10 | and p_brand = 'Brand#23' 11 | and p_container = 'MED BOX' 12 | and l_quantity < ( 13 | select 14 | 0.2 * avg(l_quantity) 15 | from 16 | lineitem 17 | where 18 | l_partkey = p_partkey 19 | ) 20 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q18.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | c_name, 5 | c_custkey, 6 | o_orderkey, 7 | o_orderdate, 8 | o_totalprice, 9 | sum(l_quantity) 10 | from 11 | customer, 12 | orders, 13 | lineitem 14 | where 15 | o_orderkey in ( 16 | select 17 | l_orderkey 18 | from 19 | lineitem 20 | group by 21 | l_orderkey having 22 | sum(l_quantity) > 300 23 | ) 24 | and c_custkey = o_custkey 25 | and o_orderkey = l_orderkey 26 | group by 27 | c_name, 28 | c_custkey, 29 | o_orderkey, 30 | o_orderdate, 31 | o_totalprice 32 | order by 33 | o_totalprice desc, 34 | o_orderdate 35 | limit 100 -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q19.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | sum(l_extendedprice* (1 - l_discount)) as revenue 5 | from 6 | lineitem, 7 | part 8 | where 9 | ( 10 | p_partkey = l_partkey 11 | and p_brand = 'Brand#12' 12 | and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') 13 | and l_quantity >= 1 and l_quantity <= 1 + 10 14 | and p_size between 1 and 5 15 | and l_shipmode in ('AIR', 'AIR REG') 16 | and l_shipinstruct = 'DELIVER IN PERSON' 17 | ) 18 | or 19 | ( 20 | p_partkey = l_partkey 21 | and p_brand = 'Brand#23' 22 | and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') 23 | and l_quantity >= 10 and l_quantity <= 10 + 10 24 | and p_size between 1 and 10 25 | and l_shipmode in ('AIR', 'AIR REG') 26 | and l_shipinstruct = 'DELIVER IN PERSON' 27 | ) 28 | or 29 | ( 30 | p_partkey = l_partkey 31 | and p_brand = 'Brand#34' 32 | and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') 33 | and l_quantity >= 20 and l_quantity <= 20 + 10 34 | and p_size between 1 and 15 35 | and l_shipmode in ('AIR', 'AIR REG') 36 | and l_shipinstruct = 'DELIVER IN PERSON' 37 | ) 38 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q2.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | s_acctbal, 5 | s_name, 6 | n_name, 7 | p_partkey, 8 | p_mfgr, 9 | s_address, 10 | s_phone, 11 | s_comment 12 | from 13 | part, 14 | supplier, 15 | partsupp, 16 | nation, 17 | region 18 | where 19 | p_partkey = ps_partkey 20 | and s_suppkey = ps_suppkey 21 | and p_size = 15 22 | and p_type like '%BRASS' 23 | and s_nationkey = n_nationkey 24 | and n_regionkey = r_regionkey 25 | and r_name = 'EUROPE' 26 | and ps_supplycost = ( 27 | select 28 | min(ps_supplycost) 29 | from 30 | partsupp, 31 | supplier, 32 | nation, 33 | region 34 | where 35 | p_partkey = ps_partkey 36 | and s_suppkey = ps_suppkey 37 | and s_nationkey = n_nationkey 38 | and n_regionkey = r_regionkey 39 | and r_name = 'EUROPE' 40 | ) 41 | order by 42 | s_acctbal desc, 43 | n_name, 44 | s_name, 45 | p_partkey 46 | limit 100 47 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q20.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | s_name, 5 | s_address 6 | from 7 | supplier, 8 | nation 9 | where 10 | s_suppkey in ( 11 | select 12 | ps_suppkey 13 | from 14 | partsupp 15 | where 16 | ps_partkey in ( 17 | select 18 | p_partkey 19 | from 20 | part 21 | where 22 | p_name like 'forest%' 23 | ) 24 | and ps_availqty > ( 25 | select 26 | 0.5 * sum(l_quantity) 27 | from 28 | lineitem 29 | where 30 | l_partkey = ps_partkey 31 | and l_suppkey = ps_suppkey 32 | and l_shipdate >= date '1994-01-01' 33 | and l_shipdate < date '1994-01-01' + interval '1' year 34 | ) 35 | ) 36 | and s_nationkey = n_nationkey 37 | and n_name = 'CANADA' 38 | order by 39 | s_name 40 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q21.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | s_name, 5 | count(*) as numwait 6 | from 7 | supplier, 8 | lineitem l1, 9 | orders, 10 | nation 11 | where 12 | s_suppkey = l1.l_suppkey 13 | and o_orderkey = l1.l_orderkey 14 | and o_orderstatus = 'F' 15 | and l1.l_receiptdate > l1.l_commitdate 16 | and exists ( 17 | select 18 | * 19 | from 20 | lineitem l2 21 | where 22 | l2.l_orderkey = l1.l_orderkey 23 | and l2.l_suppkey <> l1.l_suppkey 24 | ) 25 | and not exists ( 26 | select 27 | * 28 | from 29 | lineitem l3 30 | where 31 | l3.l_orderkey = l1.l_orderkey 32 | and l3.l_suppkey <> l1.l_suppkey 33 | and l3.l_receiptdate > l3.l_commitdate 34 | ) 35 | and s_nationkey = n_nationkey 36 | and n_name = 'SAUDI ARABIA' 37 | group by 38 | s_name 39 | order by 40 | numwait desc, 41 | s_name 42 | limit 100 -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q22.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | cntrycode, 5 | count(*) as numcust, 6 | sum(c_acctbal) as totacctbal 7 | from 8 | ( 9 | select 10 | substring(c_phone, 1, 2) as cntrycode, 11 | c_acctbal 12 | from 13 | customer 14 | where 15 | substring(c_phone, 1, 2) in 16 | ('13', '31', '23', '29', '30', '18', '17') 17 | and c_acctbal > ( 18 | select 19 | avg(c_acctbal) 20 | from 21 | customer 22 | where 23 | c_acctbal > 0.00 24 | and substring(c_phone, 1, 2) in 25 | ('13', '31', '23', '29', '30', '18', '17') 26 | ) 27 | and not exists ( 28 | select 29 | * 30 | from 31 | orders 32 | where 33 | o_custkey = c_custkey 34 | ) 35 | ) as custsale 36 | group by 37 | cntrycode 38 | order by 39 | cntrycode 40 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q3.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | l_orderkey, 5 | sum(l_extendedprice * (1 - l_discount)) as revenue, 6 | o_orderdate, 7 | o_shippriority 8 | from 9 | customer, 10 | orders, 11 | lineitem 12 | where 13 | c_mktsegment = 'BUILDING' 14 | and c_custkey = o_custkey 15 | and l_orderkey = o_orderkey 16 | and o_orderdate < date '1995-03-15' 17 | and l_shipdate > date '1995-03-15' 18 | group by 19 | l_orderkey, 20 | o_orderdate, 21 | o_shippriority 22 | order by 23 | revenue desc, 24 | o_orderdate 25 | limit 10 26 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q4.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | o_orderpriority, 5 | count(*) as order_count 6 | from 7 | orders 8 | where 9 | o_orderdate >= date '1993-07-01' 10 | and o_orderdate < date '1993-07-01' + interval '3' month 11 | and exists ( 12 | select 13 | * 14 | from 15 | lineitem 16 | where 17 | l_orderkey = o_orderkey 18 | and l_commitdate < l_receiptdate 19 | ) 20 | group by 21 | o_orderpriority 22 | order by 23 | o_orderpriority 24 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q5.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | n_name, 5 | sum(l_extendedprice * (1 - l_discount)) as revenue 6 | from 7 | customer, 8 | orders, 9 | lineitem, 10 | supplier, 11 | nation, 12 | region 13 | where 14 | c_custkey = o_custkey 15 | and l_orderkey = o_orderkey 16 | and l_suppkey = s_suppkey 17 | and c_nationkey = s_nationkey 18 | and s_nationkey = n_nationkey 19 | and n_regionkey = r_regionkey 20 | and r_name = 'ASIA' 21 | and o_orderdate >= date '1994-01-01' 22 | and o_orderdate < date '1994-01-01' + interval '1' year 23 | group by 24 | n_name 25 | order by 26 | revenue desc 27 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q6.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | sum(l_extendedprice * l_discount) as revenue 5 | from 6 | lineitem 7 | where 8 | l_shipdate >= date '1994-01-01' 9 | and l_shipdate < date '1994-01-01' + interval '1' year 10 | and l_discount between .06 - 0.01 and .06 + 0.01 11 | and l_quantity < 24 12 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q7.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | supp_nation, 5 | cust_nation, 6 | l_year, 7 | sum(volume) as revenue 8 | from 9 | ( 10 | select 11 | n1.n_name as supp_nation, 12 | n2.n_name as cust_nation, 13 | year(l_shipdate) as l_year, 14 | l_extendedprice * (1 - l_discount) as volume 15 | from 16 | supplier, 17 | lineitem, 18 | orders, 19 | customer, 20 | nation n1, 21 | nation n2 22 | where 23 | s_suppkey = l_suppkey 24 | and o_orderkey = l_orderkey 25 | and c_custkey = o_custkey 26 | and s_nationkey = n1.n_nationkey 27 | and c_nationkey = n2.n_nationkey 28 | and ( 29 | (n1.n_name = 'FRANCE' and n2.n_name = 'GERMANY') 30 | or (n1.n_name = 'GERMANY' and n2.n_name = 'FRANCE') 31 | ) 32 | and l_shipdate between date '1995-01-01' and date '1996-12-31' 33 | ) as shipping 34 | group by 35 | supp_nation, 36 | cust_nation, 37 | l_year 38 | order by 39 | supp_nation, 40 | cust_nation, 41 | l_year 42 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q8.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | o_year, 5 | sum(case 6 | when nation = 'BRAZIL' then volume 7 | else 0 8 | end) / sum(volume) as mkt_share 9 | from 10 | ( 11 | select 12 | year(o_orderdate) as o_year, 13 | l_extendedprice * (1 - l_discount) as volume, 14 | n2.n_name as nation 15 | from 16 | part, 17 | supplier, 18 | lineitem, 19 | orders, 20 | customer, 21 | nation n1, 22 | nation n2, 23 | region 24 | where 25 | p_partkey = l_partkey 26 | and s_suppkey = l_suppkey 27 | and l_orderkey = o_orderkey 28 | and o_custkey = c_custkey 29 | and c_nationkey = n1.n_nationkey 30 | and n1.n_regionkey = r_regionkey 31 | and r_name = 'AMERICA' 32 | and s_nationkey = n2.n_nationkey 33 | and o_orderdate between date '1995-01-01' and date '1996-12-31' 34 | and p_type = 'ECONOMY ANODIZED STEEL' 35 | ) as all_nations 36 | group by 37 | o_year 38 | order by 39 | o_year 40 | -------------------------------------------------------------------------------- /core/src/test/resources/tpch-sql/q9.sql: -------------------------------------------------------------------------------- 1 | -- using default substitutions 2 | 3 | select 4 | nation, 5 | o_year, 6 | sum(amount) as sum_profit 7 | from 8 | ( 9 | select 10 | n_name as nation, 11 | year(o_orderdate) as o_year, 12 | l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount 13 | from 14 | part, 15 | supplier, 16 | lineitem, 17 | partsupp, 18 | orders, 19 | nation 20 | where 21 | s_suppkey = l_suppkey 22 | and ps_suppkey = l_suppkey 23 | and ps_partkey = l_partkey 24 | and p_partkey = l_partkey 25 | and o_orderkey = l_orderkey 26 | and s_nationkey = n_nationkey 27 | and p_name like '%green%' 28 | ) as profit 29 | group by 30 | nation, 31 | o_year 32 | order by 33 | nation, 34 | o_year desc 35 | -------------------------------------------------------------------------------- /core/src/test/scala/com/pingcap/tispark/UnitSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2021 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tispark 18 | 19 | import org.apache.spark.internal.Logging 20 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite, Matchers} 21 | import org.slf4j.Logger 22 | 23 | abstract class UnitSuite 24 | extends FunSuite 25 | with BeforeAndAfterEach 26 | with BeforeAndAfterAll 27 | with Matchers 28 | with Logging { 29 | protected val logger: Logger = log 30 | } 31 | -------------------------------------------------------------------------------- /core/src/test/scala/com/pingcap/tispark/datasource/BaseBatchWriteTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tispark.datasource 18 | 19 | class BaseBatchWriteTest( 20 | override val table: String, 21 | override val database: String = "tispark_test") 22 | extends BaseBatchWriteWithoutDropTableTest(table, database) { 23 | 24 | override def beforeEach(): Unit = { 25 | super.beforeEach() 26 | dropTable() 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/BaseInitialOnceTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql 18 | 19 | class BaseInitialOnceTest extends BaseTiSparkTest { 20 | private var init = false 21 | 22 | override def beforeAll(): Unit = 23 | if (!init) { 24 | super.beforeAll() 25 | init = true 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/BaseTiSparkEnableBatchWriteTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql 18 | 19 | class BaseTiSparkEnableBatchWriteTest extends BaseTiSparkTest { 20 | override def beforeEach(): Unit = { 21 | super.beforeEach() 22 | if (!supportBatchWrite) { 23 | cancel 24 | } 25 | } 26 | } 27 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/MultipleSparkSessionTestSuite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql 18 | 19 | class MultipleSparkSessionTestSuite extends BaseTiSparkTest { 20 | test("Test multiple Spark Session register udf") { 21 | val sparkSession1 = spark 22 | assert(sparkSession1.sql("select ti_version()").count() === 1) 23 | val sparkSession2 = sparkSession1.newSession() 24 | sparkSession2.sql("use tidb_catalog") 25 | assert(sparkSession2.sql("select ti_version()").count() === 1) 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/TiSparkTestSpec.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql 20 | 21 | import org.apache.spark.sql.test.SharedSQLContext 22 | 23 | import scala.util.Random 24 | 25 | trait TiSparkTestSpec extends SharedSQLContext { 26 | // Randomizer for tests 27 | lazy val r: Random = new Random(generateDataSeed) 28 | def dbName: String 29 | def testDesc: String 30 | 31 | def test(): Unit 32 | } 33 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/expression/Having0Suite.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2017 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.expression 20 | 21 | import org.apache.spark.sql.BaseInitialOnceTest 22 | 23 | class Having0Suite extends BaseInitialOnceTest { 24 | private val allCases = Seq[String]( 25 | "select tp_int%1000 a, count(*) from full_data_type_table group by (tp_int%1000) having sum(tp_int%1000) > 100 order by a", 26 | "select tp_bigint%1000 a, count(*) from full_data_type_table group by (tp_bigint%1000) having sum(tp_bigint%1000) < 100 order by a") 27 | 28 | test("Test - Having") { 29 | allCases.foreach { query => 30 | runTest(query) 31 | } 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/test/TestConstants.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2017 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.test 20 | 21 | object TestConstants { 22 | val TiDB_ADDRESS = "tidb.addr" 23 | val TiDB_PORT = "tidb.port" 24 | val TiDB_USER = "tidb.user" 25 | val TiDB_PASSWORD = "tidb.password" 26 | val TPCH_DB_NAME = "tpch.db" 27 | val TPCDS_DB_NAME = "tpcds.db" 28 | val SHOULD_LOAD_DATA = "test.data.load" 29 | val SHOULD_GENERATE_DATA = "test.data.generate" 30 | val GENERATE_DATA_SEED = "test.data.generate.seed" 31 | val ENABLE_TIFLASH_TEST = "test.tiflash.enable" 32 | val SHOULD_SKIP_TEST = "test.skip" 33 | } 34 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/test/TestSparkSession.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.test 18 | 19 | import org.apache.spark.SparkContext 20 | import org.apache.spark.sql.SparkSession 21 | 22 | /** 23 | * A special [[SparkSession]] prepared for testing. 24 | */ 25 | private[spark] class TestSparkSession(sparkContext: SparkContext) { 26 | self => 27 | private val spark = SparkSession 28 | .builder() 29 | .sparkContext(sparkContext) 30 | .getOrCreate() 31 | SparkSession.setDefaultSession(spark) 32 | SparkSession.setActiveSession(spark) 33 | 34 | def session: SparkSession = spark 35 | } 36 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/test/Utils.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.test 18 | 19 | import java.util.Properties 20 | 21 | object Utils { 22 | def getOrElse(prop: Properties, key: String, defValue: String): String = { 23 | val jvmProp = System.getProperty(key) 24 | if (jvmProp != null) { 25 | jvmProp 26 | } else { 27 | Option(prop.getProperty(key)).getOrElse(defValue) 28 | } 29 | } 30 | 31 | def getFlagOrTrue(prop: Properties, key: String): Boolean = 32 | getFlag(prop, key, "true") 33 | 34 | private def getFlag(prop: Properties, key: String, defValue: String): Boolean = 35 | getOrElse(prop, key, defValue).equalsIgnoreCase("true") 36 | } 37 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnClusterIndexDataSuite00.scala: -------------------------------------------------------------------------------- 1 | package org.apache.spark.sql.types.pk 2 | 3 | /* 4 | * 5 | * Copyright 2019 PingCAP, Inc. 6 | * 7 | * Licensed under the Apache License, Version 2.0 (the "License"); 8 | * you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | * 19 | */ 20 | class MultiColumnClusterIndexDataSuite00 extends MultiColumnPKDataTypeSuites { 21 | init() 22 | 23 | override def currentTest: Seq[(Int, Int)] = tests(getId) 24 | 25 | if (supportClusteredIndex) { 26 | generateClusterIndexScanTestCases() 27 | } 28 | 29 | } 30 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite00.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite00 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite01.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite01 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite02.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite02 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite03.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite03 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite04.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite04 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite05.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite05 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite06.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite06 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite07.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite07 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite08.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite08 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite09.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite09 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite10.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite10 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite11.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite11 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite12.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite12 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite13.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite13 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite14.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite14 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite15.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite15 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite16.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite16 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite17.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite17 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite18.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite18 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite19.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite19 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite20.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite20 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite21.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite21 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite22.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite22 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite23.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite23 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite24.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite24 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite25.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite25 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite26.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite26 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite27.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite27 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite28.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite28 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite29.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite29 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite30.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite30 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite31.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite31 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite32.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite32 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite33.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite33 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite34.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite34 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /core/src/test/scala/org/apache/spark/sql/types/pk/MultiColumnPKDataTypeSuite35.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package org.apache.spark.sql.types.pk 20 | 21 | class MultiColumnPKDataTypeSuite35 extends MultiColumnPKDataTypeSuites { 22 | override def currentTest: Seq[(Int, Int)] = tests(getId) 23 | 24 | generateTestCases() 25 | } 26 | -------------------------------------------------------------------------------- /db-random-test/src/main/scala/com/pingcap/tispark/test/generator/GeneratorUtils.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tispark.test.generator 18 | 19 | object GeneratorUtils { 20 | val defaultEnumValues: List[String] = List("XS", "S", "M", "L", "XL", "XXL") 21 | } 22 | -------------------------------------------------------------------------------- /db-random-test/src/main/scala/com/pingcap/tispark/test/generator/Index.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2019 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package com.pingcap.tispark.test.generator 20 | 21 | trait Index { 22 | def indexColumns: List[IndexColumn] 23 | val isPrimaryKey: Boolean = false 24 | val isUnique: Boolean = false 25 | } 26 | 27 | case class Key(indexColumns: List[IndexColumn]) extends Index {} 28 | 29 | case class PrimaryKey(indexColumns: List[IndexColumn]) extends Index { 30 | override val isPrimaryKey: Boolean = true 31 | } 32 | 33 | case class UniqueKey(indexColumns: List[IndexColumn]) extends Index { 34 | override val isUnique: Boolean = true 35 | } 36 | -------------------------------------------------------------------------------- /db-random-test/src/main/scala/com/pingcap/tispark/test/generator/NullableType.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2021 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tispark.test.generator 18 | 19 | object NullableType extends Enumeration { 20 | val Nullable, NotNullable, NumericNotNullable, NotNullablePK = Value 21 | } 22 | -------------------------------------------------------------------------------- /dev/.scalafmt.conf: -------------------------------------------------------------------------------- 1 | version = 2.5.3 2 | 3 | align = none 4 | align.openParenDefnSite = false 5 | align.openParenCallSite = false 6 | align.tokens = [] 7 | optIn = { 8 | configStyleArguments = false 9 | } 10 | danglingParentheses = false 11 | docstrings = JavaDoc 12 | maxColumn = 98 13 | encoding = "UTF-8" 14 | -------------------------------------------------------------------------------- /dev/javafmt: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | echo $MAVEN_HOME 4 | 5 | mvn com.coveo:fmt-maven-plugin:format -------------------------------------------------------------------------------- /dev/scalafmt: -------------------------------------------------------------------------------- 1 | #!/usr/bin/env bash 2 | 3 | echo $MAVEN_HOME 4 | 5 | mvn mvn-scalafmt_2.12:format -Dscalafmt.skip=false 6 | -------------------------------------------------------------------------------- /docker-compose-TiDB-TLS.yaml: -------------------------------------------------------------------------------- 1 | version: '2.3' 2 | 3 | services: 4 | pd: 5 | image: pingcap/pd:v6.0.0 6 | ports: 7 | - "2379:2379" 8 | volumes: 9 | - ./config:/config:ro 10 | - ./data:/data 11 | command: 12 | - --name=pd 13 | - --client-urls=https://0.0.0.0:2379 14 | - --peer-urls=https://0.0.0.0:2380 15 | - --advertise-client-urls=https://pd:2379 16 | - --advertise-peer-urls=https://pd:2380 17 | - --initial-cluster=pd=https://pd:2380 18 | - --data-dir=/data/pd 19 | - --config=/config/pd-TLS.toml 20 | restart: on-failure 21 | tikv: 22 | image: pingcap/tikv:v6.0.0 23 | ports: 24 | - "20160:20160" 25 | volumes: 26 | - ./config:/config:ro 27 | - ./data:/data 28 | command: 29 | - --addr=0.0.0.0:20160 30 | - --advertise-addr=tikv:20160 31 | - --data-dir=/data/tikv 32 | - --pd=https://pd:2379 33 | - --config=/config/tikv-TLS.toml 34 | depends_on: 35 | - "pd" 36 | restart: on-failure 37 | tidb: 38 | image: pingcap/tidb:v6.0.0 39 | ports: 40 | - "4000:4000" 41 | - "10080:10080" 42 | volumes: 43 | - ./config:/config:ro 44 | command: 45 | - --store=tikv 46 | - --path=pd:2379 47 | - --config=/config/tidb-TLS.toml 48 | - -L=warn 49 | depends_on: 50 | - "tikv" 51 | restart: on-failure 52 | -------------------------------------------------------------------------------- /docker-compose.yaml: -------------------------------------------------------------------------------- 1 | version: '2.3' 2 | 3 | services: 4 | pd0: 5 | image: pingcap/pd:latest 6 | ports: 7 | - "2379:2379" 8 | volumes: 9 | - ./config/pd.toml:/pd.toml:ro 10 | - ./data:/data 11 | command: 12 | - --name=pd0 13 | - --client-urls=http://0.0.0.0:2379 14 | - --peer-urls=http://0.0.0.0:2380 15 | - --advertise-client-urls=http://pd0:2379 16 | - --advertise-peer-urls=http://pd0:2380 17 | - --initial-cluster=pd0=http://pd0:2380 18 | - --data-dir=/data/pd0 19 | - --config=/pd.toml 20 | restart: on-failure 21 | tikv0: 22 | image: pingcap/tikv:latest 23 | ports: 24 | - "20160:20160" 25 | volumes: 26 | - ./config/tikv.toml:/tikv.toml:ro 27 | - ./data:/data 28 | command: 29 | - --addr=0.0.0.0:20160 30 | - --advertise-addr=tikv0:20160 31 | - --data-dir=/data/tikv0 32 | - --pd=pd0:2379 33 | - --config=/tikv.toml 34 | depends_on: 35 | - "pd0" 36 | restart: on-failure 37 | tidb: 38 | image: pingcap/tidb:latest 39 | ports: 40 | - "4000:4000" 41 | - "10080:10080" 42 | volumes: 43 | - ./config/tidb.toml:/tidb.toml:ro 44 | command: 45 | - --store=tikv 46 | - --path=pd0:2379 47 | - --config=/tidb.toml 48 | - -L=warn 49 | depends_on: 50 | - "tikv0" 51 | restart: on-failure 52 | -------------------------------------------------------------------------------- /docs/architecture.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/architecture.png -------------------------------------------------------------------------------- /docs/design/imgs/delete-support/delete_step: -------------------------------------------------------------------------------- 1 | @startuml 2 | 3 | start 4 | 5 | group deleteWhere 6 | : Parse filters to WHERE clause; 7 | : Query data from TiDB; 8 | endgroup 9 | group TiDBDelete 10 | : Check data and schema; 11 | group WriteUtil 12 | : Convert Spark row to TiKV row; 13 | : Extract handle; 14 | : Encode record & index; 15 | endgroup 16 | group TwoPhaseCommitUtil 17 | : Execute 2PC; 18 | endgroup 19 | endgroup 20 | 21 | 22 | 23 | stop 24 | @enduml -------------------------------------------------------------------------------- /docs/design/imgs/delete-support/delete_step.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/design/imgs/delete-support/delete_step.png -------------------------------------------------------------------------------- /docs/design/imgs/delete-support/rowid: -------------------------------------------------------------------------------- 1 | @startuml 2 | 3 | start 4 | 5 | : get config; 6 | group get Tisession 7 | : get Tisession with config; 8 | : cache Tisession wtih pdaddress; 9 | endgroup 10 | group get catalog with Tisession 11 | : new catalog if cache is null; 12 | if (spark.tispark.show_rowid) then (true) 13 | :showRowId = true; 14 | else (flase) 15 | :showRowId = false; 16 | endif 17 | : cache catalog; 18 | endgroup 19 | group get table with catalog 20 | :get table from cache; 21 | if (showRowId) then (true) 22 | :table.copyTableWithRowId(); 23 | if (pkishandle) then (false) 24 | :add columns _tidb_rowid ; 25 | :pkishandle = true; 26 | else (true) 27 | :do nothing; 28 | endif 29 | else (false) 30 | :table; 31 | endif 32 | 33 | 34 | stop 35 | @enduml -------------------------------------------------------------------------------- /docs/design/imgs/delete-support/rowid.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/design/imgs/delete-support/rowid.png -------------------------------------------------------------------------------- /docs/design/imgs/dsv2-support/dsv2.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/design/imgs/dsv2-support/dsv2.png -------------------------------------------------------------------------------- /docs/design/imgs/dsv2-support/new_write: -------------------------------------------------------------------------------- 1 | @startuml 2 | 3 | start 4 | group saveToTiDB 5 | : processing based on global data; 6 | : call write api; 7 | endgroup 8 | group BatchWrite 9 | : pre-write primary key; 10 | endgroup 11 | group DataWriter 12 | : processing based on partition data; 13 | : pre-write secondary key; 14 | endgroup 15 | group BatchWrite 16 | : commit primary key; 17 | endgroup 18 | stop 19 | 20 | @enduml -------------------------------------------------------------------------------- /docs/design/imgs/dsv2-support/new_write.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/design/imgs/dsv2-support/new_write.png -------------------------------------------------------------------------------- /docs/design/imgs/dsv2-support/write.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/design/imgs/dsv2-support/write.png -------------------------------------------------------------------------------- /docs/design/imgs/phyical_plan_explain/code_flow_chart.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/design/imgs/phyical_plan_explain/code_flow_chart.png -------------------------------------------------------------------------------- /docs/design/imgs/stale-read/meta: -------------------------------------------------------------------------------- 1 | @startuml 2 | group every SQL 3 | TiParser -> CatalogManager: get TiCatalog's meta 4 | CatalogManager -> TiCatalog: get TiCatalog from cache 5 | TiCatalog -> TiParser: return TiCatalog's meta 6 | TiParser -> TiUtil: get TSO 7 | TiUtil -> TiUtil: parse ts according\nto the configuration 8 | TiUtil -> TiParser: return TSO 9 | 10 | TiParser -> TiSession: get snapShotCatalog with TSO 11 | TiSession -> TiSession : update snapshotTimestamp\nand cache snapShotCatalog 12 | TiSession -> TiParser: return snapShotCatalog 13 | 14 | TiParser -> MetaManager: update meta's catalog 15 | MetaManager -> MetaManager: update catalog if\nit has been changed 16 | end 17 | 18 | @enduml -------------------------------------------------------------------------------- /docs/design/imgs/stale-read/meta.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/design/imgs/stale-read/meta.png -------------------------------------------------------------------------------- /docs/dev-guide/contribute/code_style.md: -------------------------------------------------------------------------------- 1 | # Code Style 2 | 3 | We have strict code style requirements. 4 | If your code doesn't meet the requirements, it will not be allowed to be merged. 5 | 6 | TiSpark uses two languages, Java and Scala. 7 | We have format files for the respective languages and added format operations in Maven. 8 | So, you have two ways to make your code conform to code style. 9 | 10 | ## Add format files in IDEA 11 | The format files are under `dev/` called `.scalafmt.conf` and `intellij-java-google-style.xml`. 12 | 13 | You can set the Configuration to `.scalafmt.conf` as follows to enable Scala format check in IDEA. 14 | ![scala format](../pics/scala_format.png) 15 | You can import settings in `intellij-java-google-style.xml` as follows to enable Java format check in IDEA. 16 | ![java format](../pics/java_format.png) 17 | Once you have completed the above settings, you can use shortcut keys to format in IDEA to make your code conform to code style. 18 | 19 | ## Use maven to format 20 | We have also added format support in the Maven plugin. 21 | Before submitting the code, you can run the following commands to format to ensure that your code conforms to the code style. 22 | ``` 23 | mvn mvn-scalafmt_2.12:format -Dscalafmt.skip=false // For Scala 24 | mvn com.coveo:fmt-maven-plugin:format // For Java 25 | ``` -------------------------------------------------------------------------------- /docs/dev-guide/pics/conflict.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/dev-guide/pics/conflict.png -------------------------------------------------------------------------------- /docs/dev-guide/pics/fork.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/dev-guide/pics/fork.png -------------------------------------------------------------------------------- /docs/dev-guide/pics/issue_choose.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/dev-guide/pics/issue_choose.png -------------------------------------------------------------------------------- /docs/dev-guide/pics/issue_template.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/dev-guide/pics/issue_template.png -------------------------------------------------------------------------------- /docs/dev-guide/pics/java_format.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/dev-guide/pics/java_format.png -------------------------------------------------------------------------------- /docs/dev-guide/pics/overview.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/dev-guide/pics/overview.png -------------------------------------------------------------------------------- /docs/dev-guide/pics/pr.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/dev-guide/pics/pr.png -------------------------------------------------------------------------------- /docs/dev-guide/pics/pr_templete.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/dev-guide/pics/pr_templete.png -------------------------------------------------------------------------------- /docs/dev-guide/pics/scala_format.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/pingcap/tispark/4395cd4910bd1cb858c57cb478d65b4f631843fa/docs/dev-guide/pics/scala_format.png -------------------------------------------------------------------------------- /docs/features/delete_userguide.md: -------------------------------------------------------------------------------- 1 | # Delete feature 2 | 3 | TiSpark delete feature provides the ability to delete bypass TiDB with spark SQL. 4 | 5 | ## Setup 6 | To use delete in TiSpark, make sure you have configured Spark catalogs in `spark-defaults.conf` 7 | ``` 8 | spark.sql.catalog.tidb_catalog org.apache.spark.sql.catalyst.catalog.TiCatalog 9 | spark.sql.catalog.tidb_catalog.pd.addresses ${your_pd_address} 10 | ``` 11 | 12 | ## Requirement 13 | - TiDB 4.x or 5.x 14 | - Spark = 3.0.x or 3.1.x or 3.2.x or 3.3.x 15 | 16 | ## Delete with SQL 17 | ``` 18 | spark.sql("delete from tidb_catalog.db.table where xxx") 19 | ``` 20 | You can also customize some options 21 | ``` 22 | spark.sql("delete from tidb_catalog.db.table where xxx") 23 | ``` 24 | 25 | ## Limitation 26 | - Delete without WHERE clause is not supported. 27 | - Delete with subQuery is not supported. 28 | - Delete from partition table is not supported. 29 | - Delete with Pessimistic Transaction Mode is not supported. 30 | 31 | -------------------------------------------------------------------------------- /spark-wrapper/spark-3.0/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TiBasicLogicalPlan.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2022 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.catalyst.plans.logical 18 | 19 | import com.pingcap.tispark.auth.TiAuthorization 20 | 21 | object TiBasicLogicalPlan { 22 | def verifyAuthorizationRule( 23 | logicalPlan: LogicalPlan, 24 | tiAuthorization: Option[TiAuthorization]): LogicalPlan = { 25 | logicalPlan match { 26 | case st @ SetCatalogAndNamespace(catalogManager, catalogName, namespace) => 27 | if (catalogName.nonEmpty && catalogName.get.equals( 28 | "tidb_catalog") && namespace.isDefined) { 29 | namespace.get 30 | .foreach(TiAuthorization.authorizeForSetDatabase(_, tiAuthorization)) 31 | } 32 | st 33 | } 34 | } 35 | 36 | } 37 | -------------------------------------------------------------------------------- /spark-wrapper/spark-3.1/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TiBasicLogicalPlan.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2022 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package org.apache.spark.sql.catalyst.plans.logical 18 | 19 | import com.pingcap.tispark.auth.TiAuthorization 20 | 21 | object TiBasicLogicalPlan { 22 | def verifyAuthorizationRule( 23 | logicalPlan: LogicalPlan, 24 | tiAuthorization: Option[TiAuthorization]): LogicalPlan = { 25 | logicalPlan match { 26 | case st @ SetCatalogAndNamespace(catalogManager, catalogName, namespace) => 27 | if (catalogName.nonEmpty && catalogName.get.equals( 28 | "tidb_catalog") && namespace.isDefined) { 29 | namespace.get 30 | .foreach(TiAuthorization.authorizeForSetDatabase(_, tiAuthorization)) 31 | } 32 | st 33 | } 34 | } 35 | 36 | } 37 | -------------------------------------------------------------------------------- /tikv-client/BUILD: -------------------------------------------------------------------------------- 1 | load( 2 | "@com_github_zhexuany_bazel_shade//:java_shade.bzl", 3 | "java_shade" 4 | ) 5 | package(default_visibility = ["//visibility:public"]) 6 | 7 | java_binary( 8 | name = "tikv-java-client", 9 | main_class = "com.pingcap.tikv.Main", 10 | runtime_deps = [ 11 | "//src/main/java/com/pingcap/tikv:tikv-java-client-lib", 12 | ":shaded_scalding", 13 | ], 14 | ) 15 | java_shade( 16 | name = "shaded_args", 17 | input_jar = "@io_netty_netty_codec_socks//jar", 18 | rules = "shading_rule" 19 | ) 20 | 21 | java_import( 22 | name = "shaded_scalding", 23 | jars = ["shaded_args.jar"] 24 | ) 25 | 26 | filegroup( 27 | name = "protos", 28 | srcs = glob([ 29 | "kvproto/proto/*.proto", 30 | "kvproto/_vendor/src/github.com/gogo/protobuf/gogoproto/*.proto", 31 | "tipb/proto/*.proto", 32 | ]), 33 | ) 34 | 35 | load("@org_pubref_rules_protobuf//java:rules.bzl", "java_proto_library") 36 | 37 | java_proto_library( 38 | name = "java", 39 | imports = [ 40 | "external/com_google_protobuf/src/", 41 | "kvproto/proto", 42 | "kvproto/_vendor/src/github.com/gogo/protobuf", 43 | "tipb/proto", 44 | ], 45 | inputs = ["@com_google_protobuf//:well_known_protos"], 46 | protos = [":protos"], 47 | verbose = 0, # 0=no output, 1=show protoc command, 2+ more... 48 | with_grpc = True, 49 | ) 50 | -------------------------------------------------------------------------------- /tikv-client/Makefile: -------------------------------------------------------------------------------- 1 | run: 2 | bazel run :tikv-java-client 3 | uber_jar: 4 | bazel build :tikv-java-client_deploy.jar 5 | test: 6 | bazel test //src/test/java/com/pingcap/tikv:tikv-client-java-test --test_output=errors --test_timeout=3600 7 | -------------------------------------------------------------------------------- /tikv-client/scripts/proto.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | # 3 | # Copyright 2017 PingCAP, Inc. 4 | # 5 | # Licensed under the Apache License, Version 2.0 (the "License"); 6 | # you may not use this file except in compliance with the License. 7 | # You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | # 16 | 17 | CURRENT_DIR=`pwd` 18 | TISPARK_HOME="$(cd "`dirname "$0"`"/../..; pwd)" 19 | cd $TISPARK_HOME/tikv-client 20 | 21 | tipb_hash=29e23c62eeace5912f696d1b184b63d5dc3edcce 22 | 23 | if [ -d "tipb" ]; then 24 | cd tipb; git fetch -p; git checkout ${tipb_hash}; cd .. 25 | else 26 | git clone https://github.com/pingcap/tipb; cd tipb; git checkout ${tipb_hash}; cd .. 27 | fi 28 | 29 | cd $CURRENT_DIR 30 | -------------------------------------------------------------------------------- /tikv-client/shading_rule: -------------------------------------------------------------------------------- 1 | rule io.netty.** io.netty.netty4pingcap.@1 2 | -------------------------------------------------------------------------------- /tikv-client/src/main/java/com/pingcap/tikv/BUILD: -------------------------------------------------------------------------------- 1 | package(default_visibility = ["//visibility:public"]) 2 | 3 | java_library( 4 | name = "tikv-java-client-lib", 5 | srcs = glob( 6 | ["**/*.java"], 7 | ), 8 | deps = [ 9 | "//:java", 10 | "@com_fasterxml_jackson_core_jackson_annotations//jar", 11 | "@com_fasterxml_jackson_core_jackson_core//jar", 12 | "@com_fasterxml_jackson_core_jackson_databind//jar", 13 | "@com_google_code_findbugs_jsr305//jar", 14 | "@com_google_code_gson_gson//jar", 15 | "@com_google_errorprone_error_prone_annotations//jar", 16 | "@com_google_guava_guava//jar", 17 | "@com_google_protobuf_protobuf_java//jar", 18 | "@joda_time//jar", 19 | # the following are defined in rules_protobuf 20 | "@org_pubref_rules_protobuf//java:grpc_compiletime_deps", 21 | "@org_pubref_rules_protobuf//java:netty_runtime_deps", 22 | 23 | "@org_slf4j_slf4j_api//jar", 24 | "@org_slf4j_jcl_over_slf4j//jar", 25 | "@org_slf4j_jul_to_slf4j//jar", 26 | "@log4j_log4j//jar", 27 | ], 28 | ) 29 | 30 | filegroup( 31 | name = "srcs", 32 | srcs = ["BUILD"] + glob(["**/*.java"]), 33 | ) 34 | -------------------------------------------------------------------------------- /tikv-client/src/main/java/com/pingcap/tikv/Main.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tikv; 18 | 19 | public class Main { 20 | public static void main(String[] args) throws Exception {} 21 | } 22 | -------------------------------------------------------------------------------- /tikv-client/src/main/java/com/pingcap/tikv/columnar/TiChunk.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tikv.columnar; 18 | 19 | /** 20 | * TiChunk is an abstraction of Chunk data transmitted from TiKV. A Chunk represents a batch row 21 | * data in columnar format. 22 | */ 23 | public class TiChunk { 24 | private final TiColumnVector[] columnVectors; 25 | private final int numOfRows; 26 | 27 | public TiChunk(TiColumnVector[] columnVectors) { 28 | this.columnVectors = columnVectors; 29 | this.numOfRows = columnVectors[0].numOfRows(); 30 | } 31 | 32 | public TiColumnVector column(int ordinal) { 33 | return columnVectors[ordinal]; 34 | } 35 | 36 | public int numOfCols() { 37 | return columnVectors.length; 38 | } 39 | 40 | public int numOfRows() { 41 | return numOfRows; 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /tikv-client/src/main/java/com/pingcap/tikv/columnar/datatypes/CHTypeDate.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tikv.columnar.datatypes; 18 | 19 | import com.pingcap.tikv.types.DataType; 20 | import com.pingcap.tikv.types.DateType; 21 | 22 | public class CHTypeDate extends CHType { 23 | public CHTypeDate() { 24 | this.length = 2; 25 | } 26 | 27 | @Override 28 | public String name() { 29 | return "Date"; 30 | } 31 | 32 | @Override 33 | public DataType toDataType() { 34 | return DateType.DATE; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /tikv-client/src/main/java/com/pingcap/tikv/columnar/datatypes/CHTypeDateTime.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tikv.columnar.datatypes; 18 | 19 | import com.pingcap.tikv.types.DataType; 20 | import com.pingcap.tikv.types.DateTimeType; 21 | 22 | public class CHTypeDateTime extends CHType { 23 | public CHTypeDateTime() { 24 | this.length = 4; 25 | } 26 | 27 | @Override 28 | public String name() { 29 | return "DateTime"; 30 | } 31 | 32 | @Override 33 | public DataType toDataType() { 34 | return DateTimeType.DATETIME; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /tikv-client/src/main/java/com/pingcap/tikv/columnar/datatypes/CHTypeFixedString.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tikv.columnar.datatypes; 18 | 19 | import com.pingcap.tikv.types.DataType; 20 | import com.pingcap.tikv.types.StringType; 21 | 22 | public class CHTypeFixedString extends CHType { 23 | private final int length; 24 | 25 | public CHTypeFixedString(int length) { 26 | assert length > 0; 27 | this.length = length; 28 | } 29 | 30 | @Override 31 | public String name() { 32 | return "FixedString(" + length + ")"; 33 | } 34 | 35 | @Override 36 | public DataType toDataType() { 37 | return StringType.TEXT; 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /tikv-client/src/main/java/com/pingcap/tikv/columnar/datatypes/CHTypeMyDate.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tikv.columnar.datatypes; 18 | 19 | import com.pingcap.tikv.types.DataType; 20 | import com.pingcap.tikv.types.DateType; 21 | 22 | public class CHTypeMyDate extends CHType { 23 | public CHTypeMyDate() { 24 | this.length = 8; 25 | } 26 | 27 | @Override 28 | public String name() { 29 | return "MyDate"; 30 | } 31 | 32 | @Override 33 | public DataType toDataType() { 34 | return DateType.DATE; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /tikv-client/src/main/java/com/pingcap/tikv/columnar/datatypes/CHTypeMyDateTime.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tikv.columnar.datatypes; 18 | 19 | import com.pingcap.tikv.types.DataType; 20 | import com.pingcap.tikv.types.DateTimeType; 21 | 22 | public class CHTypeMyDateTime extends CHType { 23 | public CHTypeMyDateTime() { 24 | this.length = 8; 25 | } 26 | 27 | @Override 28 | public String name() { 29 | return "MyDateTime"; 30 | } 31 | 32 | @Override 33 | public DataType toDataType() { 34 | return DateTimeType.DATETIME; 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /tikv-client/src/main/java/com/pingcap/tikv/expression/ExpressionBlocklist.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2017 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tikv.expression; 18 | 19 | public class ExpressionBlocklist extends Blocklist { 20 | 21 | public ExpressionBlocklist(String exprsString) { 22 | super(exprsString); 23 | } 24 | 25 | public boolean isUnsupportedPushDownExpr(Class cls) { 26 | return isUnsupported(cls); 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /tikv-client/src/main/java/com/pingcap/tikv/handle/Handle.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2020 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tikv.handle; 18 | 19 | import java.io.Serializable; 20 | 21 | public interface Handle extends Serializable { 22 | 23 | boolean isInt(); 24 | 25 | long intValue(); 26 | 27 | Handle next(); 28 | 29 | int compare(Handle h); 30 | 31 | byte[] encoded(); 32 | 33 | byte[] encodedAsKey(); 34 | 35 | int len(); 36 | 37 | int numCols(); 38 | 39 | byte[] encodedCol(int idx); 40 | 41 | Object[] data(); 42 | } 43 | -------------------------------------------------------------------------------- /tikv-client/src/main/java/com/pingcap/tikv/key/StatisticsKeyRangeBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tikv.key; 18 | 19 | import com.pingcap.tikv.predicates.IndexRange; 20 | import org.tikv.common.util.Pair; 21 | 22 | // A builder to build key range for Statistics keys 23 | public class StatisticsKeyRangeBuilder extends KeyRangeBuilder { 24 | 25 | public StatisticsKeyRangeBuilder(IndexRange ir) { 26 | super(ir); 27 | } 28 | 29 | private Pair toPairKey() { 30 | Key lbsKey = Key.toRawKey(lPointKey.append(lKey).getBytes()); 31 | Key ubsKey = Key.toRawKey(uPointKey.append(uKey).getBytes()); 32 | return new Pair<>(lbsKey, ubsKey); 33 | } 34 | 35 | public Pair compute() { 36 | computeKeyRange(); 37 | return toPairKey(); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /tikv-client/src/main/java/com/pingcap/tikv/partition/TableCommon.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2022 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tikv.partition; 18 | 19 | import com.pingcap.tikv.meta.TiTableInfo; 20 | import java.io.Serializable; 21 | import lombok.Data; 22 | 23 | @Data 24 | public class TableCommon implements Serializable { 25 | 26 | private final long logicalTableId; 27 | 28 | private final long physicalTableId; 29 | 30 | private final TiTableInfo tableInfo; 31 | } 32 | -------------------------------------------------------------------------------- /tikv-client/src/main/java/com/pingcap/tikv/row/RowReader.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2017 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package com.pingcap.tikv.row; 20 | 21 | import com.pingcap.tikv.types.DataType; 22 | 23 | public interface RowReader { 24 | Row readRow(DataType[] dataTypes); 25 | } 26 | -------------------------------------------------------------------------------- /tikv-client/src/main/java/com/pingcap/tikv/row/RowReaderFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * 3 | * Copyright 2017 PingCAP, Inc. 4 | * 5 | * Licensed under the Apache License, Version 2.0 (the "License"); 6 | * you may not use this file except in compliance with the License. 7 | * You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | * 17 | */ 18 | 19 | package com.pingcap.tikv.row; 20 | 21 | import com.pingcap.tikv.codec.CodecDataInput; 22 | 23 | public class RowReaderFactory { 24 | public static RowReader createRowReader(CodecDataInput cdi) { 25 | return new DefaultRowReader(cdi); 26 | } 27 | } 28 | -------------------------------------------------------------------------------- /tikv-client/src/test/java/com/pingcap/tikv/BUILD: -------------------------------------------------------------------------------- 1 | package(default_visibility = ["//visibility:public"]) 2 | 3 | load(":rule.bzl", "junit_suite_test") 4 | 5 | junit_suite_test( 6 | name = "tikv-client-java-test", 7 | srcs = glob( 8 | ["**/*.java"], 9 | ), 10 | deps = [ 11 | "//src/main/java/com/pingcap/tikv:tikv-java-client-lib", 12 | "//:java", 13 | "//:java_compile_imports", 14 | "@com_fasterxml_jackson_core_jackson_annotations//jar", 15 | "@com_fasterxml_jackson_core_jackson_core//jar", 16 | "@com_fasterxml_jackson_core_jackson_databind//jar", 17 | 18 | "@org_pubref_rules_protobuf//java:grpc_compiletime_deps", 19 | "@org_pubref_rules_protobuf//java:netty_runtime_deps", 20 | "@junit_junit//jar", 21 | "@joda_time//jar", 22 | ], 23 | ) 24 | -------------------------------------------------------------------------------- /tikv-client/src/test/java/com/pingcap/tikv/TiConfigurationTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright 2019 PingCAP, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | 17 | package com.pingcap.tikv; 18 | 19 | import static org.junit.Assert.assertEquals; 20 | 21 | import java.util.ArrayList; 22 | import java.util.List; 23 | import org.junit.Test; 24 | 25 | public class TiConfigurationTest { 26 | @Test 27 | public void testListToString() { 28 | List list = new ArrayList<>(); 29 | 30 | list.add("1"); 31 | assertEquals(TiConfiguration.listToString(list), "[1]"); 32 | 33 | list.add("2"); 34 | assertEquals(TiConfiguration.listToString(list), "[1,2]"); 35 | 36 | list.add("3"); 37 | assertEquals(TiConfiguration.listToString(list), "[1,2,3]"); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /tikv-client/src/test/java/com/pingcap/tikv/rule.bzl: -------------------------------------------------------------------------------- 1 | def junit_suite_test(name, srcs, deps, size="small", resources=[], classpath_resources=[], jvm_flags=[], tags=[], data=[]): 2 | tests = [] 3 | package = PACKAGE_NAME.replace("src/test/java/", "").replace("/", ".") 4 | for src in srcs: 5 | if src.endswith("Test.java"): 6 | if "/" in src: 7 | src = package + "." + src.replace("/", ".") 8 | tests += [src.replace(".java", ".class")] 9 | 10 | 11 | native.genrule( 12 | name = name + "-AllTests-gen", 13 | outs = ["AllTests.java"], 14 | cmd = """ 15 | cat <> $@ 16 | package %s; 17 | 18 | import org.junit.runner.RunWith; 19 | import org.junit.runners.Suite; 20 | 21 | @RunWith(Suite.class) 22 | @Suite.SuiteClasses({%s}) 23 | public class AllTests {} 24 | EOF 25 | """ % (package, ",".join(tests)) 26 | ) 27 | 28 | native.java_test( 29 | name = name, 30 | srcs = srcs + ["AllTests.java"], 31 | test_class = package + ".AllTests", 32 | resources = resources, 33 | classpath_resources = classpath_resources, 34 | data = data, 35 | size = size, 36 | tags = tags, 37 | jvm_flags = jvm_flags, 38 | deps = deps + [ 39 | ], 40 | ) 41 | -------------------------------------------------------------------------------- /tikv-client/src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | # Set root logger level to DEBUG and its only appender to A1. 2 | log4j.rootLogger=INFO, A1 3 | 4 | # A1 is set to be a ConsoleAppender. 5 | log4j.appender.A1=org.apache.log4j.ConsoleAppender 6 | 7 | # A1 uses PatternLayout. 8 | log4j.appender.A1.layout=org.apache.log4j.PatternLayout 9 | log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n --------------------------------------------------------------------------------