From 3bf7630ba549f5d94b903d32cf370f8b7f372a3c Mon Sep 17 00:00:00 2001 From: Hanumath Rao Maduri Date: Mon, 18 Sep 2017 18:06:26 +0300 Subject: [PATCH] [MSPARK-45] Move Spark-OJAI connector code to Spark github repo (#132) * SPARK-45 Move Spark-OJAI connector code to Spark github repo * Fixing pom versions for maprdb spark connector. * Changes made to the connector code to be compatible with 5.2.* and 6.0 clients. (cherry picked from commit 3d44f64) --- external/maprdb/build.gradle | 65 + external/maprdb/pom.xml | 145 + external/maprdb/pom.xml.5.0 | 145 + external/maprdb/pom.xml.6.0 | 145 + .../main/scala/com/mapr/db/spark/Field.scala | 231 ++ .../scala/com/mapr/db/spark/MapRDBSpark.scala | 97 + .../spark/OJAISerializationRegistrator.scala | 35 + .../db/spark/RDD/DocumentRDDFunctions.scala | 177 ++ .../com/mapr/db/spark/RDD/FieldType.scala | 18 + .../db/spark/RDD/FilterRDDFunctions.scala | 84 + .../com/mapr/db/spark/RDD/MapRDBBaseRDD.scala | 54 + .../db/spark/RDD/MapRDBTableScanRDD.scala | 125 + .../scala/com/mapr/db/spark/RDD/RDDTYPE.scala | 31 + .../db/spark/RDD/api/java/MapRDBJavaRDD.scala | 22 + .../spark/RDD/partition/MapRDBPartition.scala | 24 + .../MapRDBBulkLoadPartitioner.scala | 89 + .../db/spark/RDD/partitioner/OJAIKEY.scala | 84 + .../mapr/db/spark/SparkContextFunctions.scala | 22 + .../db/spark/api/java/MapRDBJavaContext.java | 38 + .../api/java/PairedRDDBeanJavaFunctions.java | 35 + .../api/java/PairedRDDJavaFunctions.java | 36 + .../spark/api/java/RDDBeanJavaFunctions.java | 36 + .../db/spark/api/java/RDDJavaFunctions.java | 37 + .../api/java/SparkContextJavaFunctions.java | 37 + .../com/mapr/db/spark/codec/BeanCodec.scala | 40 + .../spark/codec/JacksonBeanCodecHelper.scala | 120 + .../db/spark/condition/DBQueryCondition.scala | 35 + .../db/spark/condition/OJAICondition.scala | 124 + .../com/mapr/db/spark/condition/package.scala | 86 + .../SerializableConfiguration.scala | 31 + .../com/mapr/db/spark/dbclient/DBClient.scala | 39 + .../db/spark/dbclient/DBOlderClientImpl.scala | 95 + .../DocumentTypeImplicits.scala | 244 ++ .../documentTypeUtils/typeconversions.scala | 720 +++++ .../documentUtils/ScalaDocumentIterator.scala | 52 + .../exceptions/SchemaMappingException.scala | 12 + .../com/mapr/db/spark/impl/OJAIDocument.scala | 93 + .../db/spark/impl/ScalaOjaiDocument.scala | 662 ++++ .../scala/com/mapr/db/spark/package.scala | 69 + .../serializers/OJAICustomSerializers.scala | 80 + .../spark/serializers/OjaiJavaSerializer.java | 46 + .../com/mapr/db/spark/sql/DefaultSource.scala | 120 + .../mapr/db/spark/sql/GenerateSchema.scala | 247 ++ .../spark/sql/MapRDBDataFrameFunctions.scala | 24 + .../sql/MapRDBDataFrameReaderFunctions.scala | 58 + .../sql/MapRDBDataFrameWriterFunctions.scala | 12 + .../mapr/db/spark/sql/MapRDBRelation.scala | 249 ++ .../db/spark/sql/SparkSessionFunctions.scala | 21 + .../scala/com/mapr/db/spark/sql/package.scala | 27 + .../db/spark/sql/types/ConflictType.scala | 13 + .../db/spark/sql/utils/MapRSqlUtils.scala | 420 +++ .../db/spark/streaming/DStreamFunctions.scala | 27 + .../com/mapr/db/spark/streaming/package.scala | 11 + .../mapr/db/spark/types/DBArrayValue.scala | 86 + .../mapr/db/spark/types/DBBinaryValue.scala | 49 + .../com/mapr/db/spark/types/DBMapValue.scala | 74 + .../mapr/db/spark/types/TypeIterator.scala | 40 + .../mapr/db/spark/utils/DefaultClass.scala | 20 + .../mapr/db/spark/utils/LoggingTrait.scala | 74 + .../com/mapr/db/spark/utils/MapRDBUtils.scala | 124 + .../com/mapr/db/spark/utils/MapRSpark.scala | 181 ++ .../db/spark/writers/BulkTableWriter.scala | 26 + .../spark/writers/OJAIKeyWriterHelper.scala | 49 + .../spark/writers/OJAIValueWriterHelper.scala | 46 + .../mapr/db/spark/writers/TableWriter.scala | 89 + .../com/mapr/db/spark/writers/Writer.scala | 35 + .../mapr/db/testCases/LoadAndSaveTests.scala | 382 +++ .../mapr/db/testCases/MaprDBSparkTests.scala | 1004 ++++++ .../testCases/OjaiDocumentAccessTesting.scala | 773 +++++ .../mapr/db/testCases/PredicateTests.scala | 687 +++++ .../db/testCases/SparkSqlAcessTests.scala | 2679 +++++++++++++++++ .../testCases/SparkSqlLoadAndSaveTests.scala | 446 +++ .../db/testCases/SparkSqlPushDownTests.scala | 429 +++ .../db/testCases/SparkStreamingTests.scala | 83 + .../com/mapr/db/testCases/WordCount.scala | 64 + .../scala/com/mapr/db/tests/AllTests.scala | 1123 +++++++ .../com/mapr/db/tests/AllTestsWithKryo.scala | 1088 +++++++ .../scala/com/mapr/db/tests/InitCluster.scala | 109 + pom.xml | 1 + 79 files changed, 15350 insertions(+) create mode 100644 external/maprdb/build.gradle create mode 100644 external/maprdb/pom.xml create mode 100644 external/maprdb/pom.xml.5.0 create mode 100644 external/maprdb/pom.xml.6.0 create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/Field.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/MapRDBSpark.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/OJAISerializationRegistrator.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/RDD/DocumentRDDFunctions.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/RDD/FieldType.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/RDD/FilterRDDFunctions.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/RDD/MapRDBBaseRDD.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/RDD/MapRDBTableScanRDD.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/RDD/RDDTYPE.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/RDD/api/java/MapRDBJavaRDD.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/RDD/partition/MapRDBPartition.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/RDD/partitioner/MapRDBBulkLoadPartitioner.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/RDD/partitioner/OJAIKEY.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/SparkContextFunctions.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/api/java/MapRDBJavaContext.java create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/api/java/PairedRDDBeanJavaFunctions.java create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/api/java/PairedRDDJavaFunctions.java create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/api/java/RDDBeanJavaFunctions.java create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/api/java/RDDJavaFunctions.java create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/api/java/SparkContextJavaFunctions.java create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/codec/BeanCodec.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/codec/JacksonBeanCodecHelper.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/condition/DBQueryCondition.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/condition/OJAICondition.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/condition/package.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/configuration/SerializableConfiguration.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/dbclient/DBClient.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/dbclient/DBOlderClientImpl.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/documentTypeUtils/DocumentTypeImplicits.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/documentTypeUtils/typeconversions.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/documentUtils/ScalaDocumentIterator.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/exceptions/SchemaMappingException.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/impl/OJAIDocument.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/impl/ScalaOjaiDocument.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/package.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/serializers/OJAICustomSerializers.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/serializers/OjaiJavaSerializer.java create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/sql/DefaultSource.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/sql/GenerateSchema.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBDataFrameFunctions.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBDataFrameReaderFunctions.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBDataFrameWriterFunctions.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBRelation.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/sql/SparkSessionFunctions.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/sql/package.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/sql/types/ConflictType.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/sql/utils/MapRSqlUtils.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/streaming/DStreamFunctions.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/streaming/package.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/types/DBArrayValue.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/types/DBBinaryValue.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/types/DBMapValue.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/types/TypeIterator.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/utils/DefaultClass.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/utils/LoggingTrait.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/utils/MapRDBUtils.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/utils/MapRSpark.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/writers/BulkTableWriter.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/writers/OJAIKeyWriterHelper.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/writers/OJAIValueWriterHelper.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/writers/TableWriter.scala create mode 100644 external/maprdb/src/main/scala/com/mapr/db/spark/writers/Writer.scala create mode 100644 external/maprdb/src/test/scala/com/mapr/db/testCases/LoadAndSaveTests.scala create mode 100644 external/maprdb/src/test/scala/com/mapr/db/testCases/MaprDBSparkTests.scala create mode 100644 external/maprdb/src/test/scala/com/mapr/db/testCases/OjaiDocumentAccessTesting.scala create mode 100644 external/maprdb/src/test/scala/com/mapr/db/testCases/PredicateTests.scala create mode 100644 external/maprdb/src/test/scala/com/mapr/db/testCases/SparkSqlAcessTests.scala create mode 100644 external/maprdb/src/test/scala/com/mapr/db/testCases/SparkSqlLoadAndSaveTests.scala create mode 100644 external/maprdb/src/test/scala/com/mapr/db/testCases/SparkSqlPushDownTests.scala create mode 100644 external/maprdb/src/test/scala/com/mapr/db/testCases/SparkStreamingTests.scala create mode 100644 external/maprdb/src/test/scala/com/mapr/db/testCases/WordCount.scala create mode 100644 external/maprdb/src/test/scala/com/mapr/db/tests/AllTests.scala create mode 100644 external/maprdb/src/test/scala/com/mapr/db/tests/AllTestsWithKryo.scala create mode 100644 external/maprdb/src/test/scala/com/mapr/db/tests/InitCluster.scala diff --git a/external/maprdb/build.gradle b/external/maprdb/build.gradle new file mode 100644 index 0000000000000..df6140e1d2539 --- /dev/null +++ b/external/maprdb/build.gradle @@ -0,0 +1,65 @@ +import PomUtil + +apply from: "${topDir}/javatest.gradle" +apply plugin: "scala" +apply plugin: "maven-publish" + +defaultTasks "publishToMavenLocal" + +group = "com.mapr.db" +version = maprVersion + +dependencies { + compile project(path: ":src/java-utils", configuration: "main") + compile project(path: ":src/fs/proto-java", configuration: "main") + compile project(path: ":src/fs/client/fileclient/java/maprfs", configuration: "main") + compile project(path: ":src/fs/client/fileclient/java/maprfs-jni", configuration: "main") + compile project(path: ":src/fs/client/dbclient-parent/client", configuration: "main") + compile project(path: ":src/fs/client/dbclient-parent/map-reduce", configuration: "main") + compile(group: "org.ojai", name: "ojai", version: ojaiVersion) { + exclude group: "org.apache.hadoop" + } + compile(group: "org.ojai", name: "ojai-scala", version: ojaiVersion) { + exclude group: "org.apache.hadoop" + } + compile group: "org.apache.spark", name: "spark-core_${scalaCompatVersion}", version: sparkCoreVersion + compile group: "org.apache.spark", name: "spark-streaming_${scalaCompatVersion}", version: sparkCoreVersion + compile group: "org.apache.spark", name: "spark-sql_${scalaCompatVersion}", version: sparkCoreVersion + compile group: "org.scala-lang", name: "scala-library", version: scalaVersion + compile group: "com.fasterxml.jackson.module", name: "jackson-module-scala_${scalaCompatVersion}", version: jacksonVersion + compile group: "org.specs2", name: "specs2-junit_${scalaCompatVersion}", version: specs2Version + compile group: "org.scalatest", name: "scalatest_${scalaCompatVersion}", version: scalatestVersion + + testCompile project(path: ":src/fs/client/dbclient-parent/client", configuration: "testjar") +} + +tasks.withType(Jar) { + baseName "maprdb-spark" +} + +File originalPom = file("pom.xml") +List originalNodes = null + +task eatPom() { + doFirst { + originalNodes = PomUtil.parsePom(originalPom, maprVersion) + } +} + +publishing { + publications { + mvnArtifact { + artifactId "maprdb-spark" + pom.withXml { + originalNodes.each { asNode().append(it) } + } + } + } +} + +model { + tasks.generatePomFileForMvnArtifactPublication { + dependsOn "eatPom" + } +} + diff --git a/external/maprdb/pom.xml b/external/maprdb/pom.xml new file mode 100644 index 0000000000000..dca25026c9222 --- /dev/null +++ b/external/maprdb/pom.xml @@ -0,0 +1,145 @@ + + + + 4.0.0 + + + org.apache.spark + spark-parent_2.11 + 2.1.0-mapr-SNAPSHOT + ../../pom.xml + + + com.mapr.db + maprdb-spark + jar + + MapR-DB Spark Library + http://www.mapr.com/ + + + + com.mapr.db + maprdb + 6.0.0-mapr-SNAPSHOT + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + provided + + + org.scala-lang + scala-library + provided + + + com.fasterxml.jackson.module + jackson-module-scala_${scala.binary.version} + ${fasterxml.jackson.version} + + + org.specs2 + specs2-core_${scala.binary.version} + 2.4.17 + + + org.specs2 + specs2-junit_${scala.binary.version} + 2.4.17 + + + org.scalatest + scalatest_${scala.binary.version} + + + com.mapr.db + maprdb-mapreduce + 6.0.0-mapr-SNAPSHOT + + + org.ojai + ojai + 2.0-mapr-1707-beta + + + org.ojai + ojai-scala + 2.0-mapr-1707-beta + + + com.mapr + mapr-java-utils + 6.0.0-mapr-SNAPSHOT + tests + test + + + + + src/main/scala + src/test/scala + + + net.alchim31.maven + scala-maven-plugin + + incremental + + -explaintypes + + + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + add-source + testCompile + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/*Tests*.* + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.8 + 1.8 + + + + + diff --git a/external/maprdb/pom.xml.5.0 b/external/maprdb/pom.xml.5.0 new file mode 100644 index 0000000000000..d0b4747a6073f --- /dev/null +++ b/external/maprdb/pom.xml.5.0 @@ -0,0 +1,145 @@ + + + + 4.0.0 + + + org.apache.spark + spark-parent_2.11 + 2.1.0-mapr-SNAPSHOT + ../../pom.xml + + + com.mapr.db + maprdb-spark + jar + + MapR-DB Spark Library + http://www.mapr.com/ + + + + com.mapr.db + maprdb + 5.2.2-mapr + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + provided + + + org.scala-lang + scala-library + provided + + + com.fasterxml.jackson.module + jackson-module-scala_${scala.binary.version} + ${fasterxml.jackson.version} + + + org.specs2 + specs2-core_${scala.binary.version} + 2.4.17 + + + org.specs2 + specs2-junit_${scala.binary.version} + 2.4.17 + + + org.scalatest + scalatest_${scala.binary.version} + + + com.mapr.db + maprdb-mapreduce + 5.2.2-mapr + + + org.ojai + ojai + 1.1-mapr-1703 + + + org.ojai + ojai-scala + 1.1-mapr-1703 + + + com.mapr + mapr-java-utils + 5.2.2-mapr + tests + test + + + + + src/main/scala + src/test/scala + + + net.alchim31.maven + scala-maven-plugin + + incremental + + -explaintypes + + + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + add-source + testCompile + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/*Tests*.* + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.8 + 1.8 + + + + + diff --git a/external/maprdb/pom.xml.6.0 b/external/maprdb/pom.xml.6.0 new file mode 100644 index 0000000000000..dca25026c9222 --- /dev/null +++ b/external/maprdb/pom.xml.6.0 @@ -0,0 +1,145 @@ + + + + 4.0.0 + + + org.apache.spark + spark-parent_2.11 + 2.1.0-mapr-SNAPSHOT + ../../pom.xml + + + com.mapr.db + maprdb-spark + jar + + MapR-DB Spark Library + http://www.mapr.com/ + + + + com.mapr.db + maprdb + 6.0.0-mapr-SNAPSHOT + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + provided + + + org.scala-lang + scala-library + provided + + + com.fasterxml.jackson.module + jackson-module-scala_${scala.binary.version} + ${fasterxml.jackson.version} + + + org.specs2 + specs2-core_${scala.binary.version} + 2.4.17 + + + org.specs2 + specs2-junit_${scala.binary.version} + 2.4.17 + + + org.scalatest + scalatest_${scala.binary.version} + + + com.mapr.db + maprdb-mapreduce + 6.0.0-mapr-SNAPSHOT + + + org.ojai + ojai + 2.0-mapr-1707-beta + + + org.ojai + ojai-scala + 2.0-mapr-1707-beta + + + com.mapr + mapr-java-utils + 6.0.0-mapr-SNAPSHOT + tests + test + + + + + src/main/scala + src/test/scala + + + net.alchim31.maven + scala-maven-plugin + + incremental + + -explaintypes + + + + + scala-compile-first + process-resources + + add-source + compile + + + + scala-test-compile + process-test-resources + + add-source + testCompile + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + **/*Tests*.* + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.8 + 1.8 + + + + + diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/Field.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/Field.scala new file mode 100644 index 0000000000000..9943fd5d61089 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/Field.scala @@ -0,0 +1,231 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark + +import com.mapr.db.spark.condition._ +import com.mapr.db.spark.utils.MapRDBUtils +import org.ojai.store.QueryCondition + +import scala.language.implicitConversions + +/** + * field class provides the functionality to represent the query conditions. + * @param fieldPath name of the field in MapRDB Table. + * @example An equality condition can be represented by + * field("a.c.d") === 10 + * Similarly a greater than condition can be represented by + * field("a.c.d") >= 10 + */ +case class field(fieldPath: String) { + + /** + * Function to provide greaterThan(>) functionality for a field. + * @param rhs right hand side value of type T + * @example field("a.d.d") > 10 + */ + def >[T](rhs: T)(implicit ev: quotes[T]) : Predicate = new greaterThan[T](this, rhs) + + /** + * Function to provide greaterThan EqualsTo(>=) functionality for a field. + * @param rhs right hand side value of type T + * @example field("a.d.d") >= 10 + */ + def >=[T](rhs: T)(implicit ev: quotes[T]) : Predicate = new greaterThanEquals(this, rhs) + + /** + * Function to provide lessThan(<) functionality for a field. + * @param rhs right hand side value of type T + * @example field("a.d.d") < 10 + */ + def <[T](rhs: T)(implicit ev: quotes[T]) : Predicate = new lessThan(this, rhs) + + /** + * Function to provide lessThan EqualsTo(<=) functionality for a field. + * @param rhs right hand side value of type T + * @example field("a.d.d") <= 10 + */ + def <=[T](rhs: T)(implicit ev: quotes[T]) : Predicate = new lessThanEquals(this, rhs) + + /** + * Function to provide equalsTo(===) functionality for a field. + * @param rhs right hand side value of type T + * @example field("a.d.d") === 10 + */ + def ===[T](rhs: T)(implicit ev: quotes[T]) : Predicate = new equalsTo(this, rhs) + + /** + * Function to provide equalsTo(===) functionality for a field to a Sequence. + * @param rhs right hand side value of type Sequence + * @example field("a.d.d") === Seq("aa",10) + */ + def ===(rhs: Seq[Any]) : Predicate = new equalsToSeq(this, MapRDBUtils.convertToSeq(rhs)) + + /** + * Function to provide equalsTo(===) functionality for a field to a Map + * @param rhs right hand side value of type Map[String, Any] + * @example field("a.d.d") === Map("aa" -> value) + */ + def ===(rhs: Map[String,Any]) : Predicate = new equalsToMap(this, MapRDBUtils.convertToMap(rhs)) + + /** + * Function to provide notEqualsTo(!=) functionality for a field. + * @param rhs right hand side value of type T + * @example field("a.d.d") != 10 + */ + def !=[T](rhs : T)(implicit ev: quotes[T]) : Predicate = new notEqualsTo(this, rhs) + + /** + * Function to provide notequalsTo(!=) functionality for a field to a Sequence. + * @param rhs right hand side value of type Sequence + * @example field("a.d.d") != Seq("aa",10) + */ + def !=(rhs : Seq[Any]) : Predicate = new notEqualsToSeq(this, MapRDBUtils.convertToSeq(rhs)) + + /** + * Function to provide notequalsTo(!=) functionality for a field to a Map + * @param rhs right hand side value of type Map[String, Any] + * @example field("a.d.d") != Map("aa" -> value) + */ + def !=(rhs: Map[String, Any]) : Predicate = new notEqualsToMap(this, MapRDBUtils.convertToMap(rhs)) + /** + * Function to provide between functionality for a field. + * @param rhs1 first right hand side value of type T + * @param rhs2 second right hand side value of type T + * @example field("a.d.d") between (10,20) + */ + def between[T](rhs1: T, rhs2: T)(implicit ev: quotes[T]) : Predicate = new between(this, rhs1, rhs2) + + /** + * Function to provide EXISTS functionality for a field. + * @example field("a.d.d") exists + */ + def exists() : exists = new exists(this) + + /** + * Function to provide NOTIN functionality for a field. + * @param rhs right hand side value of type Seq[Any] + * @example field("a.d.d") notin Seq(10,20) + */ + def notin(rhs: Seq[Any]): NOTIN = new NOTIN(this, MapRDBUtils.convertToSeq(rhs)) + + /** + * Function to provide IN functionality for a field. + * @param rhs right hand side value of type Seq[Any] + * @example field("a.d.d") in (10, 20) + */ + def in(rhs: Seq[Any]): IN = new IN(this, MapRDBUtils.convertToSeq(rhs)) + + /** + * Function to provide NOTEXISTS functionality for a field. + * @example field("a.d.d") notexists + */ + def notexists() = new notexists(this) + + /** + * Function to provide TYPEOF functionality for a field. + * @param typevalue type of the field. + * @example field("a.d.d") typeof "INT" + */ + def typeof(typevalue: String) = new TYPEOF(this, typevalue) + + /** + * Function to provide NOTTYPEOF functionality for a field. + * @param typevalue type of the field + * @example field("a.d.d") NOTTYPEOF "INT" + */ + def nottypeof(typevalue: String) = new NOTTYPEOF(this, typevalue) + + /** + * Function to provide LIKE functionality for a field. + * @param regex right hand side is a SQL like regex string + * @example field("a.d.d") like "%s" + */ + def like(regex: String) = new LIKE(this, regex) + + /** + * Function to provide NOTLIKE functionality for a field. + * @param regex right hand side is a SQL like regex string + * @example field("a.d.d") notlike "%s" + */ + def notlike(regex: String) = new NOTLIKE(this, regex) + + /** + * Function to provide MATCHES functionality for a field. + * @param regex right hand side is a regular expression + * @example field("a.d.d") matches "*s" + */ + def matches(regex: String) = new MATCHES(this, regex) + + /** + * Function to provide NOTMATCHES functionality for a field. + * @param regex right hand side is a regular expression + * @example field("a.d.d") notmatches "*s" + */ + def notmatches(regex: String) = new NOTMATCHES(this, regex) + + override def toString = s"\42" + fieldPath + "\42" +} + +object field { + val typemap = Map("INT" -> org.ojai.Value.Type.INT, + "INTEGER" -> org.ojai.Value.Type.INT, + "LONG" -> org.ojai.Value.Type.LONG, + "BOOLEAN" -> org.ojai.Value.Type.BOOLEAN, + "STRING" -> org.ojai.Value.Type.STRING, + "SHORT" -> org.ojai.Value.Type.SHORT, + "BYTE" -> org.ojai.Value.Type.BYTE, + "NULL" -> org.ojai.Value.Type.NULL, + "FLOAT" -> org.ojai.Value.Type.FLOAT, + "DOUBLE" -> org.ojai.Value.Type.DOUBLE, + "DECIMAL" -> org.ojai.Value.Type.DECIMAL, + "DATE" -> org.ojai.Value.Type.DATE, + "TIME" -> org.ojai.Value.Type.TIME, + "TIMESTAMP" -> org.ojai.Value.Type.TIMESTAMP, + "INTERVAL" -> org.ojai.Value.Type.INTERVAL, + "BINARY" -> org.ojai.Value.Type.BINARY, + "MAP" -> org.ojai.Value.Type.MAP, + "ARRAY" -> org.ojai.Value.Type.ARRAY) +} + +case class sizeOf(field: field) { + /** + * Function to provide sizeOf lessThan functionality for a field. + * @param size right hand side is size in long + * @example sizeOf(field("a.d.d")) < 10 + */ + def <(size: Long) = new SIZEOF(field, QueryCondition.Op.LESS, size) + + /** + * Function to provide sizeOf greaterThan functionality for a field. + * @param size right hand side is size in long + * @example sizeOf(field("a.d.d")) > 10 + */ + def >(size: Long) = new SIZEOF(field, QueryCondition.Op.GREATER, size) + + /** + * Function to provide sizeOf greaterThan equals to functionality for a field + * @param size right hand side is size in long + * @example sizeOf(field("a.d.d")) >= 10 + */ + def >=(size: Long) = new SIZEOF(field, QueryCondition.Op.GREATER_OR_EQUAL, size) + + /** + * Function to provide sizeOf lessThan equals to functionality for a field + * @param size right hand side is size in long + * @example sizeOf(field("a.d.d")) <= 10 + */ + def <=(size: Long) = new SIZEOF(field, QueryCondition.Op.LESS_OR_EQUAL, size) + + /** + * Function to provide sizeOf equals to functionality for a field. + * @param size right hand side is a size in long + * @example sizeOf(field("a.d.d")) === 10 + */ + def ===(size: Long) = new SIZEOF(field, QueryCondition.Op.EQUAL, size) + + /** + * Function to provide sizeOf not equals to functionality for a field. + * @param size right hand side is a size in long + * @example sizeOf(field("a.d.d")) != 10 + */ + def !=(size: Long) = new SIZEOF(field, QueryCondition.Op.NOT_EQUAL, size) +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/MapRDBSpark.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/MapRDBSpark.scala new file mode 100644 index 0000000000000..3fdbc83869785 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/MapRDBSpark.scala @@ -0,0 +1,97 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark + +import java.nio.ByteBuffer + +import com.mapr.db.spark.RDD.partitioner.MapRDBPartitioner +import com.mapr.db.spark.RDD.partitioner.OJAIKEY +import com.mapr.db.spark.dbclient.DBClient +import com.mapr.db.spark.impl.OJAIDocument +import com.mapr.db.spark.sql.utils.MapRSqlUtils +import com.mapr.db.spark.types.DBBinaryValue +import org.apache.spark.sql.Row +import org.apache.spark.sql.types.StructType +import org.ojai.Document + +/** + * MapRDBSpark is a static class which contains factory methods to create scala's ojai document and partitioner objects. + * @example Factory functions to help create scala's ojai documents + * val doc = MapRDBSpark.newDocument(jsonString) + * val doc = MapRDBSpark.newDocument(document: org.ojai.Document) + * Here are the ways to access elements in OJAIDocument + * val partitioner = MapRDBSpark.newPartitioner(tableName) + * It creates a partitioiner using the splits specified in tableName. + * + * val partitioner = MapRDBSpark.newPartitioner(Seq("AA","CC")) + * It creates a partitioner using the splits provided in the sequence. + * Here three splits will be created (null, "AA") , ("AA","CC") and ("CC", null) + * Note that this call assumes that user supplies the splits in sorted order. + */ +object MapRDBSpark { + + /** + * Factory function to convert a ByteBuffer into a serializable binary value. + * @param buff ByteBuffer + */ + def serializableBinaryValue(buff: ByteBuffer) : DBBinaryValue = { + new DBBinaryValue(buff) + } + + /** + * Factory function to create a new OJAIDocument from org.ojai.Document. + * @param doc org.ojai.Document + */ + def newDocument(doc: Document): OJAIDocument = { + new OJAIDocument(doc) + } + + /** + * Factory function to create a new OJAIDocument from org.ojai.Document. + */ + def newDocument(): OJAIDocument = { + new OJAIDocument(DBClient().newDocument()) + } + + /** + * Factory function to create a new OJAIDocument from a json string. + * @param jsonString a json document. + */ + def newDocument(jsonString: String): OJAIDocument = { + new OJAIDocument(DBClient().newDocument(jsonString)) + } + + /** + * Factory function to create a new partitioner using existing MapRDBTable. + * @param table existing tableName in MapRDB + */ + def newPartitioner[T: OJAIKEY](table: String) = { + MapRDBPartitioner(table) + } + + /** + * Factory function to create a new partitioner from splits provided as sequence. + * @param splits a sequence of splits + * Splits supported at this point are String and ByteBuffer. + * It is user's responsibility to supply the splits in ascending order. + */ + def newPartitioner[T: OJAIKEY](splits: Seq[T]) = { + MapRDBPartitioner[T](splits) + } + + /** + * A function to convert a Spark's ROW to OJAIDocument. + * @param row Spark's Dataframe or Dataset Row. + */ + def rowToDoc(row: Row) : OJAIDocument = { + MapRSqlUtils.rowToDocument(row) + } + + /** + * A function to convert an OJAI Document to Spark's Row format. + * @param ojaiDoc OJAI Document to be converted to Spark's Row format. + * @param schema Schema for the Row. + */ + def docToRow(ojaiDoc: OJAIDocument, schema : StructType) = { + MapRSqlUtils.documentToRow(ojaiDoc, schema) + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/OJAISerializationRegistrator.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/OJAISerializationRegistrator.scala new file mode 100644 index 0000000000000..8f87c2dfd142a --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/OJAISerializationRegistrator.scala @@ -0,0 +1,35 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark + +import com.esotericsoftware.kryo.Kryo +import com.esotericsoftware.kryo.serializers._ +import com.mapr.db.spark.configuration.SerializableConfiguration +import com.mapr.db.spark.types.{DBArrayValue, DBBinaryValue, DBMapValue} +import org.apache.spark.serializer.KryoRegistrator +import org.ojai.types.{ODate, OInterval, OTime, OTimestamp} + +import scala.language.implicitConversions +import com.mapr.db.spark.serializers._ +import com.mapr.db.spark.impl.OJAIDocument + +/** + * Custom registrator provided for registering classes specific to spark ojai connector + * This registrator should be used when kryo serialization is enabled for the spark application. + * + * @example sparkconf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + * .set("spark.kryo.registrator", "com.mapr.db.spark.OJAIKryoRegistrator") + */ +class OJAIKryoRegistrator extends KryoRegistrator { + + override def registerClasses(kryo: Kryo) { + kryo.register(classOf[ODate], new ODateSerializer()) + kryo.register(classOf[OTime], new OTimeSerializer()) + kryo.register(classOf[OTimestamp], new OTimeStampSerializer()) + kryo.register(classOf[OInterval], new OIntervalSerializer()) + kryo.register(classOf[DBMapValue], new ExternalizableSerializer()) + kryo.register(classOf[DBBinaryValue], new DBBinaryValueSerializer) + kryo.register(classOf[SerializableConfiguration], new OjaiJavaSerializer()) + kryo.register(classOf[DBArrayValue[_]], new ExternalizableSerializer()) + kryo.register(classOf[OJAIDocument], new ExternalizableSerializer()) + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/DocumentRDDFunctions.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/DocumentRDDFunctions.scala new file mode 100644 index 0000000000000..ec46e965346e4 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/DocumentRDDFunctions.scala @@ -0,0 +1,177 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.RDD + +import com.mapr.db.exceptions.TableNotFoundException +import com.mapr.db.spark.RDD.partitioner.MapRDBPartitioner +import com.mapr.db.spark.condition.{DBQueryCondition, Predicate} +import com.mapr.db.spark.configuration.SerializableConfiguration +import com.mapr.db.spark.dbclient.DBClient +import org.apache.hadoop.conf.Configuration +import org.apache.spark.rdd.RDD +import org.apache.spark.broadcast.Broadcast +import com.mapr.db.spark.writers._ +import com.mapr.db.spark.utils.{LoggingTrait, MapRDBUtils} +import org.apache.spark.Partitioner +import org.ojai.store.DocumentMutation +import org.ojai.{Document, DocumentConstants, Value} + +private[spark] class DocumentRDDFunctions extends LoggingTrait{ + protected def saveToMapRDBInternal[T](rdd: RDD[T], tablename: String, createTable: Boolean = false, + bulkInsert: Boolean = false, + function1: (Broadcast[SerializableConfiguration], Boolean) => Function1[Iterator[T], Unit]) : Unit = { + var isNewAndBulkLoad = (false,false) + + val partitioner: Option[Partitioner] = rdd.partitioner + val keys: Seq[Value] = if (partitioner.isDefined && partitioner.get.isInstanceOf[MapRDBPartitioner]) { + logDebug("RDD's partitioner is being used to create the table" + partitioner) + partitioner.get.asInstanceOf[MapRDBPartitioner].splits + } else { + logDebug("it has no partitioner") + Seq.empty + } + + try { + isNewAndBulkLoad = MapRDBUtils.checkOrCreateTable(tablename, bulkInsert, createTable, keys) + } catch{ + case e: TableNotFoundException => logError("Table: "+ tablename +" not found and createTable set to: " + createTable) + throw e + case any: Exception => throw any + } + + val hadoopConf = new Configuration() + val serializableConf = new SerializableConfiguration(hadoopConf) + val cnf: Broadcast[SerializableConfiguration] = rdd.context.broadcast(serializableConf) + rdd.foreachPartition(function1(cnf, isNewAndBulkLoad._2)) + if (isNewAndBulkLoad._1 && isNewAndBulkLoad._2) + MapRDBUtils.setBulkLoad(tablename, false) + } +} + +private[spark] case class OJAIDocumentRDDFunctions[T](rdd: RDD[T])(implicit f: OJAIValue[T]) extends DocumentRDDFunctions { + + @transient val sparkContext = rdd.sparkContext + + def saveToMapRDB(tablename: String, createTable: Boolean = false, bulkInsert: Boolean = false, idFieldPath: String = DocumentConstants.ID_KEY) : Unit = { + logDebug("saveToMapRDB in OJAIDocumentRDDFunctions is called for table: "+tablename+" with bulkinsert flag set: "+bulkInsert+" and createTable:"+ createTable) + var getID : (Document) => Value = null + if (idFieldPath == DocumentConstants.ID_KEY) + getID = (doc: Document) => doc.getId + else + getID = (doc: Document) => doc.getValue(idFieldPath) + this.saveToMapRDBInternal(rdd,tablename,createTable,bulkInsert, + (cnf: Broadcast[SerializableConfiguration], isnewAndBulkLoad : Boolean) => (iter : Iterator[T]) => { if (iter.nonEmpty) { + val writer = Writer.initialize(tablename, cnf.value, isnewAndBulkLoad, true) + while(iter.hasNext) { + val element = iter.next + f.write(f.getValue(element), getID, writer) + } + writer.close() + }}) + } + + def insertToMapRDB(tablename: String, createTable: Boolean = false, bulkInsert: Boolean = false, idFieldPath: String = DocumentConstants.ID_KEY) : Unit = { + logDebug("insertToMapRDB in OJAIDocumentRDDFunctions is called for table: "+tablename+" with bulkinsert flag set: "+bulkInsert+" and createTable:"+ createTable) + var getID : (Document) => Value = null + if (idFieldPath == DocumentConstants.ID_KEY) + getID = (doc: Document) => doc.getId + else + getID = (doc: Document) => doc.getValue(idFieldPath) + this.saveToMapRDBInternal(rdd,tablename,createTable,bulkInsert, + (cnf: Broadcast[SerializableConfiguration], isnewAndBulkLoad : Boolean) => (iter : Iterator[T]) => { if (iter.nonEmpty) { + val writer = Writer.initialize(tablename, cnf.value, isnewAndBulkLoad, false) + while(iter.hasNext) { + val element = iter.next + f.write(f.getValue(element), getID, writer) + } + writer.close() + }}) + } + + def updateToMapRDB(tablename: String, mutation: (T) => DocumentMutation, getID: (T) => Value) : Unit = { + logDebug("updateToMapRDB in OJAIDocumentRDDFunctions is called for table: "+tablename) + this.saveToMapRDBInternal(rdd, tablename, false, false, + (cnf: Broadcast[SerializableConfiguration], isnewAndBulkLoad : Boolean) => (iter : Iterator[T]) => if (iter.nonEmpty) { + val writer = TableUpdateWriter(DBClient().getTable(tablename)) + while(iter.hasNext) { + val element = iter.next + f.update(mutation(element), getID(element), writer) + } + writer.close() + }) + } + + def updateToMapRDB(tablename: String, mutation: (T) => DocumentMutation, getID: (T) => Value, condition: Predicate) : Unit = { + logDebug("updateToMapRDB in OJAIDocumentRDDFunctions is called for table: "+tablename) + val queryCondition = DBQueryCondition(condition.build.build()) + + this.saveToMapRDBInternal(rdd, tablename, false, false, + (cnf: Broadcast[SerializableConfiguration], isnewAndBulkLoad : Boolean) => (iter : Iterator[T]) => if (iter.nonEmpty) { + val writer = TableCheckAndMutateWriter(DBClient().getTable(tablename)) + while(iter.hasNext) { + val element = iter.next + f.checkAndUpdate(mutation(element), queryCondition, getID(element), writer) + } + writer.close() + }) + } +} + +private[spark] case class PairedDocumentRDDFunctions[K,V](rdd: RDD[(K,V)])(implicit f: OJAIKey[K], v: OJAIValue[V]) extends DocumentRDDFunctions{ + + @transient val sparkContext = rdd.sparkContext + def saveToMapRDB(tablename: String, createTable: Boolean = false, bulkInsert: Boolean = false) : Unit = { + logDebug("saveToMapRDB in PairedDocumentRDDFunctions is called for table: " + tablename + " with bulkinsert flag set: " + bulkInsert + " and createTable:" + createTable) + this.saveToMapRDBInternal[(K, V)](rdd, tablename, createTable, bulkInsert, + (cnf: Broadcast[SerializableConfiguration], isnewAndBulkLoad: Boolean) => (iter: Iterator[(K, V)]) => if (iter.nonEmpty) { + val writer = Writer.initialize(tablename, cnf.value, isnewAndBulkLoad, true) + while (iter.hasNext) { + val element = iter.next + f.write(v.getValue(element._2), f.getValue(element._1), writer) + } + writer.close() + }) + } + + def insertToMapRDB(tablename: String, createTable: Boolean = false, bulkInsert: Boolean = false) : Unit = { + logDebug("insertToMapRDB in PairedDocumentRDDFunctions is called for table: "+tablename+" with bulkinsert flag set: "+bulkInsert+" and createTable:"+ createTable) + this.saveToMapRDBInternal[(K,V)](rdd,tablename,createTable,bulkInsert, + (cnf: Broadcast[SerializableConfiguration], isnewAndBulkLoad : Boolean) => (iter : Iterator[(K,V)]) => if (iter.nonEmpty) { + val writer = Writer.initialize(tablename, cnf.value, isnewAndBulkLoad, false) + while(iter.hasNext) { + val element = iter.next + f.write(v.getValue(element._2), f.getValue(element._1), writer) + } + writer.close() + }) + } + + def updateToMapRDB(tablename: String, mutation: (V) => DocumentMutation) : Unit = { + logDebug("updateToMapRDB in PairedDocumentRDDFunctions is called for table: "+tablename) + + this.saveToMapRDBInternal[(K,V)](rdd, tablename, false, false, + (cnf: Broadcast[SerializableConfiguration], isnewAndBulkLoad : Boolean) => (iter : Iterator[(K,V)]) => if (iter.nonEmpty) { + val writer = TableUpdateWriter(DBClient().getTable(tablename)) + while(iter.hasNext) { + val element = iter.next + f.update(mutation(element._2), f.getValue(element._1), writer) + } + writer.close() + }) + } + + def updateToMapRDB(tablename: String, mutation: (V) => DocumentMutation, condition: Predicate) : Unit = { + logDebug("updateToMapRDB in PairedDocumentRDDFunctions is called for table: "+tablename) + + val queryCondition = DBQueryCondition(condition.build.build()) + + this.saveToMapRDBInternal[(K,V)](rdd, tablename, false, false, + (cnf: Broadcast[SerializableConfiguration], isnewAndBulkLoad : Boolean) => (iter : Iterator[(K,V)]) => if (iter.nonEmpty) { + val writer = TableCheckAndMutateWriter(DBClient().getTable(tablename)) + while(iter.hasNext) { + val element = iter.next + f.checkAndMutate(mutation(element._2), queryCondition, f.getValue(element._1), writer) + } + writer.close() + }) + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/FieldType.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/FieldType.scala new file mode 100644 index 0000000000000..c8ed00dad7876 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/FieldType.scala @@ -0,0 +1,18 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.RDD + +import org.ojai.FieldPath + +trait FIELD[T] extends Serializable { + def getFields(fields: Seq[Any]): Seq[String] +} + +object FIELD { + implicit val fieldStrings = new FIELD[String] { + def getFields(fields: Seq[Any]) = fields.asInstanceOf[Seq[String]] + } + + implicit val fieldPaths = new FIELD[FieldPath] { + def getFields(fields: Seq[Any]) = fields.map(field => field.asInstanceOf[FieldPath].asPathString()) + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/FilterRDDFunctions.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/FilterRDDFunctions.scala new file mode 100644 index 0000000000000..221226dfcb383 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/FilterRDDFunctions.scala @@ -0,0 +1,84 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.RDD + +import com.mapr.db.spark.field +import com.mapr.db.spark.condition._ +import com.mapr.db.spark.impl.OJAIDocument +import com.mapr.db.spark.utils.DefaultClass.DefaultType +import com.mapr.db.spark.writers.OJAIKey +import org.apache.spark.rdd.RDD +import scala.language.implicitConversions +import com.mapr.db.spark.dbclient.DBClient +import scala.reflect._ + +case class FilterRDDFunctions[K : OJAIKey : quotes](rdd : RDD[K]) { + + def joinWithMapRDB[D : ClassTag](tableName: String)(implicit e: D DefaultType OJAIDocument, reqType: RDDTYPE[D]): RDD[D] = { + rdd.mapPartitions( partition => { + val table = DBClient().getTable(tableName) + + partition.flatMap(item => { + val condition = field("_id") === item + reqType.getValue(table.find(condition.build).iterator(), classTag[D].runtimeClass.asInstanceOf[Class[D]]) + }) + }) + } + + def bulkJoinWithMapRDB[D : ClassTag](tableName: String)(implicit e: D DefaultType OJAIDocument, reqType: RDDTYPE[D]): RDD[D] = { + rdd.mapPartitions( partition => { + val table = DBClient().getTable(tableName) + var gets = Seq[K]() + var res = List[D]() + + while (partition.hasNext) { + gets = gets :+ partition.next + if (gets.size == 4) { + val condition = field("_id") in gets + res = res ++ reqType.getValue(table.find(condition.build).iterator(), classTag[D].runtimeClass.asInstanceOf[Class[D]]) + gets = Seq[K]() + } + } + + if (gets.size > 0) { + val condition = field("_id") in gets + res = res ++ reqType.getValue(table.find(condition.build).iterator(), classTag[D].runtimeClass.asInstanceOf[Class[D]]) + gets = Seq[K]() + } + res.iterator + }) + +// def bulkJoinWithMapRDB[D : ClassTag](tableName: String)(implicit e: D DefaultType OJAIDocument, reqType: RDDTYPE[D]): RDD[D] = { +// rdd.mapPartitions( partition => { +// val table = MapRDBImpl.getTable(tableName) +// val preparedPartitions : Seq[Seq[K]] = partition.foldLeft[Seq[Seq[K]]](Seq[Seq[K]]()) { +// case (Nil, item) => Seq(Seq(item)) +// case (result, s) => { +// if (result.last.size < 4) result.dropRight(1) :+ (result.last :+ s) +// else result :+ Seq(s) +// } +// } +// +// val output : Iterator[D]= preparedPartitions.map(items => { +// val condition = field("_id") in items +// reqType.getValue(table.find(condition.build).iterator(), classTag[D].runtimeClass.asInstanceOf[Class[D]])}) +// output }) +// output }) +// return preparedPartitions.asInstanceOf[RDD[D]] + + +// .flatMap(items : Seq[K] => { +// val condition = field("_id") in items +// reqType.getValue(table.find(condition.build).iterator(), classTag[D].runtimeClass.asInstanceOf[Class[D]]) +// } +// }) +// partition.foldLeft[List[K](List[Iterator[_]]()){ +// case (Nil, s) => List(List(s)) +// case (result, s) => if (result.last.size < 4) result.dropRight(1) :+ (result.last :+ s) +// else result :+ List(s) +// }.flatMap(items => { +// val condition = field("_id") in items +// reqType.getValue(table.find(condition.build).iterator(), classTag[D].runtimeClass.asInstanceOf[Class[D]]) +// }) +// }) + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/MapRDBBaseRDD.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/MapRDBBaseRDD.scala new file mode 100644 index 0000000000000..19a28863371a0 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/MapRDBBaseRDD.scala @@ -0,0 +1,54 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.RDD + +import com.mapr.db.exceptions.DBException +import com.mapr.db.spark.condition.{DBQueryCondition, Predicate} +import com.mapr.db.spark.dbclient.DBClient +import org.apache.spark.rdd.RDD + +import scala.language.existentials +import scala.reflect.ClassTag +import org.apache.spark.SparkContext +import org.ojai.store.QueryCondition + +private[spark] abstract class MapRDBBaseRDD[T : ClassTag] (@transient val sc: SparkContext, + tableName: String, condition: DBQueryCondition, + beanClass: Class[T], fields:Seq[String] = "*" :: Nil) + extends RDD[T](sc, Seq.empty) { + + type Self <: MapRDBBaseRDD[T] + + protected def copy(tableName : String = tableName, fields : Seq[String] = fields, + cond: DBQueryCondition = condition, beanClass: Class[T] = beanClass): Self + + def where(pred: Predicate) : Self = { + if (condition != null && !condition.condition.isEmpty) { + copy(cond = DBQueryCondition(DBClient().newCondition().and().condition(condition.condition).condition(pred.build.build()).close.build())) + } else { + copy(cond = DBQueryCondition(pred.build.build())) + } + } + + def where(condition: QueryCondition): Self = { + if (this.condition != null && !this.condition.condition.isEmpty) { + copy(cond = DBQueryCondition(DBClient().newCondition().and().condition(this.condition.condition).condition(condition).build())) + } else { + copy(cond = DBQueryCondition(condition)) + } + } + + def limit(value: Integer) : Self = { + throw new NotImplementedError() + } + + def select[T : FIELD](projectedFields: T*)(implicit ev: FIELD[T]) : Self = { + if (fields == null) { + copy(fields = ev.getFields(projectedFields)) + } else { + val fieldProjections = ev.getFields(projectedFields) + val outputFields = fieldProjections.filter(fld => !this.fields.contains(fld)) + if (outputFields.size > 0) throw new DBException("Fields:" + fieldProjections + " doesn't exist in the RDD ") + else copy(fields = fieldProjections.filter(fld => this.fields.contains(fld))) + } + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/MapRDBTableScanRDD.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/MapRDBTableScanRDD.scala new file mode 100644 index 0000000000000..f82af147f3417 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/MapRDBTableScanRDD.scala @@ -0,0 +1,125 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.RDD + +import com.mapr.db.spark.condition._ +import com.mapr.db.spark.RDD.partition.MaprDBPartition +import com.mapr.db.impl.{ConditionImpl, IdCodec, TabletInfoImpl} +import com.mapr.db.spark.RDD.partitioner.MapRDBPartitioner +import org.apache.spark.{Partition, Partitioner, SparkContext, TaskContext} +import org.apache.spark.broadcast.Broadcast + +import scala.language.existentials +import scala.reflect.ClassTag +import org.ojai.Document +import com.mapr.db.spark.configuration.SerializableConfiguration +import com.mapr.db.spark.dbclient.DBClient +import com.mapr.db.spark.impl.OJAIDocument +import com.mapr.db.spark.utils.DefaultClass.DefaultType +import com.mapr.db.spark.utils.MapRSpark +import org.apache.spark.sql.SparkSession +import org.ojai.Value + +import scala.reflect.runtime.universe._ + +private[spark] class MapRDBTableScanRDD[T: ClassTag](@transient sparkSession: SparkSession, @transient sc: SparkContext, + cnf: Broadcast[SerializableConfiguration], columns : Seq[String], + val tableName: String, val condition: DBQueryCondition, val beanClass: Class[T]) + (implicit e : T DefaultType OJAIDocument, reqType : RDDTYPE[T]) + extends MapRDBBaseRDD[T](sc,tableName,condition,beanClass,columns) { + + @transient private lazy val table = DBClient().getTable(tableName) + @transient private lazy val tabletinfos = if (condition == null || condition.condition.isEmpty) DBClient().getTabletInfos(tableName) + else DBClient().getTabletInfos(tableName,condition.condition) + @transient private lazy val getSplits : Seq[Value] = { + val keys = tabletinfos.map(tableinfo => IdCodec.decode(tableinfo.getCondition.asInstanceOf[ConditionImpl].getRowkeyRanges.get(0).getStopRow)) + keys.dropRight(1) + } + + private def getPartitioner : Partitioner = { + if (getSplits.isEmpty) + return null + if (getSplits(0).getType == Value.Type.STRING) { + return MapRDBPartitioner(getSplits.map(_.getString)) + } else { + return MapRDBPartitioner(getSplits.map(_.getBinary)) + } + } + + def toDF[T <: Product : TypeTag]() = maprspark[T]() + + def maprspark[T <: Product : TypeTag]() = { + MapRSpark.builder.sparkSession(sparkSession).configuration() + .sparkContext(sparkSession.sparkContext).setDBCond(condition) + .setTable(tableName).setColumnProjection(Option(columns)).build.toDF[T]() + } + + override val partitioner : Option[Partitioner] = Option(getPartitioner) + + override type Self = MapRDBTableScanRDD[T] + + override def getPartitions: Array[Partition] = { + val splits = tabletinfos.zipWithIndex.map(a => { + val tabcond = a._1.getCondition + MaprDBPartition(a._2, tableName, a._1.getLocations, DBClient().getEstimatedSize(a._1), + DBQueryCondition(tabcond)).asInstanceOf[Partition] + }) + logDebug("Partitions for the table:"+tableName+" are "+splits) + return splits.toArray + } + + override def getPreferredLocations(split: Partition): Seq[String] = { + logDebug("Preferred Locations: "+split.asInstanceOf[MaprDBPartition].locations) + split.asInstanceOf[MaprDBPartition].locations + } + + override def compute(split: Partition, context: TaskContext): Iterator[T] = { + val cd = split.asInstanceOf[MaprDBPartition].cond.condition + var combinedCond = DBClient().newCondition() + var isFullTableScan = true + + if (cd !=null && !cd.isEmpty) { + combinedCond.condition(cd) + isFullTableScan = false + } + + if (condition != null && !condition.condition.isEmpty) + if (isFullTableScan) combinedCond.condition(condition.condition) + else { + combinedCond = DBClient().newCondition().and().condition(condition.condition).condition(cd).close() + } + + logDebug("Condition applied during table.find:" + combinedCond.toString) + + var itrs: java.util.Iterator[Document] = null + if (columns != null) { + logDebug("Columns projected from table:" + columns) + itrs = table.find(combinedCond.build(), columns.toArray: _*).iterator() + } + else + itrs = table.find(combinedCond.build()).iterator() + val ojaiCursor = reqType.getValue(itrs, beanClass) + + context.addTaskCompletionListener((ctx: TaskContext) => { + logDebug("Task completed") + }) + ojaiCursor + } + + override def copy(tblName : String = tableName, columns : Seq[String] = columns, + cnd: DBQueryCondition = condition, bclass: Class[T] = beanClass): Self = + new MapRDBTableScanRDD[T](sparkSession,sc, cnf, columns, tblName, cnd, bclass) +} + +object MapRDBTableScanRDD { + def apply[T : ClassTag](sparkSession: SparkSession, sc: SparkContext, cnf : Broadcast[SerializableConfiguration], + tableName: String, columns: Seq[String], + cond : DBQueryCondition, beanClass: Class[T])(implicit f: RDDTYPE[T]): MapRDBTableScanRDD[T] = { + + new MapRDBTableScanRDD[T](sparkSession, + sc = sc, + cnf, columns, + tableName = tableName, + cond, beanClass) + } +} + diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/RDDTYPE.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/RDDTYPE.scala new file mode 100644 index 0000000000000..6370716818e7a --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/RDDTYPE.scala @@ -0,0 +1,31 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.RDD + +import java.util + +import com.mapr.db.spark.documentUtils.{BeanIterator, JavaBeanIterator, OJAIDocumentIterator} +import com.mapr.db.spark.impl.OJAIDocument +import org.ojai.Document + + +sealed trait RDDTYPE[T] extends Serializable { + def getValue(docs: java.util.Iterator[Document], beanClass: Class[T]): Iterator[T] +} + +object RDDTYPE extends BaseRDDTYPE { + implicit def defaultType: RDDTYPE[OJAIDocument] = new RDDTYPE[OJAIDocument] { + def getValue(docs: java.util.Iterator[Document], beanClass: Class[OJAIDocument]): Iterator[OJAIDocument] = + new OJAIDocumentIterator(docs) + } +} + +trait BaseRDDTYPE { + implicit def overridedefaulttype[T <: AnyRef]: RDDTYPE[T] = new RDDTYPE[T] { + def getValue(doc: java.util.Iterator[Document], beanClass: Class[T]): Iterator[T] = + new BeanIterator(doc, beanClass) + } + + def overrideJavaDefaultType[T <: java.lang.Object]: RDDTYPE[T] = new RDDTYPE[T] { + override def getValue(doc: util.Iterator[Document], beanClass: Class[T]): Iterator[T] = new JavaBeanIterator(doc, beanClass) + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/api/java/MapRDBJavaRDD.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/api/java/MapRDBJavaRDD.scala new file mode 100644 index 0000000000000..4e775d0de1d9d --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/api/java/MapRDBJavaRDD.scala @@ -0,0 +1,22 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.RDD.api.java; + +import com.mapr.db.spark.RDD.MapRDBBaseRDD +import org.apache.spark.api.java.JavaRDD +import org.ojai.store.QueryCondition +import scala.reflect.ClassTag + +case class MapRDBJavaRDD[R : ClassTag](override val rdd: MapRDBBaseRDD[R]) + extends JavaRDD[R](rdd) { + + def where(condition: QueryCondition): MapRDBJavaRDD[R] = { + MapRDBJavaRDD(rdd.where(condition)) + } + + @annotation.varargs + def select(projectedFields: String*) : MapRDBJavaRDD[R] = { + MapRDBJavaRDD(rdd.select[String](projectedFields:_*)) + } +} + + diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/partition/MapRDBPartition.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/partition/MapRDBPartition.scala new file mode 100644 index 0000000000000..4f849e52c5ffe --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/partition/MapRDBPartition.scala @@ -0,0 +1,24 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.RDD.partition + +import org.apache.spark.Partition +import com.mapr.db.spark.condition.DBQueryCondition +/** + * An identifier for a partition in a MapRTableScanRDD. + * + * @param index The partition's index within its parent RDD + * @param locations The preferred locations (hostnames) for the data + * @param tableName name of the table to which this partiton belongs. + * @param cond queryCondition associated with a partition + */ + +case class MaprDBPartition(val index: Int, val tableName: String, val locations: Seq[String], + val size: Long, val cond : DBQueryCondition) extends Partition { + + override def hashCode(): Int = super.hashCode() + + override def equals(other: Any): Boolean = other match { + case p: MaprDBPartition if tableName.equals(p.tableName) && locations.equals(p.locations) && size.equals(p.size) => true + case _ => false + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/partitioner/MapRDBBulkLoadPartitioner.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/partitioner/MapRDBBulkLoadPartitioner.scala new file mode 100644 index 0000000000000..5a12a2c998cb0 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/partitioner/MapRDBBulkLoadPartitioner.scala @@ -0,0 +1,89 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.RDD.partitioner + +import java.nio.ByteBuffer + +import com.mapr.db.impl.{ConditionImpl, IdCodec} +import com.mapr.db.spark.types.DBBinaryValue +import org.apache.spark.Partitioner +import org.ojai.Value +import scala.language.implicitConversions +import com.mapr.db.impl.ConditionNode.RowkeyRange +import com.mapr.db.spark.utils.{LoggingTrait, MapRDBUtils} +import com.mapr.db.spark.MapRDBSpark +import com.mapr.db.spark.dbclient.DBClient + + +object MapRDBPartitioner { + def apply[T](table: String)(implicit k: OJAIKEY[T]): Partitioner = { + var keys : Seq[Value] = DBClient().getTabletInfos(table).map(tableinfo => + IdCodec.decode(tableinfo.getCondition.asInstanceOf[ConditionImpl].getRowkeyRanges.get(0).getStopRow)).toSeq + keys = keys.dropRight(1) + if (keys.isEmpty) + return MapRDBPartitioner[String](Seq.empty) + + if (!(k.getclass() == "String" && keys(0).getType == Value.Type.STRING) && + !((k.getclass() == "ByteBuffer" || k.getclass() == "DBBinaryValue") && keys(0).getType == Value.Type.BINARY)) + throw new RuntimeException("partition keys donot match: " + "RDD's key is of type " + k.getclass + " and table's ID is of type " + keys(0).getType) + + if (keys(0).getType == Value.Type.STRING) { + return MapRDBPartitioner[String](keys.map(_.getString)) + } else { + return MapRDBPartitioner(keys.map(value => MapRDBSpark.serializableBinaryValue(value.getBinary))) + } + } + + def apply[T](splits: Seq[T])(implicit ojaikey: OJAIKEY[T]): MapRDBSplitPartitioner[T] = { + new MapRDBSplitPartitioner[T](splits) {} + } +} + +private[spark] abstract class MapRDBPartitioner extends Partitioner with LoggingTrait { + @transient private[spark] val splits : Seq[Value] +} + +//private[spark] abstract case class MapRDBTablePartitioner[T] (var tableName: String)(implicit ojaikey: OJAIKEY[T]) extends MapRDBPartitioner { +// +// @transient private lazy val table = DBClient().getTable(tableName) +// @transient private lazy val tabletinfos = DBClient().getTabletInfos(tableName,null) +// @transient private lazy val maptabletinfos: Map[ScanRange, Int] = tabletinfos.zipWithIndex.toMap +// +// @transient override lazy val splits : Seq[Value] = { +// val tbinfos = tabletinfos.map(tabletInfo => IdCodec.decode(tabletInfo.getCondition.asInstanceOf[ConditionImpl].getRowkeyRanges.get(0).getStopRow)) +// tbinfos.dropRight(1) +// } +// +// override def numPartitions: Int = tabletinfos.length +// +// override def getPartition(key: Any): Int = { +// maptabletinfos.get(ojaikey.getTabletInfo(maptabletinfos, ojaikey.getValue(key))) match { +// case Some(a) => a +// case None => logError("No Partition exists for key: "+ key) +// throw new RuntimeException("no partition for this key") +// } +// } +//} + +private[spark] abstract case class MapRDBSplitPartitioner[T] (@transient var inputSplits: Seq[T])(implicit ojaikey: OJAIKEY[T]) extends MapRDBPartitioner { + + private[spark] val splitsinBinary: Seq[DBBinaryValue] = inputSplits.map(ojaikey.getBytes(_)).map(ByteBuffer.wrap(_)).map(new DBBinaryValue(_)) + @transient private lazy val ranges: Seq[RowkeyRange] = ((Seq(null.asInstanceOf[ojaikey.Self]) ++ splitsinBinary.map(ojaikey.getValueFromBinary(_))) + .zip(splitsinBinary.map(ojaikey.getValueFromBinary(_)) ++ Seq(null.asInstanceOf[ojaikey.Self]))) + .map(range => ojaikey.getRange(range)) + + @transient override lazy val splits : Seq[Value] = { + splitsinBinary.map(value => value.getByteBuffer().array()).map(bytes => IdCodec.decode(bytes)) + } + + override def numPartitions: Int = splitsinBinary.size + 1 + + override def getPartition(key: Any): Int = { + var partition: Int = 0; + for(thisrange <- ranges) { + if (MapRDBUtils.containsRow(ojaikey.getBytes(key), thisrange)) return partition + else partition = partition+ 1 + } + logError("No Partition exists for key: "+ key) + throw new RuntimeException("no partition for this key") + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/partitioner/OJAIKEY.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/partitioner/OJAIKEY.scala new file mode 100644 index 0000000000000..ed7e4c50627c0 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/RDD/partitioner/OJAIKEY.scala @@ -0,0 +1,84 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.RDD.partitioner + +import java.nio.ByteBuffer + +import com.mapr.db.{Table, TabletInfo} +import com.mapr.db.impl.ConditionNode.RowkeyRange +import com.mapr.db.impl.IdCodec +import com.mapr.db.spark.types.DBBinaryValue + + +trait OJAIKEY[T] extends Serializable { + type Self + def getValue(value: Any): Self + def getTabletInfo(table: Table, value: Self) + def getRange(splitkeys: (Self, Self)): RowkeyRange + def getBytes(value: Any): Array[Byte] + def getValueFromBinary(value: DBBinaryValue): Self + def getclass() : String +} + +object OJAIKEY { + implicit def idkey = new OJAIKEY[DBBinaryValue] { + override type Self = ByteBuffer + override def getValue(value: Any): Self = value.asInstanceOf[DBBinaryValue].getByteBuffer() + override def getTabletInfo(table: Table, value: Self) = table.getTabletInfo(value) + override def getRange(splitkeys: (Self, Self)): RowkeyRange = { + if (splitkeys._1 == null) { + if (splitkeys._2 == null) { + new RowkeyRange(null, null) + } else { + new RowkeyRange(null,splitkeys._2.array()) + } + } else { + if (splitkeys._2 == null) { + new RowkeyRange(splitkeys._1.array(),null) + } else + new RowkeyRange(splitkeys._1.array(),splitkeys._2.array()) + } + } + override def getBytes(value: Any): Array[Byte] = IdCodec.encodeAsBytes(value.asInstanceOf[DBBinaryValue].getByteBuffer()) + + override def getValueFromBinary(value: DBBinaryValue) = IdCodec.decodeBinary(value.getByteBuffer()) + + override def getclass(): String = "DBBinaryValue" + } + + implicit def idbytebuff = new OJAIKEY[ByteBuffer] { + override type Self = ByteBuffer + override def getValue(value: Any): Self = value.asInstanceOf[ByteBuffer] + override def getTabletInfo(table: Table, value: Self) = table.getTabletInfo(value) + override def getRange(splitkeys: (Self, Self)): RowkeyRange = { + if (splitkeys._1 == null) { + if (splitkeys._2 == null) { + new RowkeyRange(null, null) + } else { + new RowkeyRange(null,splitkeys._2.array()) + } + } else { + if (splitkeys._2 == null) { + new RowkeyRange(splitkeys._1.array(),null) + } else + new RowkeyRange(splitkeys._1.array(),splitkeys._2.array()) + } + } + override def getBytes(value: Any): Array[Byte] = IdCodec.encodeAsBytes(value.asInstanceOf[ByteBuffer]) + + override def getValueFromBinary(value: DBBinaryValue) = IdCodec.decodeBinary(value.getByteBuffer()) + + override def getclass(): String = "ByteBuffer" + } + + implicit def strkey = new OJAIKEY[String] { + override type Self = String + override def getValue(value: Any): Self = value.asInstanceOf[Self] + override def getTabletInfo(table: Table, value: Self) = table.getTabletInfo(value) + override def getRange(splitkeys: (Self, Self)): RowkeyRange = new RowkeyRange(IdCodec.encodeAsBytes(splitkeys._1), + IdCodec.encodeAsBytes(splitkeys._2)) + override def getBytes(value: Any): Array[Byte] = IdCodec.encodeAsBytes(value.asInstanceOf[String]) + + override def getValueFromBinary(value: DBBinaryValue) = IdCodec.decodeString(value.array()) + override def getclass(): String = "String" + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/SparkContextFunctions.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/SparkContextFunctions.scala new file mode 100644 index 0000000000000..f2b3e067c9a2c --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/SparkContextFunctions.scala @@ -0,0 +1,22 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark + +import utils.DefaultClass.DefaultType +import org.apache.spark.SparkContext +import com.mapr.db.spark.utils.MapRSpark + +import scala.reflect.ClassTag +import scala.reflect._ +import com.mapr.db.spark.RDD.RDDTYPE +import com.mapr.db.spark.impl.OJAIDocument + +case class SparkContextFunctions( @transient val sc: SparkContext) extends Serializable { + /** + * Spark MapRDB connector specific functions to load json tables as RDD[OJAIDocument] + * @param tableName name of the table in MapRDB + * @example val docs = sc.loadMapRDBTable("tableName") + */ + def loadFromMapRDB[T: ClassTag](tableName: String)(implicit e: T DefaultType OJAIDocument, f: RDDTYPE[T]) = + MapRSpark.builder.sparkContext(sc).configuration().setTable(tableName).build().toRDD[T](classTag[T].runtimeClass.asInstanceOf[Class[T]]) + +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/MapRDBJavaContext.java b/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/MapRDBJavaContext.java new file mode 100644 index 0000000000000..a3fbadd0d862f --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/MapRDBJavaContext.java @@ -0,0 +1,38 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.api.java; + +import com.mapr.db.spark.impl.OJAIDocument; +import org.apache.spark.SparkContext; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +public class MapRDBJavaContext { + private MapRDBJavaContext() { + assert false; + } + + public static SparkContextJavaFunctions maprDBSparkContext(SparkContext sparkContext) { + return new SparkContextJavaFunctions(sparkContext); + } + + public static SparkContextJavaFunctions maprDBSparkContext(JavaSparkContext sparkContext) { + return new SparkContextJavaFunctions(JavaSparkContext.toSparkContext(sparkContext)); + } + + public static RDDJavaFunctions maprDBSparkContext(JavaRDD rdd) { + return new RDDJavaFunctions(rdd); + } + + public static RDDBeanJavaFunctions maprDBSparkContext(JavaRDD rdd, Class clazz) { + return new RDDBeanJavaFunctions(rdd, clazz); + } + + public static PairedRDDJavaFunctions maprDBSparkContext(JavaPairRDD rdd, Class keyClazz) { + return new PairedRDDJavaFunctions(rdd, keyClazz); + } + + public static PairedRDDBeanJavaFunctions maprDBSparkContext(JavaPairRDD rdd, Class keyClazz, Class valueClazz) { + return new PairedRDDBeanJavaFunctions(rdd, keyClazz, valueClazz); + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/PairedRDDBeanJavaFunctions.java b/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/PairedRDDBeanJavaFunctions.java new file mode 100644 index 0000000000000..1957475b998fb --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/PairedRDDBeanJavaFunctions.java @@ -0,0 +1,35 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.api.java; + +import com.mapr.db.spark.RDD.PairedDocumentRDDFunctions; +import com.mapr.db.spark.utils.MapRDBUtils; +import com.mapr.db.spark.writers.OJAIKey; +import com.mapr.db.spark.writers.OJAIValue; +import com.mapr.db.spark.writers.OJAIValue$; +import org.apache.spark.api.java.JavaPairRDD; +import scala.reflect.ClassTag$; + +public class PairedRDDBeanJavaFunctions { + public final JavaPairRDD rdd; + public final PairedDocumentRDDFunctions ojaiDocumentRDDFunctions; + + + public PairedRDDBeanJavaFunctions(JavaPairRDD rdd, Class keyClazz, Class valueClazz) { + OJAIValue val = OJAIValue$.MODULE$.overrideDefault(); + OJAIKey key = MapRDBUtils.getOjaiKey(ClassTag$.MODULE$.apply(keyClazz)); + this.rdd = rdd; + this.ojaiDocumentRDDFunctions = new PairedDocumentRDDFunctions(rdd.rdd(), key, val); + } + + public void saveToMapRDB(String tableName) { + this.saveToMapRDB(tableName, false, false); + } + + public void saveToMapRDB(String tableName, boolean createTable) { + this.saveToMapRDB(tableName, createTable, false); + } + + public void saveToMapRDB(String tableName, boolean createTable, boolean bulkInsert) { + this.ojaiDocumentRDDFunctions.saveToMapRDB(tableName, createTable, bulkInsert); + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/PairedRDDJavaFunctions.java b/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/PairedRDDJavaFunctions.java new file mode 100644 index 0000000000000..a779380e6e1b8 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/PairedRDDJavaFunctions.java @@ -0,0 +1,36 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.api.java; + +import com.mapr.db.spark.RDD.PairedDocumentRDDFunctions; +import com.mapr.db.spark.impl.OJAIDocument; +import com.mapr.db.spark.utils.MapRDBUtils; +import com.mapr.db.spark.writers.OJAIKey; +import com.mapr.db.spark.writers.OJAIValue; +import com.mapr.db.spark.writers.OJAIValue$; +import org.apache.spark.api.java.JavaPairRDD; +import scala.reflect.ClassTag$; + +public class PairedRDDJavaFunctions { + public final JavaPairRDD rdd; + public final PairedDocumentRDDFunctions ojaiDocumentRDDFunctions; + + + public PairedRDDJavaFunctions(JavaPairRDD rdd, Class keyClazz) { + OJAIValue val = OJAIValue$.MODULE$.defaultOJAIDocument(); + OJAIKey key = MapRDBUtils.getOjaiKey(ClassTag$.MODULE$.apply(keyClazz)); + this.rdd = rdd; + this.ojaiDocumentRDDFunctions = new PairedDocumentRDDFunctions(rdd.rdd(), key, val); + } + + public void saveToMapRDB(String tableName) { + this.saveToMapRDB(tableName, false, false); + } + + public void saveToMapRDB(String tableName, boolean createTable) { + this.saveToMapRDB(tableName, createTable, false); + } + + public void saveToMapRDB(String tableName, boolean createTable, boolean bulkInsert) { + this.ojaiDocumentRDDFunctions.saveToMapRDB(tableName, createTable, bulkInsert); + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/RDDBeanJavaFunctions.java b/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/RDDBeanJavaFunctions.java new file mode 100644 index 0000000000000..27096eb351e77 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/RDDBeanJavaFunctions.java @@ -0,0 +1,36 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.api.java; + +import com.mapr.db.spark.RDD.OJAIDocumentRDDFunctions; +import com.mapr.db.spark.writers.OJAIValue; +import com.mapr.db.spark.writers.OJAIValue$; +import org.apache.spark.api.java.JavaRDD; +import org.ojai.DocumentConstants; + +public class RDDBeanJavaFunctions { + public final JavaRDD rdd; + public final OJAIDocumentRDDFunctions ojaiDocumentRDDFunctions; + + + public RDDBeanJavaFunctions(JavaRDD rdd, Class clazz) { + OJAIValue val = OJAIValue$.MODULE$.overrideJavaDefault(); + this.rdd = rdd; + this.ojaiDocumentRDDFunctions = new OJAIDocumentRDDFunctions(rdd.rdd(), val); + } + + public void saveToMapRDB(String tableName) { + this.saveToMapRDB(tableName, false, false, DocumentConstants.ID_KEY); + } + + public void saveToMapRDB(String tableName, boolean createTable) { + this.saveToMapRDB(tableName, createTable, false, DocumentConstants.ID_KEY); + } + + public void saveToMapRDB(String tableName, boolean createTable, boolean bulkInsert) { + this.saveToMapRDB(tableName, createTable, bulkInsert, DocumentConstants.ID_KEY); + } + + public void saveToMapRDB(String tableName, boolean createTable, boolean bulkInsert, String ID) { + this.ojaiDocumentRDDFunctions.saveToMapRDB(tableName, createTable, bulkInsert, ID); + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/RDDJavaFunctions.java b/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/RDDJavaFunctions.java new file mode 100644 index 0000000000000..a41ed487f5e98 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/RDDJavaFunctions.java @@ -0,0 +1,37 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.api.java; + +import com.mapr.db.spark.RDD.OJAIDocumentRDDFunctions; +import com.mapr.db.spark.impl.OJAIDocument; +import com.mapr.db.spark.writers.OJAIValue; +import com.mapr.db.spark.writers.OJAIValue$; +import org.apache.spark.api.java.JavaRDD; +import org.ojai.DocumentConstants; + +public class RDDJavaFunctions { + public final JavaRDD rdd; + public final OJAIDocumentRDDFunctions ojaiDocumentRDDFunctions; + + + public RDDJavaFunctions(JavaRDD rdd) { + OJAIValue val = OJAIValue$.MODULE$.defaultOJAIDocument(); + this.rdd = rdd; + this.ojaiDocumentRDDFunctions = new OJAIDocumentRDDFunctions(rdd.rdd(), val); + } + + public void saveToMapRDB(String tableName) { + this.saveToMapRDB(tableName, false, false, DocumentConstants.ID_KEY); + } + + public void saveToMapRDB(String tableName, boolean createTable) { + this.saveToMapRDB(tableName, createTable, false, DocumentConstants.ID_KEY); + } + + public void saveToMapRDB(String tableName, boolean createTable, boolean bulkInsert) { + this.saveToMapRDB(tableName, createTable, bulkInsert, DocumentConstants.ID_KEY); + } + + public void saveToMapRDB(String tableName, boolean createTable, boolean bulkInsert, String ID) { + this.ojaiDocumentRDDFunctions.saveToMapRDB(tableName, createTable, bulkInsert, ID); + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/SparkContextJavaFunctions.java b/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/SparkContextJavaFunctions.java new file mode 100644 index 0000000000000..c71b3b88b349a --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/api/java/SparkContextJavaFunctions.java @@ -0,0 +1,37 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.api.java; + +import com.mapr.db.spark.RDD.MapRDBBaseRDD; +import com.mapr.db.spark.RDD.RDDTYPE; +import com.mapr.db.spark.RDD.RDDTYPE$; +import com.mapr.db.spark.RDD.api.java.MapRDBJavaRDD; +import com.mapr.db.spark.impl.OJAIDocument; +import com.mapr.db.spark.utils.MapRSpark; +import org.apache.spark.SparkContext; +import scala.reflect.ClassTag$; +import org.apache.hadoop.conf.Configuration; + + +public class SparkContextJavaFunctions { + public final SparkContext sparkContext; + + public SparkContextJavaFunctions(SparkContext sparkContext) { + this.sparkContext = sparkContext; + } + + public MapRDBJavaRDD loadFromMapRDB(String tableName) { + RDDTYPE f = RDDTYPE$.MODULE$.defaultType(); + MapRDBBaseRDD rdd = MapRSpark.builder().sparkContext(sparkContext) + .configuration(new Configuration()) + .setTable(tableName).build().toJavaRDD(OJAIDocument.class, ClassTag$.MODULE$.apply(OJAIDocument.class), f); + return new MapRDBJavaRDD<>(rdd, ClassTag$.MODULE$.apply(OJAIDocument.class)); + } + + public MapRDBJavaRDD loadFromMapRDB(String tableName, Class clazz) { + RDDTYPE f = RDDTYPE$.MODULE$.overrideJavaDefaultType(); + MapRDBBaseRDD rdd = MapRSpark.builder().sparkContext(sparkContext) + .configuration(new Configuration()) + .setTable(tableName).build().toJavaRDD(clazz, ClassTag$.MODULE$.apply(clazz), f); + return new MapRDBJavaRDD(rdd, ClassTag$.MODULE$.apply(clazz)); + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/codec/BeanCodec.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/codec/BeanCodec.scala new file mode 100644 index 0000000000000..762d96afa97cd --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/codec/BeanCodec.scala @@ -0,0 +1,40 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.codec + +import java.io.IOException +import org.ojai.annotation.API +import com.mapr.db.spark.codec.JacksonBeanCodecHelper._ +import org.ojai.{Document, DocumentBuilder, DocumentReader} +import org.ojai.beans.jackson.{DocumentGenerator, DocumentParser} +import org.ojai.exceptions.{DecodingException, EncodingException} + +@API.Internal +object BeanCodec { + @throws[DecodingException] + def decode(db: DocumentBuilder, bean: Any): Document = { + if (bean == null) return null + val gen: DocumentGenerator = new DocumentGenerator(db) + try { + MAPPER.writeValue(gen, bean) + return gen.getDocument + } + catch { + case e: Exception => { + throw new DecodingException("Failed to convert the java bean to Document", e) + } + } + } + + @throws[EncodingException] + def encode[T](dr: DocumentReader, beanClass: Class[T]): T = { + if (dr == null) return null.asInstanceOf[T] + try { + return MAPPER.readValue(new DocumentParser(dr), beanClass) + } + catch { + case e: IOException => { + throw new EncodingException("Failed to create java bean from Document", e) + } + } + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/codec/JacksonBeanCodecHelper.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/codec/JacksonBeanCodecHelper.scala new file mode 100644 index 0000000000000..18086e31a8fb9 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/codec/JacksonBeanCodecHelper.scala @@ -0,0 +1,120 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.codec + +import org.ojai.beans.jackson.JacksonHelper +import java.io.IOException +import org.ojai.annotation.API +import org.ojai.types.ODate +import org.ojai.types.OInterval +import org.ojai.types.OTime +import org.ojai.types.OTimestamp +import com.fasterxml.jackson.core.JsonGenerator +import com.fasterxml.jackson.core.JsonParser +import com.fasterxml.jackson.core.JsonProcessingException +import com.fasterxml.jackson.core.Version +import com.fasterxml.jackson.core.util.VersionUtil +import com.fasterxml.jackson.databind.DeserializationContext +import com.fasterxml.jackson.databind.JsonDeserializer +import com.fasterxml.jackson.databind.JsonSerializer +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.databind.SerializerProvider +import com.fasterxml.jackson.databind.module.SimpleModule +import com.fasterxml.jackson.module.scala.DefaultScalaModule + +@API.Internal +object JacksonBeanCodecHelper { + + var VERSION: Version = null + val MAPPER: ObjectMapper = new ObjectMapper + + class ByteSerializer extends JsonSerializer[Byte] { + @throws[IOException] + @throws[JsonProcessingException] + def serialize(value: Byte, jgen: JsonGenerator, provider: SerializerProvider) { + jgen.writeObject(value) + } + } + + class IntervalSerializer extends JsonSerializer[OInterval] { + @throws[IOException] + @throws[JsonProcessingException] + def serialize(value: OInterval, jgen: JsonGenerator, provider: SerializerProvider) { + jgen.writeObject(value) + } + } + + class IntervalDeserializer extends JsonDeserializer[OInterval] { + @throws[IOException] + @throws[JsonProcessingException] + def deserialize(p: JsonParser, ctxt: DeserializationContext): OInterval = { + return p.getEmbeddedObject.asInstanceOf[OInterval] + } + } + + class DateSerializer extends JsonSerializer[ODate] { + @throws[IOException] + @throws[JsonProcessingException] + def serialize(value: ODate, jgen: JsonGenerator, provider: SerializerProvider) { + jgen.writeObject(value) + } + } + + class DateDeserializer extends JsonDeserializer[ODate] { + @throws[IOException] + @throws[JsonProcessingException] + def deserialize(p: JsonParser, ctxt: DeserializationContext): ODate = { + return p.getEmbeddedObject.asInstanceOf[ODate] + } + } + + class TimeSerializer extends JsonSerializer[OTime] { + @throws[IOException] + @throws[JsonProcessingException] + def serialize(value: OTime, jgen: JsonGenerator, provider: SerializerProvider) { + jgen.writeObject(value) + } + } + + class TimeDeserializer extends JsonDeserializer[OTime] { + @throws[IOException] + @throws[JsonProcessingException] + def deserialize(p: JsonParser, ctxt: DeserializationContext): OTime = { + return p.getEmbeddedObject.asInstanceOf[OTime] + } + } + + class TimestampSerializer extends JsonSerializer[OTimestamp] { + @throws[IOException] + @throws[JsonProcessingException] + def serialize(value: OTimestamp, jgen: JsonGenerator, provider: SerializerProvider) { + jgen.writeObject(value) + } + } + + class TimestampDeserializer extends JsonDeserializer[OTimestamp] { + @throws[IOException] + @throws[JsonProcessingException] + def deserialize(p: JsonParser, ctxt: DeserializationContext): OTimestamp = { + return p.getEmbeddedObject.asInstanceOf[OTimestamp] + } + } + + val version: String = classOf[JacksonHelper].getPackage.getImplementationVersion + val version_str: String = if (version == null) "" + else version + VERSION = VersionUtil.parseVersion(version_str, "org.ojai", "core") + val module: SimpleModule = new SimpleModule("OjaiSerializers", VERSION) + val byteSerializer: JacksonBeanCodecHelper.ByteSerializer = new JacksonBeanCodecHelper.ByteSerializer + module.addSerializer(classOf[Byte], byteSerializer) + module.addSerializer(classOf[OInterval], new JacksonHelper.IntervalSerializer) + module.addDeserializer(classOf[OInterval], new JacksonHelper.IntervalDeserializer) + module.addSerializer(classOf[ODate], new JacksonHelper.DateSerializer) + module.addDeserializer(classOf[ODate], new JacksonHelper.DateDeserializer) + module.addSerializer(classOf[OTime], new JacksonHelper.TimeSerializer) + module.addDeserializer(classOf[OTime], new JacksonHelper.TimeDeserializer) + module.addSerializer(classOf[OTimestamp], new JacksonHelper.TimestampSerializer) + module.addDeserializer(classOf[OTimestamp], new JacksonHelper.TimestampDeserializer) + MAPPER.registerModule(DefaultScalaModule) + MAPPER.registerModule(module) +} + diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/condition/DBQueryCondition.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/condition/DBQueryCondition.scala new file mode 100644 index 0000000000000..abcbc638af41f --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/condition/DBQueryCondition.scala @@ -0,0 +1,35 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.condition + +import java.io.{Externalizable, ObjectInput, ObjectOutput} +import org.ojai.store.QueryCondition +import com.mapr.db.impl.ConditionImpl +import com.mapr.db.spark.utils.MapRDBUtils +import com.mapr.db.util.ByteBufs + + +private[spark] class DBQueryCondition(@transient var condition: QueryCondition) extends Externalizable { + + def this() { + this(null) + } + + override def readExternal(in: ObjectInput): Unit = { + val size = in.readInt() + val condSerialized = ByteBufs.allocate(size) + MapRDBUtils.readBytes(condSerialized,size,in) + this.condition = ConditionImpl.parseFrom(condSerialized) + } + + override def writeExternal(out: ObjectOutput): Unit = { + val desc = condition.asInstanceOf[ConditionImpl].getDescriptor + val condSerialized = desc.getSerialized + out.writeInt(condSerialized.capacity()) + out.write(condSerialized.array()) + } +} + +object DBQueryCondition { + def apply(cond: QueryCondition): DBQueryCondition = new DBQueryCondition(cond) +} + diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/condition/OJAICondition.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/condition/OJAICondition.scala new file mode 100644 index 0000000000000..cfd3974b00308 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/condition/OJAICondition.scala @@ -0,0 +1,124 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.condition + +import com.mapr.db.spark.dbclient.DBClient +import org.ojai.store.QueryCondition +import scala.language.implicitConversions +import scala.collection.JavaConverters._ +import com.mapr.db.spark.field +import org.ojai.exceptions.TypeException + +trait Predicate extends Serializable{ + def and (rhs: Predicate) : AND = new AND(this, rhs) + def or (rhs: Predicate) : OR = new OR(this, rhs) + protected val closeParan = "}" + protected val openParan = "{" + protected def prettyPrint(elements: Seq[String]): String = elements.mkString(",") + protected def prettyPrint(mapelems: Map[String, String]) : String = + mapelems.foldLeft("") { (s: String, pair: (String, String)) => s + pair._1 +":"+ pair._2 +","}.stripSuffix(",") + def build : QueryCondition +} + +trait quotes[T] extends Serializable{ + type Self + def build(colName: String,value: T, oper: QueryCondition.Op): QueryCondition +} + +case class AND(lhs: Predicate, rhs: Predicate) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().and().condition(lhs.build.build()).condition(rhs.build.build()).close() +} + +case class OR(lhs: Predicate, rhs: Predicate) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().or().condition(lhs.build.build()).condition(rhs.build.build()).close() +} + +case class greaterThan[T: quotes](c: field, rhs: T)(implicit ev: quotes[T]) extends Predicate { + override def build: QueryCondition = ev.build(c.fieldPath, rhs, QueryCondition.Op.GREATER) +} + +case class greaterThanEquals[T : quotes](c: field, rhs: T)(implicit ev: quotes[T]) extends Predicate { + override def build: QueryCondition = ev.build(c.fieldPath, rhs, QueryCondition.Op.GREATER_OR_EQUAL) +} + +case class lessThan[T: quotes](c: field, rhs: T)(implicit ev: quotes[T]) extends Predicate { + override def build: QueryCondition = ev.build(c.fieldPath, rhs, QueryCondition.Op.LESS) +} + +case class lessThanEquals[T: quotes](c: field, rhs: T)(implicit ev: quotes[T]) extends Predicate { + override def build: QueryCondition = ev.build(c.fieldPath, rhs, QueryCondition.Op.LESS_OR_EQUAL) +} + +case class equalsTo[T: quotes](c: field, rhs: T)(implicit ev: quotes[T]) extends Predicate { + override def build: QueryCondition = ev.build(c.fieldPath, rhs, QueryCondition.Op.EQUAL) +} + +case class equalsToSeq(c: field, rhs: Seq[AnyRef]) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().equals(c.fieldPath, rhs.asJava) +} + +case class equalsToMap(c: field, rhs: Map[String, AnyRef]) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().equals(c.fieldPath, rhs.asJava) +} + +case class notEqualsTo[T: quotes](c: field, rhs: T)(implicit ev: quotes[T]) extends Predicate { + override def build: QueryCondition = ev.build(c.fieldPath, rhs, QueryCondition.Op.NOT_EQUAL) +} + +case class notEqualsToSeq(c: field, rhs: Seq[AnyRef]) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().notEquals(c.fieldPath, rhs.asJava) +} + +case class notEqualsToMap(c: field, rhs: Map[String, AnyRef]) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().notEquals(c.fieldPath, rhs.asJava) +} + +case class between[T : quotes](c: field, rhs1: T, rhs2 : T)(implicit ev: quotes[T]) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().and().condition(ev.build(c.fieldPath, rhs1, QueryCondition.Op.GREATER_OR_EQUAL).build()) + .condition(ev.build(c.fieldPath, rhs2, QueryCondition.Op.LESS_OR_EQUAL).build()).close() +} + +case class exists(c: field) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().exists(c.fieldPath) +} + +case class IN(c: field, rhs: Seq[AnyRef]) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().in(c.fieldPath, rhs.asJava) +} + +case class NOTIN(c: field, rhs: Seq[AnyRef]) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().notIn(c.fieldPath, rhs.asJava) +} + +case class notexists(c: field) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().notExists(c.fieldPath) +} + +case class TYPEOF(c: field, typevalue: String) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().typeOf(c.fieldPath, field.typemap.get(typevalue) + .getOrElse(throw new TypeException("Type: "+ typevalue + " doesn't exist"))) +} + +case class NOTTYPEOF(c: field, typevalue: String) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().notTypeOf(c.fieldPath, field.typemap.get(typevalue) + .getOrElse(throw new TypeException("Type: "+ typevalue + " doesn't exist"))) +} + +case class LIKE(c:field, regex: String) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().like(c.fieldPath, regex) +} + +case class NOTLIKE(c:field, regex: String) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().notLike(c.fieldPath, regex) +} + +case class MATCHES(c: field, regex: String) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().matches(c.fieldPath, regex) +} + +case class NOTMATCHES(c: field, regex: String) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().notMatches(c.fieldPath, regex) +} + +case class SIZEOF(c: field, op: QueryCondition.Op, size: Long) extends Predicate { + override def build: QueryCondition = DBClient().newCondition().sizeOf(c.fieldPath, op, size) +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/condition/package.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/condition/package.scala new file mode 100644 index 0000000000000..0c5dd14d4d9eb --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/condition/package.scala @@ -0,0 +1,86 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark + +import java.nio.ByteBuffer + +import com.mapr.db.spark.dbclient.DBClient +import org.ojai.store.QueryCondition +import org.ojai.types.{ODate, OInterval, OTime, OTimestamp} + +package object condition { + + implicit def quotesInteger = new quotes[Integer] { + override type Self = Integer + override def build(colName: String, value: Integer, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value) + } + + implicit def quotesInt = new quotes[Int] { + override type Self = Int + override def build(colName: String, value: Int, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value) + } + + implicit def quotesLong = new quotes[Long] { + override type Self = Long + override def build(colName: String, value: Long, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value) + } + + implicit def quotesString = new quotes[String] { + override type Self = String + override def build(colName: String, value: String, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value) + } + + implicit def quotesODate = new quotes[ODate] { + override type Self = ODate + override def build(colName: String, value: ODate, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value) + } + + implicit def quotesOTime = new quotes[OTime] { + override type Self = OTime + override def build(colName: String, value: OTime, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value) + } + + implicit def quotesByte = new quotes[Byte] { + override type Self = Byte + override def build(colName: String, value: Byte, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value) + } + + implicit def quotesShort = new quotes[Short] { + override type Self = Short + override def build(colName: String, value: Short, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value) + } + + implicit def quotesBoolean = new quotes[Boolean] { + override type Self = Boolean + override def build(colName: String, value: Boolean, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value) + } + + implicit def quotesBigDecimal = new quotes[BigDecimal] { + override type Self = BigDecimal + override def build(colName: String, value: BigDecimal, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value.bigDecimal) + } + + implicit def quotesDouble = new quotes[Double] { + override type Self = Double + override def build(colName: String, value: Double, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value) + } + + implicit def quotesFloat = new quotes[Float] { + override type Self = Float + override def build(colName: String, value: Float, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value) + } + + implicit def quotesOInterval = new quotes[OInterval] { + override type Self = OInterval + override def build(colName: String, value: OInterval, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value) + } + + implicit def quotesOTimestamp = new quotes[OTimestamp] { + override type Self = OTimestamp + override def build(colName: String, value: OTimestamp, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value) + } + + implicit def quotesByteBuffer = new quotes[ByteBuffer] { + override type Self = ByteBuffer + override def build(colName: String, value: ByteBuffer, oper: QueryCondition.Op) = DBClient().newCondition().is(colName, oper, value) + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/configuration/SerializableConfiguration.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/configuration/SerializableConfiguration.scala new file mode 100644 index 0000000000000..03172ad8fca1b --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/configuration/SerializableConfiguration.scala @@ -0,0 +1,31 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.configuration + +import java.io._ + +import com.esotericsoftware.kryo.io.{Input, Output} +import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import org.apache.hadoop.conf.Configuration + +class SerializableConfiguration(@transient var value: Configuration) extends Serializable with KryoSerializable { + private def writeObject(out: ObjectOutputStream): Unit = { + out.defaultWriteObject() + value.write(out) + } + + private def readObject(in: ObjectInputStream): Unit = { + value = new Configuration(false) + value.readFields(in) + } + + def write(kryo: Kryo, out: Output): Unit = { + val dos = new DataOutputStream(out) + value.write(dos) + dos.flush() + } + + def read(kryo: Kryo, in: Input): Unit = { + value = new Configuration(false) + value.readFields(new DataInputStream(in)) + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/dbclient/DBClient.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/dbclient/DBClient.scala new file mode 100644 index 0000000000000..f3349b8268fa6 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/dbclient/DBClient.scala @@ -0,0 +1,39 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.dbclient + +import com.mapr.db.TabletInfo +import com.mapr.db.{Table, TableDescriptor, TabletInfo} +import org.ojai.store.{DocumentStore, QueryCondition} +import org.ojai.{Document, DocumentBuilder, Value} + +trait DBClient { + def newDocument() : Document + def newDocument(jsonString: String): Document + def newCondition(): QueryCondition + def deleteTable(tablePath: String): Unit + def tableExists(tablePath: String): Boolean + def newTableDescriptor(): TableDescriptor + def createTable(tablePath: String): Unit + def createTable(tableDesc: TableDescriptor): Unit + def createTable(tableDesc: TableDescriptor, keys: Array[Value]): Unit + def isBulkLoad(tablePath: String): Boolean + def alterTable(tableDesc: TableDescriptor): Unit + def getTable(tablePath: String): DocumentStore + def getTableDescriptor(tablePath: String): TableDescriptor + def newDocumentBuilder(): DocumentBuilder + def getTabletInfos(tablePath: String, cond: QueryCondition): Seq[TabletInfo] + def getTabletInfos(tablePath: String): Seq[TabletInfo] + def getEstimatedSize(scanRange: TabletInfo) : Long +} + +object DBClient { + val CLIENT_VERSION = "1.0" + + val newClient = DBOlderClientImpl + val oldClient = DBOlderClientImpl + + def apply(): DBClient = { + if (CLIENT_VERSION.equals("2.0")) return newClient + else return oldClient + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/dbclient/DBOlderClientImpl.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/dbclient/DBOlderClientImpl.scala new file mode 100644 index 0000000000000..ac6fd0c8b41c8 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/dbclient/DBOlderClientImpl.scala @@ -0,0 +1,95 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.dbclient + +import com.mapr.db.{TableDescriptor, TabletInfo, MapRDB} +import com.mapr.db.impl.AdminImpl +import com.mapr.db.TabletInfo +import org.ojai.store.QueryCondition +import org.ojai.{Document, Value} + +object DBOlderClientImpl extends DBClient { + @Override + def newDocument(): Document = { + MapRDB.newDocument() + } + + @Override + override def getTabletInfos(tablePath: String, cond: QueryCondition): Seq[TabletInfo] = { + MapRDB.getTable(tablePath).getTabletInfos(cond) + } + + @Override + override def getTabletInfos(tablePath: String): Seq[TabletInfo] = { + MapRDB.getTable(tablePath).getTabletInfos + } + + @Override + def newDocument(jsonString: String) = { + MapRDB.newDocument(jsonString) + } + + @Override + def newCondition() = { + MapRDB.newCondition() + } + + @Override + def deleteTable(tablePath: String) = { + MapRDB.deleteTable(tablePath) + } + + @Override + def tableExists(tablePath: String) = { + MapRDB.tableExists(tablePath) + } + + @Override + def newTableDescriptor() = { + MapRDB.newTableDescriptor() + } + + @Override + def createTable(tablePath: String) = { + MapRDB.createTable(tablePath) + } + + @Override + def createTable(tableDesc: TableDescriptor) = { + MapRDB.newAdmin().createTable(tableDesc) + } + + @Override + def createTable(tableDesc: TableDescriptor, keys: Array[Value]) = { + MapRDB.newAdmin().asInstanceOf[AdminImpl].createTable(tableDesc, keys) + } + + @Override + def isBulkLoad(tablePath: String) = { + MapRDB.newAdmin().getTableDescriptor(tablePath).isBulkLoad + } + + @Override + def alterTable(tableDesc: TableDescriptor) = { + MapRDB.newAdmin().alterTable(tableDesc) + } + + @Override + def getTable(tablePath: String) = { + MapRDB.getTable(tablePath) + } + + @Override + def getTableDescriptor(tablePath: String) = { + MapRDB.newAdmin().getTableDescriptor(tablePath) + } + + @Override + override def getEstimatedSize(scanRange: TabletInfo): Long = { + return 0 + } + + @Override + def newDocumentBuilder() = { + MapRDB.newDocumentBuilder() + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/documentTypeUtils/DocumentTypeImplicits.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/documentTypeUtils/DocumentTypeImplicits.scala new file mode 100644 index 0000000000000..bb7e66cd9f1b9 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/documentTypeUtils/DocumentTypeImplicits.scala @@ -0,0 +1,244 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.documentTypeUtils + +import com.mapr.db.spark.impl.OJAIDocument +import java.nio.ByteBuffer +import scala.collection.JavaConverters._ +import scala.language.experimental.macros +import scala.language.{dynamics, implicitConversions} +import org.ojai.exceptions.TypeException +import org.ojai.Value +import org.ojai.types._ +import com.mapr.db.spark.types.DBArrayValue +import com.mapr.db.spark.types.DBBinaryValue +import com.mapr.db.spark.types.DBMapValue +import org.ojai.Document + +sealed trait OJAIType[T] { + type Self + def getValue(doc: Document, fieldPath: String) : Self + def setValue(dc: OJAIDocument, name: String,value: T): Unit +} + +object OJAIType { + implicit val ojaianyref = new OJAIType[AnyRef] { + type Self = AnyRef + def getValue(doc: Document, fieldPath: String):Self = { + val result = doc.getValue(fieldPath) + if (result != null && result.getType == Value.Type.MAP) { + if (result.getObject != null) + new DBMapValue(result.getMap.asScala.toMap) + else null + } else if ( result != null && result.getType == Value.Type.ARRAY) { + if (result.getObject != null) + new DBArrayValue(result.getList.asScala) + else null + } else if (result != null && result.getType == Value.Type.BINARY) { + if (result.getObject != null) + new DBBinaryValue(result.getBinary) + else null + } + else { + if (result != null) + return result.getObject + return null + } + } + + def setValue(dc: OJAIDocument, name: String,value: AnyRef): Unit = { + throw new TypeException("Invalid value of datatype " + value.getClass + " is set to the document") + } + } + + implicit val ojaiinteger = new OJAIType[Integer] { + type Self = Integer + def getValue(doc: Document, fieldPath: String): Self = doc.getIntObj(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: Integer): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaiint = new OJAIType[Int] { + type Self = Integer + def getValue(doc: Document, fieldPath: String): Self = doc.getInt(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: Int): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaiDouble = new OJAIType[Double] { + type Self = Double + def getValue(doc: Document, fieldPath: String): Self = doc.getDouble(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: Double): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaiDoubleObj = new OJAIType[java.lang.Double] { + type Self = java.lang.Double + def getValue(doc: Document, fieldPath: String): Self = doc.getDoubleObj(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: java.lang.Double): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaifloat = new OJAIType[Float] { + type Self = Float + def getValue(doc: Document, fieldPath: String): Self = doc.getFloat(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: Float): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaifloatObj = new OJAIType[java.lang.Float] { + type Self = java.lang.Float + def getValue(doc: Document, fieldPath: String): Self = doc.getFloatObj(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: java.lang.Float): Unit = dc.getDoc.set(name, value) + } + + implicit val ojailong = new OJAIType[Long] { + type Self = Long + def getValue(doc: Document, fieldPath: String): Self = doc.getLong(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: Long): Unit = dc.getDoc.set(name, value) + } + + implicit val ojailongObj = new OJAIType[java.lang.Long] { + type Self = java.lang.Long + def getValue(doc: Document, fieldPath: String): Self = doc.getLongObj(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: java.lang.Long): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaidate = new OJAIType[ODate] { + type Self = ODate + def getValue(doc: Document, fieldPath: String): Self = doc.getDate(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: ODate): Unit = dc.getDoc.set(name, value) + } + + implicit val ojatime = new OJAIType[OTime] { + type Self = OTime + def getValue(doc: Document, fieldPath: String): Self = doc.getTime(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: OTime): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaitimestmp = new OJAIType[OTimestamp] { + type Self = OTimestamp + def getValue(doc: Document, fieldPath: String): Self = doc.getTimestamp(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: OTimestamp): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaiintrvl = new OJAIType[OInterval] { + type Self = OInterval + def getValue(doc: Document, fieldPath: String): Self = doc.getInterval(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: OInterval): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaiString = new OJAIType[String] { + type Self = String + def getValue(doc: Document, fieldPath: String): Self = doc.getString(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: String): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaibool = new OJAIType[Boolean] { + type Self = Boolean + def getValue(doc: Document, fieldPath: String): Self = doc.getBoolean(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: Boolean): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaiboolObj = new OJAIType[java.lang.Boolean] { + type Self = java.lang.Boolean + def getValue(doc: Document, fieldPath: String): Self = doc.getBooleanObj(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: java.lang.Boolean): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaibyte = new OJAIType[Byte] { + type Self = Byte + def getValue(doc: Document, fieldPath: String): Self = doc.getByte(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: Byte): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaibyteObj = new OJAIType[java.lang.Byte] { + type Self = java.lang.Byte + def getValue(doc: Document, fieldPath: String): Self = doc.getByte(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: java.lang.Byte): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaishort = new OJAIType[Short] { + type Self = Short + def getValue(doc: Document, fieldPath: String): Self = doc.getShort(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: Short): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaishortObj = new OJAIType[java.lang.Short] { + type Self = java.lang.Short + def getValue(doc: Document, fieldPath: String): Self = doc.getShort(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: java.lang.Short): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaiBigDecimal = new OJAIType[BigDecimal] { + type Self = BigDecimal + def getValue(doc:Document, fieldPath: String) : Self = doc.getDecimal(fieldPath) + def setValue(dc: OJAIDocument, name: String,value: BigDecimal): Unit = dc.getDoc.set(name, value.bigDecimal) + + } + implicit val ojaiarrayvalue = new OJAIType[DBArrayValue[AnyRef]] { + type Self = DBArrayValue[AnyRef] + def getValue(doc: Document, fieldPath: String) : Self = { + val result: java.util.List[Object] = doc.getList(fieldPath) + if (result == null) + return null + else + new DBArrayValue(result.asScala) + } + def setValue(dc: OJAIDocument, name: String,value: DBArrayValue[AnyRef]): Unit = dc.getDoc.set(name, value.arr.asJava) + } + + implicit val ojaimapstringvalue = new OJAIType[Map[String,AnyRef]] { + type Self = DBMapValue + def getValue(doc:Document, fieldPath: String) : Self = { + val result : java.util.Map[String, Object] = doc.getMap(fieldPath) + if (result == null) + return null + else + new DBMapValue(result.asScala.toMap) + } + override def setValue(dc: OJAIDocument, name: String, value: Map[String,AnyRef]): Unit = dc.getDoc.set(name, value.asJava) + } + + implicit val ojaiarrayanyref = new OJAIType[Seq[AnyRef]] { + type Self = Seq[AnyRef] + def getValue(doc: Document, fieldPath: String) : Self = { + val result: java.util.List[Object] = doc.getList(fieldPath) + if (result == null) + return null + else + new DBArrayValue(result.asScala) + } + override def setValue(dc: OJAIDocument, name: String, value: Seq[AnyRef]): Unit = dc.getDoc.set(name,value.map(_.asInstanceOf[AnyRef]).asJava) + } + + implicit val ojaiarrayany = new OJAIType[Seq[Any]] { + type Self = DBArrayValue[AnyRef] + def getValue(doc: Document, fieldPath: String) : Self = { + val result: java.util.List[Object] = doc.getList(fieldPath) + if (result == null) + return null + else + new DBArrayValue(result.asScala) + } + override def setValue(dc: OJAIDocument, name: String, value: Seq[Any]): Unit = dc.getDoc.set(name,value.map(_.asInstanceOf[AnyRef]).asJava) + } + + implicit val ojaibinary = new OJAIType[DBBinaryValue] { + type Self = DBBinaryValue + def getValue(doc: Document, fieldPath: String) : Self = new DBBinaryValue(doc.getBinary(fieldPath)) + override def setValue(dc: OJAIDocument, name: String, value: DBBinaryValue): Unit = dc.getDoc.set(name, value.getByteBuffer()) + } + + implicit val ojaibytebuffer = new OJAIType[ByteBuffer] { + type Self = ByteBuffer + def getValue(doc: Document, fieldPath: String) : Self = doc.getBinary(fieldPath) + override def setValue(dc: OJAIDocument, name: String, value: ByteBuffer): Unit = dc.getDoc.set(name, value) + } + + implicit val ojaimapvalue = new OJAIType[DBMapValue] { + type Self = DBMapValue + def getValue(doc: Document, fieldPath: String) : Self = { + val result:java.util.Map[String, Object] = doc.getMap(fieldPath) + if (result == null) + return null + else + new DBMapValue(result.asScala.toMap) + } + override def setValue(dc: OJAIDocument, name: String, value: DBMapValue): Unit = dc.getDoc.set(name, value.asJava) + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/documentTypeUtils/typeconversions.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/documentTypeUtils/typeconversions.scala new file mode 100644 index 0000000000000..370fc84a3aa08 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/documentTypeUtils/typeconversions.scala @@ -0,0 +1,720 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.documentTypeUtils + +import java.nio.ByteBuffer +import org.ojai.Value +import org.ojai.types._ + +sealed trait convert[A,B] { + def get(value: A): B +} + +object conversions { + //string to other datatypes conversions + val string2intconversion = new convert[Value, Int] { + def get(value: Value) : Int = value.getString.toInt + } + val string2shortconversion = new convert[Value, Short] { + def get(value: Value): Short = value.getString.toShort + } + val string2binaryconversion = new convert[Value, ByteBuffer] { + def get(value: Value): ByteBuffer = throw new Exception("string cannot be converted to binary") + } + val string2booleanconversion = new convert[Value, Boolean] { + def get(value: Value): Boolean = value.getString.toBoolean + } + val string2arrayconversion = null + val string2mapconversion = null + val string2byteconversion = new convert[Value, Byte] { + def get(value: Value): Byte = value.getString.toByte + } + val string2dateconversion = new convert[Value, ODate]{ + def get(value: Value): ODate = ODate.parse(value.getString) + } + val string2decimalconversion = new convert[Value, BigDecimal] { + def get(value: Value): BigDecimal = new java.math.BigDecimal(value.getString) + } + val string2doubleconversion = new convert[Value, Double] { + def get(value: Value): Double = value.getString.toDouble + } + val string2floatconversion = new convert[Value, Float] { + def get(value: Value): Float = value.getString.toFloat + } + val string2intervalconversion = null + val string2Longconversion = new convert[Value, Long]{ + def get(value: Value): Long = value.getString.toLong + } + val string2Stringconversion = new convert[Value, String] { + def get(value: Value): String = value.getString + } + val string2Timeconversion = new convert[Value,OTime] { + def get(value: Value): OTime = OTime.parse(value.getString) + } + val string2Timestampconversion = new convert[Value,OTimestamp] { + def get(value: Value): OTimestamp = OTimestamp.parse(value.getString) + } + + //boolean to other data types conversions + val boolean2intconversion = new convert[Value, Int] { + def get(value: Value) : Int = value.getBoolean match { case true => 1 + case false => 0 } + } + val boolean2shortconversion = new convert[Value, Short] { + def get(value: Value): Short = value.getBoolean match { case true => 1 + case false => 0} + } + val boolean2binaryconversion = null + val boolean2booleanconversion = new convert[Value, Boolean] { + def get(value: Value): Boolean = value.getBoolean + } + val boolean2arrayconversion = null + val boolean2mapconversion = null + val boolean2byteconversion = new convert[Value, Byte] { + def get(value: Value): Byte = value.getBoolean match { case true => 1 + case false => 0} + } + val boolean2dateconversion = null + val boolean2decimalconversion = null + val boolean2doubleconversion = null + val boolean2floatconversion = null + val boolean2intervalconversion = null + val boolean2Longconversion = new convert[Value, Long]{ + def get(value: Value): Long = value.getBoolean match { case true => 1 + case false => 0} + } + val boolean2Stringconversion = new convert[Value, String] { + def get(value: Value): String = value.getBoolean.toString + } + val boolean2Timeconversion = null + val boolean2Timestampconversion = null + + //short to other data types conversions + val short2intconversion = new convert[Value, Int] { + def get(value: Value) : Int = value.getShort.toInt + } + val short2shortconversion = new convert[Value, Short] { + def get(value: Value): Short = value.getShort + } + val short2binaryconversion = null + val short2booleanconversion = null + val short2arrayconversion = null + val short2mapconversion = null + val short2byteconversion = new convert[Value, Byte] { + def get(value: Value): Byte = value.getShort.toByte + } + val short2dateconversion = new convert[Value, ODate]{ + def get(value: Value): ODate = ODate.fromDaysSinceEpoch(value.getShort) + } + val short2decimalconversion = new convert[Value, BigDecimal] { + def get(value: Value): BigDecimal = BigDecimal(value.getShort.toFloat) + } + val short2doubleconversion = new convert[Value, Double] { + def get(value: Value): Double = value.getShort.toDouble + } + val short2floatconversion = new convert[Value, Float] { + def get(value: Value): Float = value.getShort.toFloat + } + val short2intervalconversion = new convert[Value, OInterval]{ + def get(value: Value): OInterval = new OInterval(value.getShort) + } + val short2Longconversion = new convert[Value, Long]{ + def get(value: Value): Long = value.getShort + } + val short2Stringconversion = new convert[Value, String] { + def get(value: Value): String = value.getShort.toString + } + val short2Timeconversion = new convert[Value,OTime] { + def get(value: Value): OTime = new OTime(value.getShort) + } + val short2Timestampconversion = new convert[Value,OTimestamp] { + def get(value: Value): OTimestamp = new OTimestamp(value.getShort) + } + + //int to other datatype conversions + val int2intconversion = new convert[Value, Int] { + def get(value: Value) : Int = value.getInt + } + val int2shortconversion = new convert[Value, Short] { + def get(value: Value): Short = value.getInt.toShort + } + val int2binaryconversion = null + val int2booleanconversion = null + val int2arrayconversion = null + val int2mapconversion = null + val int2byteconversion = new convert[Value, Byte] { + def get(value: Value): Byte = value.getInt.toByte + } + val int2dateconversion = new convert[Value, ODate]{ + def get(value: Value): ODate = ODate.fromDaysSinceEpoch(value.getInt) + } + val int2decimalconversion = new convert[Value, BigDecimal] { + def get(value: Value): BigDecimal = BigDecimal(value.getInt) + } + val int2doubleconversion = new convert[Value, Double] { + def get(value: Value): Double = value.getInt + } + val int2floatconversion = new convert[Value, Float] { + def get(value: Value): Float = value.getInt + } + val int2intervalconversion = new convert[Value, OInterval]{ + def get(value: Value): OInterval = new OInterval(value.getInt) + } + val int2Longconversion = new convert[Value, Long]{ + def get(value: Value): Long = value.getInt + } + val int2Stringconversion = new convert[Value, String] { + def get(value: Value): String = value.getInt.toString + } + val int2Timeconversion = new convert[Value,OTime] { + def get(value: Value): OTime = OTime.fromMillisOfDay(value.getInt) + } + val int2Timestampconversion = new convert[Value,OTimestamp] { + def get(value: Value): OTimestamp = new OTimestamp(value.getInt) + } + + + //array to other datatype conversions + val array2intconversion = null + val array2shortconversion = null + val array2binaryconversion = null + val array2booleanconversion = null + val array2arrayconversion = null + val array2mapconversion = null + val array2byteconversion = null + val array2dateconversion = null + val array2decimalconversion = null + val array2doubleconversion = null + val array2floatconversion = null + val array2intervalconversion = null + val array2Longconversion = null + val array2Stringconversion = new convert[Value, String] { + def get(value: Value): String = value.getList.toString + } + val array2Timeconversion = null + val array2Timestampconversion = null + + //map to other datatypes conversions + val map2intconversion = null + val map2shortconversion = null + val map2binaryconversion = null + val map2booleanconversion = null + val map2arrayconversion = null + val map2mapconversion = null + val map2byteconversion = null + val map2dateconversion = null + val map2decimalconversion = null + val map2doubleconversion = null + val map2floatconversion = null + val map2intervalconversion = null + val map2Longconversion = null + val map2Stringconversion = new convert[Value, String] { + def get(value: Value): String = value.getMap.toString + } + val map2Timeconversion = null + val map2Timestampconversion = null + + //binary to other datatypes conversions + val binary2intconversion = null + val binary2shortconversion = null + val binary2binaryconversion = new convert[Value, ByteBuffer] { + def get(value: Value): ByteBuffer = value.getBinary + } + val binary2booleanconversion = null + val binary2arrayconversion = null + val binary2mapconversion = null + val binary2byteconversion = null + val binary2dateconversion = null + val binary2decimalconversion = null + val binary2doubleconversion = null + val binary2floatconversion = null + val binary2intervalconversion = null + val binary2Longconversion = null + val binary2Stringconversion = null + val binary2Timeconversion = null + val binary2Timestampconversion = null + + //byte to other datatypes + val byte2intconversion = new convert[Value, Int] { + def get(value: Value) : Int = value.getByte + } + val byte2shortconversion = new convert[Value, Short] { + def get(value: Value): Short = value.getByte + } + val byte2binaryconversion = null + val byte2booleanconversion = null + val byte2arrayconversion = null + val byte2mapconversion = null + val byte2byteconversion = new convert[Value, Byte] { + def get(value: Value): Byte = value.getByte + } + val byte2dateconversion = new convert[Value, ODate]{ + def get(value: Value): ODate = ODate.fromDaysSinceEpoch(value.getByte) + } + val byte2decimalconversion = new convert[Value, BigDecimal] { + def get(value: Value): BigDecimal = BigDecimal(value.getByte) + } + val byte2doubleconversion = new convert[Value, Double] { + def get(value: Value): Double = value.getByte + } + val byte2floatconversion = new convert[Value, Float] { + def get(value: Value): Float = value.getByte + } + val byte2intervalconversion = new convert[Value, OInterval]{ + def get(value: Value): OInterval = new OInterval(value.getByte) + } + val byte2Longconversion = new convert[Value, Long]{ + def get(value: Value): Long = value.getByte + } + val byte2Stringconversion = new convert[Value, String] { + def get(value: Value): String = value.getByte.toString + } + val byte2Timeconversion = new convert[Value,OTime] { + def get(value: Value): OTime = OTime.fromMillisOfDay(value.getByte) + } + val byte2Timestampconversion = new convert[Value,OTimestamp] { + def get(value: Value): OTimestamp = new OTimestamp(value.getByte) + } + + + //date to other datatype conversions + val date2intconversion = new convert[Value, Int] { + def get(value: Value) : Int = value.getDateAsInt + } + val date2shortconversion = null + val date2binaryconversion = new convert[Value, ByteBuffer] { + def get(value: Value): ByteBuffer = throw new Exception("cannot convert date to binary") + } + val date2booleanconversion = new convert[Value, Boolean] { + def get(value: Value): Boolean = throw new Exception("cannot convert date to boolean") + } + val date2arrayconversion = null + val date2mapconversion = null + val date2byteconversion = new convert[Value, Byte] { + def get(value: Value): Byte = throw new Exception("cannot convert date to byte") + } + val date2dateconversion = new convert[Value, ODate]{ + def get(value: Value): ODate = value.getDate + } + val date2decimalconversion = new convert[Value, BigDecimal] { + def get(value: Value): BigDecimal = BigDecimal(value.getDateAsInt) + } + val date2doubleconversion = new convert[Value, Double] { + def get(value: Value): Double = value.getDateAsInt + } + val date2floatconversion = new convert[Value, Float] { + def get(value: Value): Float = value.getDateAsInt + } + val date2intervalconversion = new convert[Value, OInterval]{ + def get(value: Value): OInterval = throw new Exception("cannot convert date to interval") + } + val date2Longconversion = new convert[Value, Long]{ + def get(value: Value): Long = value.getDateAsInt + } + val date2Stringconversion = new convert[Value, String] { + def get(value: Value): String = value.getDate.toDateStr + } + val date2Timeconversion = new convert[Value,OTime] { + def get(value: Value): OTime = throw new Exception("cannot convert date to time") + } + val date2Timestampconversion = new convert[Value,OTimestamp] { + def get(value: Value): OTimestamp = { + val date: ODate = value.getDate + new OTimestamp(date.getYear, date.getMonth, date.getDayOfMonth, 0, 0, 0, 0) + } + } + + //decimal to other datatype conversions + val decimal2intconversion = new convert[Value, Int] { + def get(value: Value) : Int = throw new Exception("cannot convert decimal to int") + } + val decimal2shortconversion = new convert[Value, Short] { + def get(value: Value): Short = throw new Exception("cannot convert decimal to short") + } + val decimal2binaryconversion = new convert[Value, ByteBuffer] { + def get(value: Value): ByteBuffer = throw new Exception("cannot convert decimal to Binary") + } + val decimal2booleanconversion = new convert[Value, Boolean] { + def get(value: Value): Boolean = throw new Exception("cannot convert decimal to boolean") + } + val decimal2arrayconversion = null + val decimal2mapconversion = null + val decimal2byteconversion = new convert[Value, Byte] { + def get(value: Value): Byte = throw new Exception("cannot convert decimal to byte") + } + val decimal2dateconversion = new convert[Value, ODate]{ + def get(value: Value): ODate = throw new Exception("cannot convert decimal to byte") + } + val decimal2decimalconversion = new convert[Value, BigDecimal] { + def get(value: Value): BigDecimal = value.getDecimal + } + val decimal2doubleconversion = new convert[Value, Double] { + def get(value: Value): Double = throw new Exception("cannot convert decimal to double") + } + val decimal2floatconversion = new convert[Value, Float] { + def get(value: Value): Float = throw new Exception("cannot convert decimal to float") + } + val decimal2intervalconversion = new convert[Value, OInterval]{ + def get(value: Value): OInterval = throw new Exception("cannot convert decimal to interval") + } + val decimal2Longconversion = new convert[Value, Long]{ + def get(value: Value): Long = throw new Exception("cannot convert decimal to long") + } + val decimal2Stringconversion = new convert[Value, String] { + def get(value: Value): String = value.getDecimal.toEngineeringString + } + val decimal2Timeconversion = new convert[Value,OTime] { + def get(value: Value): OTime = throw new Exception("cannot convert decimal to time") + } + val decimal2Timestampconversion = new convert[Value,OTimestamp] { + def get(value: Value): OTimestamp = throw new Exception("cannot convert decimal to timeStamp") + } + + //double to other datatypes conversions + val double2intconversion = new convert[Value, Int] { + def get(value: Value) : Int = value.getDouble.toInt + } + val double2shortconversion = new convert[Value, Short] { + def get(value: Value): Short = throw new Exception("cannot convert double to short") + } + val double2binaryconversion = new convert[Value, ByteBuffer] { + def get(value: Value): ByteBuffer = throw new Exception("cannot convert double to binary") + } + val double2booleanconversion = new convert[Value, Boolean] { + def get(value: Value): Boolean = value.getBoolean + } + val double2arrayconversion = null + val double2mapconversion = null + val double2byteconversion = new convert[Value, Byte] { + def get(value: Value): Byte = throw new Exception("cannot convert double to byte") + } + val double2dateconversion = new convert[Value, ODate]{ + def get(value: Value): ODate = throw new Exception("cannot convert double to date") + } + val double2decimalconversion = new convert[Value, BigDecimal] { + def get(value: Value): BigDecimal = BigDecimal(value.getDouble) + } + val double2doubleconversion = new convert[Value, Double] { + def get(value: Value): Double = value.getDouble + } + val double2floatconversion = new convert[Value, Float] { + def get(value: Value): Float = throw new Exception("cannot convert double to float") + } + val double2intervalconversion = new convert[Value, OInterval]{ + def get(value: Value): OInterval = throw new Exception("cannot convert double to interval") + } + val double2Longconversion = new convert[Value, Long]{ + def get(value: Value): Long = throw new Exception("cannot convert double to long") + } + val double2Stringconversion = new convert[Value, String] { + def get(value: Value): String = value.getDouble.toString + } + val double2Timeconversion = new convert[Value,OTime] { + def get(value: Value): OTime = throw new Exception("cannot convert double to Time") + } + val double2Timestampconversion = new convert[Value,OTimestamp] { + def get(value: Value): OTimestamp = throw new Exception("cannot convert double to timestamp") + } + + //float to other datatypes conversions + val float2intconversion = null + val float2shortconversion = null + val float2binaryconversion = null + val float2booleanconversion = null + val float2arrayconversion = null + val float2mapconversion = null + val float2byteconversion = null + val float2dateconversion = null + val float2decimalconversion = new convert[Value, BigDecimal] { + def get(value: Value): BigDecimal = BigDecimal(value.getFloat) + } + val float2doubleconversion = new convert[Value, Double] { + def get(value: Value): Double = value.getFloat + } + val float2floatconversion = new convert[Value, Float] { + def get(value: Value): Float = value.getFloat + } + val float2intervalconversion = new convert[Value, OInterval]{ + def get(value: Value): OInterval = throw new Exception("cannot convert float to interval") + } + val float2Longconversion = new convert[Value, Long]{ + def get(value: Value): Long = throw new Exception("cannot convert float to long") + } + val float2Stringconversion = new convert[Value, String] { + def get(value: Value): String = value.getFloat.toString + } + val float2Timeconversion = new convert[Value,OTime] { + def get(value: Value): OTime = throw new Exception("cannot convert float to time") + } + val float2Timestampconversion = new convert[Value,OTimestamp] { + def get(value: Value): OTimestamp = throw new Exception("cannot convert float to timestamp") + } + + //interval to other types conversions + val interval2intconversion = new convert[Value, Int] { + def get(value: Value) : Int = value.getIntervalAsLong.toInt + } + val interval2shortconversion = new convert[Value, Short] { + def get(value: Value): Short = value.getIntervalAsLong.toShort + } + val interval2binaryconversion = new convert[Value, ByteBuffer] { + def get(value: Value): ByteBuffer = throw new Exception("cannot convert interval to binary") + } + val interval2booleanconversion = new convert[Value, Boolean] { + def get(value: Value): Boolean = throw new Exception("cannot convert interval to boolean") + } + val interval2arrayconversion = null + val interval2mapconversion = null + val interval2byteconversion = new convert[Value, Byte] { + def get(value: Value): Byte = value.getIntervalAsLong.toByte + } + val interval2dateconversion = new convert[Value, ODate]{ + def get(value: Value): ODate = throw new Exception("cannot convert interval to date") + } + val interval2decimalconversion = new convert[Value, BigDecimal] { + def get(value: Value): BigDecimal = throw new Exception("cannot convert interval to decimal") + } + val interval2doubleconversion = new convert[Value, Double] { + def get(value: Value): Double = throw new Exception("cannot convet interval to double") + } + val interval2floatconversion = new convert[Value, Float] { + def get(value: Value): Float = throw new Exception("cannot convert interval to float") + } + val interval2intervalconversion = new convert[Value, OInterval]{ + def get(value: Value): OInterval = value.getInterval + } + val interval2Longconversion = new convert[Value, Long]{ + def get(value: Value): Long = value.getIntervalAsLong + } + val interval2Stringconversion = new convert[Value, String] { + def get(value: Value): String = value.getInterval.toString + } + val interval2Timeconversion = new convert[Value,OTime] { + def get(value: Value): OTime = throw new Exception("cannot convert interval to time") + } + val interval2Timestampconversion = new convert[Value,OTimestamp] { + def get(value: Value): OTimestamp = throw new Exception("cannot conver interval to timestamp") + } + + //long to other datatypes convesions + val long2intconversion = new convert[Value, Int] { + def get(value: Value) : Int = value.getLong.toInt + } + val long2shortconversion = new convert[Value, Short] { + def get(value: Value): Short = value.getLong.toShort + } + val long2binaryconversion = new convert[Value, ByteBuffer] { + def get(value: Value): ByteBuffer = throw new Exception("cannot convert from long to binary") + } + val long2booleanconversion = new convert[Value, Boolean] { + def get(value: Value): Boolean = value.getBoolean + } + val long2arrayconversion = null + val long2mapconversion = null + val long2byteconversion = new convert[Value, Byte] { + def get(value: Value): Byte = value.getLong.toByte + } + val long2dateconversion = new convert[Value, ODate]{ + def get(value: Value): ODate = ODate.fromDaysSinceEpoch(value.getLong.toInt) + } + val long2decimalconversion = new convert[Value, BigDecimal] { + def get(value: Value): BigDecimal = BigDecimal(value.getLong) + } + val long2doubleconversion = new convert[Value, Double] { + def get(value: Value): Double = value.getLong + } + val long2floatconversion = new convert[Value, Float] { + def get(value: Value): Float = value.getLong + } + val long2intervalconversion = new convert[Value, OInterval]{ + def get(value: Value): OInterval = new OInterval(value.getLong) + } + val long2Longconversion = new convert[Value, Long]{ + def get(value: Value): Long = value.getLong + } + val long2Stringconversion = new convert[Value, String] { + def get(value: Value): String = value.getLong.toString + } + val long2Timeconversion = new convert[Value,OTime] { + def get(value: Value): OTime = new OTime(value.getLong) + } + val long2Timestampconversion = new convert[Value,OTimestamp] { + def get(value: Value): OTimestamp = new OTimestamp(value.getLong) + } + + //time to other datatype conversions + val time2intconversion = new convert[Value, Int] { + def get(value: Value) : Int = value.getTimeAsInt + } + val time2shortconversion = new convert[Value, Short] { + def get(value: Value): Short = value.getTimeAsInt.toShort + } + val time2binaryconversion = new convert[Value, ByteBuffer] { + def get(value: Value): ByteBuffer = throw new Exception("cannot convert time to binary") + } + val time2booleanconversion = new convert[Value, Boolean] { + def get(value: Value): Boolean = throw new Exception("cannot convert time to boolean") + } + val time2arrayconversion = null + val time2mapconversion = null + val time2byteconversion = new convert[Value, Byte] { + def get(value: Value): Byte = throw new Exception("cannot convert time to byte") + } + val time2dateconversion = new convert[Value, ODate]{ + def get(value: Value): ODate = throw new Exception("cannot convert time to date") + } + val time2decimalconversion = new convert[Value, BigDecimal] { + def get(value: Value): BigDecimal = throw new Exception("cannot convert time to decimal") + } + val time2doubleconversion = new convert[Value, Double] { + def get(value: Value): Double = throw new Exception("canot convert time to double") + } + val time2floatconversion = new convert[Value, Float] { + def get(value: Value): Float = throw new Exception("cannot convert time to float") + } + val time2intervalconversion = new convert[Value, OInterval]{ + def get(value: Value): OInterval = throw new Exception("cannot convert time to interval") + } + val time2Longconversion = new convert[Value, Long]{ + def get(value: Value): Long = value.getTimeAsInt + } + val time2Stringconversion = new convert[Value, String] { + def get(value: Value): String = value.getTime.toFullTimeStr + } + val time2Timeconversion = new convert[Value,OTime] { + def get(value: Value): OTime = value.getTime + } + val time2Timestampconversion = new convert[Value,OTimestamp] { + def get(value: Value): OTimestamp = throw new Exception("cannot conver time to timestamp") + } + + //timestamp to other datatypes conversions + val timestamp2intconversion = new convert[Value, Int] { + def get(value: Value) : Int = value.getTimestampAsLong.toInt + } + val timestamp2shortconversion = new convert[Value, Short] { + def get(value: Value): Short = value.getTimestampAsLong.toShort + } + val timestamp2binaryconversion = new convert[Value, ByteBuffer] { + def get(value: Value): ByteBuffer = throw new Exception("cannot convert timetstamp to binary") + } + val timestamp2booleanconversion = new convert[Value, Boolean] { + def get(value: Value): Boolean = throw new Exception("cannot convert timestamp to boolean") + } + val timestamp2arrayconversion = null + val timestamp2mapconversion = null + val timestamp2byteconversion = new convert[Value, Byte] { + def get(value: Value): Byte = throw new Exception("cannot convert timestamp to byte") + } + val timestamp2dateconversion = new convert[Value, ODate]{ + def get(value: Value): ODate = null + } + val timestamp2decimalconversion = new convert[Value, BigDecimal] { + def get(value: Value): BigDecimal = null + } + val timestamp2doubleconversion = new convert[Value, Double] { + def get(value: Value): Double = 0 + } + val timestamp2floatconversion = new convert[Value, Float] { + def get(value: Value): Float = 0 + } + val timestamp2intervalconversion = new convert[Value, OInterval]{ + def get(value: Value): OInterval = null + } + val timestamp2Longconversion = new convert[Value, Long]{ + def get(value: Value): Long = 0 + } + val timestamp2Stringconversion = new convert[Value, String] { + def get(value: Value): String = null + } + val timestamp2Timeconversion = new convert[Value,OTime] { + def get(value: Value): OTime = null + } + val timestamp2Timestampconversion = new convert[Value,OTimestamp] { + def get(value: Value): OTimestamp = value.getTimestamp + } +} + +object typeconversions { + def convert[B](value: Value, from: Value.Type, to: Value.Type): B = { + val func = Option(typeconversions.conversionFunctions(from.getCode-1)(to.getCode-1)) + func match { + case Some(a) => a.get(value).asInstanceOf[B] + case None => throw new Exception("incompatible types") + } + } + + /* + * This section deals with type casting of data whenever it is possible. + * If a field in the table has different type for the data across different documents + * Consider field "a.c" has integer type data but it is specified as int in some documents and string in some documents. + * If the user specifies all the data to be represented as int then, convert the string data to int whenever it is possible + * otherwise throw an exception. + */ + val conversionFunctions = Array(Array(conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion,conversions.interval2Timeconversion), + Array(conversions.interval2Timeconversion,conversions.boolean2booleanconversion ,conversions.boolean2Stringconversion ,conversions.boolean2byteconversion , + conversions.boolean2shortconversion, conversions.boolean2intconversion,conversions.boolean2Longconversion,conversions.boolean2floatconversion,conversions + .boolean2doubleconversion, + conversions.boolean2decimalconversion, conversions.boolean2dateconversion,conversions.boolean2Timeconversion,conversions.boolean2Timestampconversion,conversions + .boolean2intervalconversion, conversions.boolean2binaryconversion, conversions.boolean2mapconversion,conversions.boolean2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.string2booleanconversion ,conversions.string2Stringconversion ,conversions.string2byteconversion , + conversions.string2shortconversion, conversions.string2intconversion,conversions.string2Longconversion,conversions.string2floatconversion,conversions.string2doubleconversion, + conversions.string2decimalconversion, conversions.string2dateconversion,conversions.string2Timeconversion,conversions.string2Timestampconversion,conversions + .string2intervalconversion, conversions.string2binaryconversion, conversions.string2mapconversion,conversions.string2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.byte2booleanconversion ,conversions.byte2Stringconversion ,conversions.byte2byteconversion , + conversions.byte2shortconversion, conversions.byte2intconversion,conversions.byte2Longconversion,conversions.byte2floatconversion,conversions.byte2doubleconversion, + conversions.byte2decimalconversion, conversions.byte2dateconversion,conversions.byte2Timeconversion,conversions.byte2Timestampconversion,conversions + .byte2intervalconversion, conversions.byte2binaryconversion, conversions.byte2mapconversion,conversions.byte2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.short2booleanconversion ,conversions.short2Stringconversion ,conversions.short2byteconversion , + conversions.short2shortconversion, conversions.short2intconversion,conversions.short2Longconversion,conversions.short2floatconversion,conversions.short2doubleconversion, + conversions.short2decimalconversion, conversions.short2dateconversion,conversions.short2Timeconversion,conversions.short2Timestampconversion,conversions + .short2intervalconversion, conversions.short2binaryconversion, conversions.short2mapconversion,conversions.short2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.int2booleanconversion ,conversions.int2Stringconversion ,conversions.int2byteconversion , + conversions.int2shortconversion, conversions.int2intconversion,conversions.int2Longconversion,conversions.int2floatconversion,conversions.int2doubleconversion, + conversions.int2decimalconversion, conversions.int2dateconversion,conversions.int2Timeconversion,conversions.int2Timestampconversion,conversions + .int2intervalconversion, conversions.int2binaryconversion, conversions.int2mapconversion,conversions.int2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.long2booleanconversion ,conversions.long2Stringconversion ,conversions.long2byteconversion , + conversions.long2shortconversion, conversions.long2intconversion,conversions.long2Longconversion,conversions.long2floatconversion,conversions.long2doubleconversion, + conversions.long2decimalconversion, conversions.long2dateconversion,conversions.long2Timeconversion,conversions.long2Timestampconversion,conversions + .long2intervalconversion, conversions.long2binaryconversion, conversions.long2mapconversion,conversions.long2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.float2booleanconversion ,conversions.float2Stringconversion ,conversions.float2byteconversion , + conversions.float2shortconversion, conversions.float2intconversion,conversions.float2Longconversion,conversions.float2floatconversion,conversions.float2doubleconversion, + conversions.float2decimalconversion, conversions.float2dateconversion,conversions.float2Timeconversion,conversions.float2Timestampconversion,conversions + .float2intervalconversion, conversions.float2binaryconversion, conversions.float2mapconversion,conversions.float2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.double2booleanconversion ,conversions.double2Stringconversion ,conversions.double2byteconversion , + conversions.double2shortconversion, conversions.double2intconversion,conversions.double2Longconversion,conversions.double2floatconversion,conversions.double2doubleconversion, + conversions.double2decimalconversion, conversions.double2dateconversion,conversions.double2Timeconversion,conversions.double2Timestampconversion,conversions + .double2intervalconversion, conversions.double2binaryconversion, conversions.double2mapconversion,conversions.double2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.decimal2booleanconversion ,conversions.decimal2Stringconversion ,conversions.decimal2byteconversion , + conversions.decimal2shortconversion, conversions.decimal2intconversion,conversions.decimal2Longconversion,conversions.decimal2floatconversion,conversions.decimal2doubleconversion, + conversions.decimal2decimalconversion, conversions.decimal2dateconversion,conversions.decimal2Timeconversion,conversions.decimal2Timestampconversion,conversions + .decimal2intervalconversion, conversions.decimal2binaryconversion, conversions.decimal2mapconversion,conversions.decimal2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.date2booleanconversion ,conversions.date2Stringconversion ,conversions.date2byteconversion , + conversions.date2shortconversion, conversions.date2intconversion,conversions.date2Longconversion,conversions.date2floatconversion,conversions.date2doubleconversion, + conversions.date2decimalconversion, conversions.date2dateconversion,conversions.date2Timeconversion,conversions.date2Timestampconversion,conversions + .date2intervalconversion, conversions.date2binaryconversion, conversions.date2mapconversion,conversions.date2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.time2booleanconversion ,conversions.time2Stringconversion ,conversions.time2byteconversion , + conversions.time2shortconversion, conversions.time2intconversion,conversions.time2Longconversion,conversions.time2floatconversion,conversions.time2doubleconversion, + conversions.time2decimalconversion, conversions.time2dateconversion,conversions.time2Timeconversion,conversions.time2Timestampconversion,conversions + .time2intervalconversion, conversions.time2binaryconversion, conversions.time2mapconversion,conversions.time2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.timestamp2booleanconversion ,conversions.timestamp2Stringconversion ,conversions.timestamp2byteconversion , + conversions.timestamp2shortconversion, conversions.timestamp2intconversion,conversions.timestamp2Longconversion,conversions.timestamp2floatconversion,conversions.timestamp2doubleconversion, + conversions.timestamp2decimalconversion, conversions.timestamp2dateconversion,conversions.timestamp2Timeconversion,conversions.timestamp2Timestampconversion,conversions + .timestamp2intervalconversion, conversions.timestamp2binaryconversion, conversions.timestamp2mapconversion,conversions.timestamp2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.interval2booleanconversion ,conversions.interval2Stringconversion ,conversions.interval2byteconversion , + conversions.interval2shortconversion, conversions.interval2intconversion,conversions.interval2Longconversion,conversions.interval2floatconversion,conversions.interval2doubleconversion, + conversions.interval2decimalconversion, conversions.interval2dateconversion,conversions.interval2Timeconversion,conversions.interval2Timestampconversion,conversions + .interval2intervalconversion, conversions.interval2binaryconversion, conversions.interval2mapconversion,conversions.interval2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.binary2booleanconversion ,conversions.binary2Stringconversion ,conversions.binary2byteconversion , + conversions.binary2shortconversion, conversions.binary2intconversion,conversions.binary2Longconversion,conversions.binary2floatconversion,conversions.binary2doubleconversion, + conversions.binary2decimalconversion, conversions.binary2dateconversion,conversions.binary2Timeconversion,conversions.binary2Timestampconversion,conversions + .binary2intervalconversion, conversions.binary2binaryconversion, conversions.binary2mapconversion,conversions.binary2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.map2booleanconversion ,conversions.map2Stringconversion ,conversions.map2byteconversion , + conversions.map2shortconversion, conversions.map2intconversion,conversions.map2Longconversion,conversions.map2floatconversion,conversions.map2doubleconversion, + conversions.map2decimalconversion, conversions.map2dateconversion,conversions.map2Timeconversion,conversions.map2Timestampconversion,conversions + .map2intervalconversion, conversions.map2binaryconversion, conversions.map2mapconversion,conversions.map2arrayconversion), + Array(conversions.interval2Timeconversion,conversions.array2booleanconversion ,conversions.array2Stringconversion ,conversions.array2byteconversion , + conversions.array2shortconversion, conversions.array2intconversion,conversions.array2Longconversion,conversions.array2floatconversion,conversions.array2doubleconversion, + conversions.array2decimalconversion, conversions.array2dateconversion,conversions.array2Timeconversion,conversions.array2Timestampconversion,conversions + .array2intervalconversion, conversions.array2binaryconversion, conversions.array2mapconversion,conversions.array2arrayconversion) + )} + diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/documentUtils/ScalaDocumentIterator.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/documentUtils/ScalaDocumentIterator.scala new file mode 100644 index 0000000000000..5b5ca3a1f3c82 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/documentUtils/ScalaDocumentIterator.scala @@ -0,0 +1,52 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.documentUtils + +import org.ojai.Value +import java.util.Map + +import com.mapr.db.spark.codec.BeanCodec +import com.mapr.db.spark.impl.OJAIDocument +import org.ojai.Document + +/* + * This class is a bridge between the java Map Iterator and scala Map Iterator. + * It enables users to iterate over java Map. + */ +class ScalaDocumentIterator(val iter: java.util.Iterator[Map.Entry[String, Value]]) + extends Iterator[Tuple2[String,Value]] { + + def hasNext = iter.hasNext + + def next() = { + val elem = iter.next() + (elem.getKey -> elem.getValue) + } +} + + +class OJAIDocumentIterator(val iter: java.util.Iterator[Document]) extends Iterator[OJAIDocument] { + + def hasNext = iter.hasNext + + def next() = { + new OJAIDocument(iter.next()) + } +} + +class BeanIterator[T](val iter: java.util.Iterator[Document], val beanClass: Class[T]) extends Iterator[T] { + + def hasNext = iter.hasNext + + def next() = { + BeanCodec.encode[T](iter.next.asReader(), beanClass) + } +} + +case class JavaBeanIterator[T](val iter: java.util.Iterator[Document], val beanClass: Class[T]) extends Iterator[T] { + + def hasNext = iter.hasNext + + def next() = { + org.ojai.beans.BeanCodec.encode[T](iter.next.asReader(), beanClass) + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/exceptions/SchemaMappingException.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/exceptions/SchemaMappingException.scala new file mode 100644 index 0000000000000..cee035e8efa1a --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/exceptions/SchemaMappingException.scala @@ -0,0 +1,12 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.exceptions + +import com.mapr.db.exceptions.DBException + +class SchemaMappingException(message: String, throwable: Throwable) + extends DBException(message, throwable){ + + def this(message: String) { + this(message, null) + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/impl/OJAIDocument.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/impl/OJAIDocument.scala new file mode 100644 index 0000000000000..31ee741aaf8cb --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/impl/OJAIDocument.scala @@ -0,0 +1,93 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.impl + +import java.nio.ByteBuffer + +import com.mapr.db.spark.documentTypeUtils.OJAIType +import com.mapr.db.spark.types.{DBArrayValue, DBBinaryValue, DBMapValue} +import com.mapr.db.spark.utils.DefaultClass.DefaultType +import org.ojai.types._ + +import scala.collection.JavaConverters._ +import scala.language.experimental.macros +import scala.language.{dynamics, implicitConversions} + +/** + * OJAIDocument represents a JSON document which can be accessed with dynamic object model as well as through setters and getters. + * OJAIDocument extends the original org.ojai.Document with input and output types mapped to that of scala types. + * To create an OJAIDocument use the factory function newDocument in MapRDBSpark or through sparkContext functions loadMapRDBTable. + * @constructor Create a new scala's version of ojai Document from java's version of ojai document. + * @param dc java's version of ojai Document + * @example val doc = MapRDBSpark.newDocument(jsonString) or val docs = sc.loadMapRDBTable(tableName) + * + * Here are the ways to access elements in OJAIDocument + * doc.address -- accesses the element with address as the key and returns the value specified for it as AnyRef + * doc.address[String] -- access the elements with address as the key and returns the value specified for it as String + * doc.getString("address") -- getString is an accessor method that can also be used to get the value for address as String. + * + * Here are the ways to set value to a key in OJAIDocument + * doc.city = "San Francisco" or doc.set("city", "San Francisco") + */ +class OJAIDocument(@transient private var dc: org.ojai.Document) + extends ScalaOjaiDocument[OJAIDocument](dc) with Dynamic{ + + override def THIS: OJAIDocument = this + + private[spark] def this() { + this(null) + } + //Getter functionality of the dynamic object model is provided by the following function. + def selectDynamic[T](fieldPath: String)(implicit e: T DefaultType AnyRef, f : OJAIType[T]): f.Self = { + f.getValue(this.getDoc, fieldPath) + } + + //Setter functionality of the dynamic object model is provided by the following function. + def updateDynamic[T <: Any](fieldPath: String)(v: T) : Unit = { + if (v == null) { + getDoc.setNull(fieldPath) + return + } + if (v.isInstanceOf[String]) + getDoc.set(fieldPath, v.asInstanceOf[String]) + else if (v.isInstanceOf[ODate]) + getDoc.set(fieldPath, v.asInstanceOf[ODate]) + else if (v.isInstanceOf[Integer]) + getDoc.set(fieldPath, v.asInstanceOf[Integer]) + else if (v.isInstanceOf[Byte]) + getDoc.set(fieldPath, v.asInstanceOf[Byte]) + else if (v.isInstanceOf[Short]) + getDoc.set(fieldPath, v.asInstanceOf[Short]) + else if (v.isInstanceOf[Boolean]) + getDoc.set(fieldPath, v.asInstanceOf[Boolean]) + else if (v.isInstanceOf[BigDecimal]) + getDoc.set(fieldPath, v.asInstanceOf[BigDecimal].bigDecimal) + else if (v.isInstanceOf[Double]) + getDoc.set(fieldPath, v.asInstanceOf[Double]) + else if (v.isInstanceOf[Float]) + getDoc.set(fieldPath, v.asInstanceOf[Float]) + else if (v.isInstanceOf[OInterval]) + getDoc.set(fieldPath, v.asInstanceOf[OInterval]) + else if (v.isInstanceOf[Long]) + getDoc.set(fieldPath, v.asInstanceOf[Long]) + else if (v.isInstanceOf[OTime]) + getDoc.set(fieldPath, v.asInstanceOf[OTime]) + else if (v.isInstanceOf[OTimestamp]) + getDoc.set(fieldPath, v.asInstanceOf[OTimestamp]) + else if (v.isInstanceOf[Map[_, _]]) + getDoc.set(fieldPath, v.asInstanceOf[Map[String,AnyRef]].asJava) + else if (v.isInstanceOf[Seq[Any]]) + getDoc.set(fieldPath, v.asInstanceOf[Seq[AnyRef]].asJava) + else if (v.isInstanceOf[DBMapValue]) + getDoc.set(fieldPath, v.asInstanceOf[DBMapValue].map({ case (k,v) => k -> v.asInstanceOf[AnyRef]}).asJava) + else if (v.isInstanceOf[DBArrayValue[_]]) + getDoc.set(fieldPath, v.asInstanceOf[DBArrayValue[AnyRef]].arr.asJava) + else if (v.isInstanceOf[DBBinaryValue]) + getDoc.set(fieldPath, v.asInstanceOf[DBBinaryValue].getByteBuffer()) + else if (v.isInstanceOf[ByteBuffer]) + getDoc.set(fieldPath, v.asInstanceOf[ByteBuffer]) + else if (v.isInstanceOf[Array[Byte]]) + getDoc.set(fieldPath, v.asInstanceOf[Array[Byte]]) + else + throw new RuntimeException("Not valid value to set") + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/impl/ScalaOjaiDocument.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/impl/ScalaOjaiDocument.scala new file mode 100644 index 0000000000000..82ac3d04f19a4 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/impl/ScalaOjaiDocument.scala @@ -0,0 +1,662 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.impl + +import java.io.{Externalizable, ObjectInput, ObjectOutput} +import java.nio.{ByteBuffer, ByteOrder} + +import com.mapr.db.impl.IdCodec +import com.mapr.db.rowcol.RowcolCodec +import com.mapr.db.spark.documentUtils.ScalaDocumentIterator +import com.mapr.db.spark.types.{DBArrayValue, DBBinaryValue, DBMapValue} +import com.mapr.db.spark.utils.{LoggingTrait, MapRDBUtils} +import com.mapr.db.util.ByteBufs +import org.ojai.{DocumentReader, FieldPath, Value} +import org.ojai.exceptions.DecodingException +import org.ojai.json.JsonOptions +import org.ojai.types._ + +import scala.collection.JavaConverters._ +import scala.language.implicitConversions + +/** + * This class implements scala's version of OJAIDocument. + * It encapsulates the org.ojai.Document and provides the functionality to + * the caller with relevant scala's types for ojai's java types. + */ +private[spark] abstract class ScalaOjaiDocument[B <: ScalaOjaiDocument[B]](@transient private var dc: org.ojai.Document) + extends org.ojai.scala.Document with Externalizable with LoggingTrait{ + + // constructor required for serialization. + def this() { + this(null) + } + + def THIS: B + + lazy val getDoc = dc + + override def writeExternal(objectOutput: ObjectOutput): Unit = { + val idbuff = IdCodec.encode(dc.getId) + objectOutput.writeInt(idbuff.capacity()) + idbuff.order(ByteOrder.LITTLE_ENDIAN) + objectOutput.write(idbuff.array(),0,idbuff.capacity()) + val buff = RowcolCodec.encode(dc) + buff.order(ByteOrder.LITTLE_ENDIAN) + objectOutput.writeInt(buff.capacity()) + objectOutput.write(buff.array(),0,buff.capacity()) + logDebug("Serializing OJAI Document: bytes written:" + buff.capacity() + " bytes written for ID field: "+ idbuff.capacity()) + } + + override def readExternal(objectinput: ObjectInput): Unit = { + val idbuffersize = objectinput.readInt() + val idbuffer = ByteBufs.allocate(idbuffersize) + MapRDBUtils.readBytes(idbuffer,idbuffersize,objectinput) + val buffersize = objectinput.readInt() + val buff: ByteBuffer = ByteBufs.allocate(buffersize) + MapRDBUtils.readBytes(buff,buffersize,objectinput) + dc = RowcolCodec.decode(buff,idbuffer,false,false,true) + logDebug("Deserializing OJAI Document: bytes read:" + buffersize + " bytes read for ID field: "+ idbuffersize) + } + + override def toString = { + dc.asJsonString() + } + + override def setId(id: Value): B = { + this.dc.setId(id) + THIS + } + + override def getId(): Value = { + this.dc.getId + } + + override def setId(_id: String): B = { + this.dc.setId(_id) + THIS + } + + override def getIdString(): String = { + this.dc.getIdString + } + + override def setId(_id: ByteBuffer): B = { + this.dc.setId(_id) + THIS + } + + def setId(_id: DBBinaryValue): B = { + this.dc.setId(_id.getByteBuffer()) + THIS + } + + override def getIdBinary(): ByteBuffer = { + this.dc.getIdBinary + } + + def getIdBinarySerializable(): DBBinaryValue = { + new DBBinaryValue(this.dc.getIdBinary) + } + + override def isReadOnly(): Boolean = { + this.isReadOnly() + } + + override def size(): Int = { + this.dc.size + } + + @throws(classOf[DecodingException]) + override def toJavaBean[T](beanClass: Class[T]) : T = { + this.dc.toJavaBean(beanClass) + } + + override def empty(): B = { + this.dc.empty + THIS + } + + override def set(fieldPath: String, value: String): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: FieldPath, value: String): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: String, value: Boolean): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: FieldPath, value: Boolean): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: String, value: Byte): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: FieldPath, value: Byte): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: String, value: Short): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: FieldPath, value: Short): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: String, value: Integer): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: FieldPath, value: Integer): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: String, value: Long): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: FieldPath, value: Long): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: String, value: Float): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: FieldPath, value: Float): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: String, value: Double): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: FieldPath, value: Double): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: String, value: BigDecimal): B = { + this.dc.set(fieldPath, value.bigDecimal) + THIS + } + + override def set(fieldPath: FieldPath, value: BigDecimal): B = { + this.dc.set(fieldPath, value.bigDecimal) + THIS + } + + override def set(fieldPath: String, value: OTime): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: FieldPath, value: OTime): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: String, value: ODate): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: FieldPath, value: ODate): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: String, value: OTimestamp): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: FieldPath, value: OTimestamp): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: String, value: OInterval): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: FieldPath, value: OInterval): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: String, value: Seq[Byte],off: Integer,len: Integer): B = { + this.dc.set(fieldPath, value.asJava) + THIS + } + + override def set(fieldPath: FieldPath, value: Seq[Byte], off: Integer, len: Integer): B = { + this.dc.set(fieldPath, value.asJava) + THIS + } + + override def set(fieldPath: String, value: ByteBuffer): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: FieldPath, value:ByteBuffer): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: String, value: Map[String, _ <: AnyRef]): B = { + this.dc.set(fieldPath, value.asJava) + THIS + } + + override def set(fieldPath: FieldPath, value: Map[String, _<: AnyRef]): B = { + this.dc.set(fieldPath, value.asJava) + THIS + } + + override def set(fieldPath: String, value: org.ojai.scala.Document): B = { + this.dc.set(fieldPath, value.asInstanceOf[ScalaOjaiDocument[_]].dc) + THIS + } + + override def set(fieldPath: FieldPath, value: org.ojai.scala.Document): B = { + this.dc.set(fieldPath, value.asInstanceOf[ScalaOjaiDocument[_]].dc) + THIS + } + + override def set(fieldPath: String, value: Value): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: FieldPath, value: Value): B = { + this.dc.set(fieldPath, value) + THIS + } + + override def set(fieldPath: String, value: Seq[_ <: AnyRef]): B = { + this.dc.set(fieldPath, value.asJava) + THIS + } + + override def set(fieldPath: FieldPath, value: Seq[_ <: AnyRef]): B = { + this.dc.set(fieldPath, value.asJava) + THIS + } + + override def setArray(fieldPath: String, values: AnyRef*): B = { + this.dc.setArray(fieldPath, values:_*) + THIS + } + + override def setArray(fieldPath: FieldPath, values: AnyRef*): B = { + this.dc.setArray(fieldPath, values:_*) + THIS + } + + def setArray(fieldPath: String, values: Array[Boolean]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: FieldPath, values: Array[Boolean]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: String, values: Array[Byte]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: FieldPath, values: Array[Byte]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: String, values: Array[Short]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: FieldPath, values: Array[Short]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: String, values: Array[Int]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: FieldPath, values: Array[Int]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: String, values:Array[Long]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: FieldPath, values: Array[Long]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: String, values: Array[Float]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: FieldPath, values: Array[Float]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: String, values: Array[Double]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: FieldPath, values: Array[Double]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: String, values: Array[String]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + def setArray(fieldPath: FieldPath, values: Array[String]): B = { + this.dc.setArray(fieldPath, values) + THIS + } + + override def setNull(fieldPath: String): B = { + this.dc.setNull(fieldPath) + THIS + } + + override def setNull(fieldPath: FieldPath): B = { + this.dc.setNull(fieldPath) + THIS + } + + override def delete(fieldPath: String): B = { + this.dc.delete(fieldPath) + THIS + } + + override def delete(fieldPath: FieldPath): B = { + this.dc.delete(fieldPath) + THIS + } + + override def getString(fieldPath: String): String = { + this.dc.getString(fieldPath) + } + + override def getString(fieldPath: FieldPath): String = { + this.dc.getString(fieldPath) + } + + override def getBoolean(fieldPath: String): Boolean = { + this.dc.getBoolean(fieldPath) + } + + override def getBoolean(fieldPath: FieldPath): Boolean = { + this.dc.getBoolean(fieldPath) + } + + override def getBooleanObj(fieldPath: String): java.lang.Boolean = { + this.dc.getBooleanObj(fieldPath) + } + + override def getBooleanObj(fieldPath: FieldPath): java.lang.Boolean = { + this.dc.getBooleanObj(fieldPath) + } + + override def getByte(fieldPath: String): Byte = { + this.dc.getByte(fieldPath) + } + + override def getByte(fieldPath: FieldPath): Byte = { + this.dc.getByte(fieldPath) + } + + override def getByteObj(fieldPath: String): java.lang.Byte = { + this.dc.getByteObj(fieldPath) + } + + override def getByteObj(fieldPath: FieldPath): java.lang.Byte = { + this.dc.getByteObj(fieldPath) + } + + override def getShort(fieldPath: String): Short = { + this.dc.getShort(fieldPath) + } + + override def getShort(fieldPath: FieldPath): Short = { + this.dc.getShort(fieldPath) + } + + override def getShortObj(fieldPath: String): java.lang.Short = { + this.dc.getShortObj(fieldPath) + } + + override def getShortObj(fieldPath: FieldPath): java.lang.Short = { + this.dc.getShortObj(fieldPath) + } + + override def getInt(fieldPath: String): Int = { + this.dc.getInt(fieldPath) + } + + override def getInt(fieldPath: FieldPath): Int = { + this.dc.getInt(fieldPath) + } + + override def getIntObj(fieldPath: String): Integer = { + this.dc.getIntObj(fieldPath) + } + + override def getIntObj(fieldPath: FieldPath): Integer = { + this.dc.getIntObj(fieldPath) + } + + override def getLong(fieldPath: String): Long = { + this.dc.getLong(fieldPath) + } + + override def getLong(fieldPath: FieldPath): Long = { + this.dc.getLong(fieldPath) + } + + override def getLongObj(fieldPath: String):java.lang.Long = { + this.dc.getLongObj(fieldPath) + } + + override def getLongObj(fieldPath: FieldPath): java.lang.Long = { + this.dc.getLongObj(fieldPath) + } + + override def getFloat(fieldPath: String): Float = { + this.dc.getFloat(fieldPath) + } + + override def getFloat(fieldPath: FieldPath): Float = { + this.dc.getFloat(fieldPath) + } + + override def getFloatObj(fieldPath: String): java.lang.Float = { + this.dc.getFloatObj(fieldPath) + } + + override def getFloatObj(fieldPath: FieldPath): java.lang.Float = { + this.dc.getFloatObj(fieldPath) + } + + override def getDouble(fieldPath: String): Double = { + this.dc.getDouble(fieldPath) + } + + override def getDouble(fieldPath: FieldPath): Double = { + this.dc.getDouble(fieldPath) + } + + override def getDoubleObj(fieldPath: String): java.lang.Double = { + this.dc.getDoubleObj(fieldPath) + } + + override def getDoubleObj(fieldPath: FieldPath): java.lang.Double = { + this.dc.getDoubleObj(fieldPath) + } + + override def getDecimal(fieldPath: String): BigDecimal = { + BigDecimal.javaBigDecimal2bigDecimal(this.dc.getDecimal(fieldPath)) + } + + override def getDecimal(fieldPath: FieldPath): BigDecimal = { + BigDecimal.javaBigDecimal2bigDecimal(this.dc.getDecimal(fieldPath)) + } + + override def getTime(fieldPath: String): OTime = { + this.dc.getTime(fieldPath) + } + + override def getTime(fieldPath: FieldPath): OTime = { + this.dc.getTime(fieldPath) + } + + override def getDate(fieldPath: String): ODate = { + this.dc.getDate(fieldPath) + } + + override def getDate(fieldPath: FieldPath): ODate = { + this.dc.getDate(fieldPath) + } + + override def getTimestamp(fieldPath: String): OTimestamp = { + this.dc.getTimestamp(fieldPath) + } + + override def getTimestamp(fieldPath: FieldPath): OTimestamp = { + this.dc.getTimestamp(fieldPath) + } + + override def getBinary(fieldPath: String): ByteBuffer = { + this.dc.getBinary(fieldPath) + } + + def getBinarySerializable(fieldPath: String): DBBinaryValue = { + new DBBinaryValue(this.dc.getBinary(fieldPath)) + } + + override def getBinary(fieldPath: FieldPath): ByteBuffer = { + this.dc.getBinary(fieldPath) + } + + def getBinarySerializable(fieldPath: FieldPath): DBBinaryValue = { + new DBBinaryValue(this.dc.getBinary(fieldPath)) + } + + override def getInterval(fieldPath: String): OInterval = { + this.dc.getInterval(fieldPath) + } + + override def getInterval(fieldPath: FieldPath): OInterval = { + this.dc.getInterval(fieldPath) + } + + override def getValue(fieldPath: String): Value = { + this.dc.getValue(fieldPath) + } + + override def getValue(fieldPath: FieldPath): Value = { + this.dc.getValue(fieldPath) + } + + override def getMap(fieldPath: String): Map[String, AnyRef] = { + val result : java.util.Map[String, Object] = this.dc.getMap(fieldPath) + if (result == null) + return null + else + new DBMapValue(result.asScala.toMap) + } + + override def getMap(fieldPath: FieldPath): Map[String, AnyRef] = { + val result : java.util.Map[String, Object] = this.dc.getMap(fieldPath) + if (result == null) + return null + else + new DBMapValue(result.asScala.toMap) + } + + override def getList(fieldPath: String): Seq[AnyRef] = { + val result: java.util.List[Object] = this.dc.getList(fieldPath) + if (result == null) + return null + else + new DBArrayValue(result.asScala) + } + + override def getList(fieldPath: FieldPath): Seq[AnyRef] = { + val result: java.util.List[Object] = this.dc.getList(fieldPath) + if (result == null) + return null + else + new DBArrayValue(result.asScala) + } + + override def asJsonString(): String = { + this.dc.asJsonString + } + + override def asJsonString(options: JsonOptions): String = { + this.dc.asJsonString(options) + } + + override def asReader(): DocumentReader = { + this.dc.asReader() + } + + override def asReader(fieldPath: String): DocumentReader = { + this.dc.asReader(fieldPath) + } + + override def asReader(fieldPath: FieldPath): DocumentReader = { + this.dc.asReader(fieldPath) + } + + override def asMap(): Map[String, AnyRef] = { + new DBMapValue(dc.asMap().asScala.toMap) + } + + override def iterator = { + new ScalaDocumentIterator(this.dc.iterator) + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/package.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/package.scala new file mode 100644 index 0000000000000..318aa0c76dea9 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/package.scala @@ -0,0 +1,69 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db + +import java.nio.ByteBuffer + +import com.mapr.db.impl.IdCodec +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD +import com.mapr.db.spark.RDD._ +import com.mapr.db.spark.condition.quotes +import com.mapr.db.spark.types.DBBinaryValue +import com.mapr.db.spark.writers.OJAIKey +import com.mapr.db.spark.writers.OJAIValue +import com.mapr.org.apache.hadoop.hbase.util.Bytes + +package object spark { + + implicit val ojaiStringKeyOrdering = new Ordering[String] { + override def compare(a: String, b: String) = { + a.compareTo(b) + } + } + + implicit val ojaiDBBinaryKeyOrdering = new Ordering[DBBinaryValue] { + override def compare(a: DBBinaryValue, b: DBBinaryValue) = { + val prevBuf = IdCodec.encodeAsBytes(a.getByteBuffer()) + val curBuf = IdCodec.encodeAsBytes(b.getByteBuffer()) + Bytes.compareTo(prevBuf, 0, prevBuf.length, curBuf, 0, curBuf.length) + } + } + + implicit val ojaiByteBufferOrdering = new Ordering[ByteBuffer] { + override def compare(a: ByteBuffer, b: ByteBuffer) = { + val prevBuf = IdCodec.encodeAsBytes(a) + val curBuf = IdCodec.encodeAsBytes(b) + Bytes.compareTo(prevBuf, 0, prevBuf.length, curBuf, 0, curBuf.length) + } + } + + /** + * Spark MapRDB connector specific functions to load json tables as RDD[OJAIDocument] + * @param sc sparkContext + * @example val docs = sc.loadMapRDBTable("tableName") + */ + implicit def toSparkContextFunctions(sc: SparkContext) : SparkContextFunctions = SparkContextFunctions(sc) + + /** + * Spark MapRDB connector specific functions to save either RDD[OJAIDocument] or RDD of anyobject + * @param rdd rdd on which this function is called + * @example docs.saveToMapRDB("tableName") + * It might throw a DecodingException if the RDD or anyObject is not possible to convert to a document. + */ + implicit def toDocumentRDDFunctions[D : OJAIValue](rdd: RDD[D]) = OJAIDocumentRDDFunctions[D](rdd) + + /** + * Spark MapRDB connector specific functions to save either RDD[(String, OJAIDocument)] or RDD[(String, anyobject)] + * @param rdd rdd on which this function is called + * @example docs.saveToMapRDB("tableName") + * It might throw a DecodingException if the RDD or anyObject is not possible to convert to a document. + */ + implicit def toPairedRDDFunctions[K: OJAIKey, V: OJAIValue](rdd: RDD[(K,V)]) = PairedDocumentRDDFunctions[K,V](rdd) + + /** + * Spark MapRDB connector specific functions to join external RDD with a MapRDB table. + * @param rdd rdd on which this function is called + * @example docs.joinWithMapRDB("tableName") + */ + implicit def toFilterRDDFunctions[K : OJAIKey : quotes](rdd : RDD[K]) = FilterRDDFunctions(rdd) +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/serializers/OJAICustomSerializers.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/serializers/OJAICustomSerializers.scala new file mode 100644 index 0000000000000..eb57867299dbc --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/serializers/OJAICustomSerializers.scala @@ -0,0 +1,80 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.serializers + +import java.nio.ByteBuffer + +import com.esotericsoftware.kryo.Kryo +import com.esotericsoftware.kryo.io.{Input, Output} +import com.mapr.db.spark.types.DBBinaryValue +import com.mapr.db.util.ByteBufs +import org.ojai.types.{ODate, OInterval, OTime, OTimestamp} + +import scala.language.implicitConversions + + +class ODateSerializer extends com.esotericsoftware.kryo.Serializer[ODate]{ + override def write(kryo: Kryo, output: Output, a: ODate): Unit = { + output.writeInt(a.toDaysSinceEpoch) + } + override def read(kryo: Kryo, input: Input, t: Class[ODate]): ODate = { + val date = ODate.fromDaysSinceEpoch(input.readInt()) + date + } +} + +class OTimeSerializer extends com.esotericsoftware.kryo.Serializer[OTime]{ + override def write(kryo: Kryo, output: Output, a: OTime): Unit = { + output.writeInt(a.toTimeInMillis) + } + override def read(kryo: Kryo, input: Input, t: Class[OTime]): OTime = { + val time = OTime.fromMillisOfDay(input.readInt()) + time + } +} + +class OTimeStampSerializer extends com.esotericsoftware.kryo.Serializer[OTimestamp]{ + override def write(kryo: Kryo, output: Output, a: OTimestamp): Unit = { + output.writeLong(a.getMillis) + } + override def read(kryo: Kryo, input: Input, t: Class[OTimestamp]): OTimestamp = { + val timestmp = new OTimestamp(input.readLong()) + timestmp + } +} + +class OIntervalSerializer extends com.esotericsoftware.kryo.Serializer[OInterval]{ + override def write(kryo: Kryo, output: Output, a: OInterval): Unit = { + output.writeLong(a.getMilliseconds) + } + override def read(kryo: Kryo, input: Input, t: Class[OInterval]): OInterval = { + val milis = new OInterval(input.readLong()) + milis + } +} + +class DBBinaryValueSerializer extends com.esotericsoftware.kryo.Serializer[DBBinaryValue]{ + override def write(kryo: Kryo, output: Output, a: DBBinaryValue): Unit = { + output.writeInt(a.getByteBuffer().capacity()) + output.write(a.getByteBuffer().array()) + } + override def read(kryo: Kryo, input: Input, t: Class[DBBinaryValue]): DBBinaryValue = { + val length = input.readInt() + val bytearray = new Array[Byte](length) + input.read(bytearray) + new DBBinaryValue(ByteBufs.wrap(bytearray)) + } +} + +class DBByteBufferValueSerializer extends com.esotericsoftware.kryo.Serializer[ByteBuffer]{ + override def write(kryo: Kryo, output: Output, a: ByteBuffer): Unit = { + output.writeInt(a.capacity()) + output.write(a.array()) + } + override def read(kryo: Kryo, input: Input, t: Class[ByteBuffer]): ByteBuffer = { + val length = input.readInt() + val bytearray = new Array[Byte](length) + input.read(bytearray) + ByteBufs.wrap(bytearray) + } +} + diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/serializers/OjaiJavaSerializer.java b/external/maprdb/src/main/scala/com/mapr/db/spark/serializers/OjaiJavaSerializer.java new file mode 100644 index 0000000000000..06231bf00fbc2 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/serializers/OjaiJavaSerializer.java @@ -0,0 +1,46 @@ +package com.mapr.db.spark.serializers; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.KryoException; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.esotericsoftware.kryo.util.ObjectMap; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; + +public class OjaiJavaSerializer extends Serializer { + public OjaiJavaSerializer() { + } + + public void write(Kryo kryo, Output output, Object object) { + try { + ObjectMap ex = kryo.getGraphContext(); + ObjectOutputStream objectStream = (ObjectOutputStream)ex.get(this); + if(objectStream == null) { + objectStream = new ObjectOutputStream(output); + ex.put(this, objectStream); + } + + objectStream.writeObject(object); + objectStream.flush(); + } catch (Exception var6) { + throw new KryoException("Error during Java serialization.", var6); + } + } + + public Object read(Kryo kryo, Input input, Class type) { + try { + ObjectMap ex = kryo.getGraphContext(); + ObjectInputStream objectStream = (ObjectInputStream)ex.get(this); + if(objectStream == null) { + objectStream = new ObjectInputStream(input); + ex.put(this, objectStream); + } + + return objectStream.readObject(); + } catch (Exception var6) { + throw new KryoException("Error during Java deserialization.", var6); + } + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/sql/DefaultSource.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/DefaultSource.scala new file mode 100644 index 0000000000000..da118b3f877ba --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/DefaultSource.scala @@ -0,0 +1,120 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.sql + +import java.nio.ByteBuffer + +import com.mapr.db.exceptions.TableExistsException +import com.mapr.db.impl.ConditionImpl +import com.mapr.db.spark.dbclient.DBClient +import com.mapr.db.spark.utils.MapRSpark +import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.SaveMode._ +import org.ojai.store.QueryCondition + +class DefaultSource extends DataSourceRegister + with RelationProvider + with SchemaRelationProvider + with CreatableRelationProvider { + + override def shortName(): String = "maprdb" + + override def createRelation(sqlContext: SQLContext, + parameters: Map[String, String]) : BaseRelation = { + //val cParser = new ConditionParser(); + val condition : Option[QueryCondition] = //parameters.get("FilterCondition").map(condition => cParser.parseCondition(condition)).orElse( + parameters.get("QueryCondition").map(cond => ConditionImpl.parseFrom(ByteBuffer.wrap(cond.getBytes))) + createMapRDBRelation(sqlContext, + parameters.get("tableName"), None, + parameters.get("sampleSize"), condition, + parameters.get("ColumnProjection"),parameters.get("Operation").getOrElse("InsertOrReplace"), + parameters.get("FailOnConflict").getOrElse("false")) + } + + + override def createRelation(sqlContext: SQLContext, + parameters: Map[String, String], schema: StructType) : BaseRelation = { + //val cParser = new ConditionParser(); + val condition : Option[QueryCondition] = parameters.get("QueryCondition").map(cond => ConditionImpl.parseFrom(ByteBuffer.wrap(cond.getBytes))) + createMapRDBRelation(sqlContext, parameters.get("tableName"), Some(schema), + parameters.get("sampleSize"), condition, + parameters.get("ColumnProjection"),parameters.get("Operation").getOrElse("InsertOrReplace"), + parameters.get("FailOnConflict").getOrElse("false")) + } + + + override def createRelation(sqlContext: SQLContext, mode: SaveMode, + parameters: Map[String, String], data: DataFrame): BaseRelation = { + + require(parameters.get("tableName").isDefined) + //val cParser = new ConditionParser(); + val condition : Option[QueryCondition] = parameters.get("QueryCondition").map(cond => ConditionImpl.parseFrom(ByteBuffer.wrap(cond.getBytes))) + lazy val tableExists = DBClient().tableExists(parameters.get("tableName").get) + lazy val tableName = parameters.get("tableName").get + lazy val createTheTable = if (tableExists) false else true + lazy val bulkMode = parameters.get("bulkMode").getOrElse("false").toBoolean + val operation = parameters.get("Operation").getOrElse("ErrorIfExists") + mode match { + case ErrorIfExists => {} + case _ => throw new UnsupportedOperationException("Any mode operation is not supported for MapRDB Table. Please use Operation option instead") + } + + operation match { + case "Insert" => { + MapRSpark.insert(data, tableName,"_id", createTable = createTheTable, bulkInsert = bulkMode) + } + + case "InsertOrReplace" => { + MapRSpark.save(data, tableName,"_id",createTable = createTheTable, bulkInsert = bulkMode) + } + + case "ErrorIfExists" => { + if (tableExists) throw new TableExistsException("Table: " + tableName + " already Exists") + else MapRSpark.save(data, tableName,"_id",createTable = true, bulkInsert = bulkMode) + } + + case "Overwrite" => { + DBClient().deleteTable(tableName) + MapRSpark.save(data, tableName,"_id",createTable = true, bulkInsert = bulkMode) + } + + case "Update" => { + MapRSpark.update(data, tableName,"_id",createTable = false, bulkInsert = bulkMode) + } + + case _ => throw new UnsupportedOperationException("Not supported operation") } + + createMapRDBRelation(sqlContext, Some(tableName), Some(data.schema), + parameters.get("sampleSize"), condition, + parameters.get("ColumnProjection"), parameters.get("Operation").getOrElse("InsertOrReplace"), + parameters.get("FailOnConflict").getOrElse("false")) + + } + + + private def createMapRDBRelation(sqlContext: SQLContext, + tableName: Option[String], userSchema: Option[StructType], + sampleSize: Option[String], queryCondition: Option[QueryCondition], + colProjection: Option[String], Operation: String, failOnConflict: String): BaseRelation = { + + require(tableName.isDefined) + val columns = colProjection.map(colList => colList.split(",").toSeq.filter(_.size > 0)) + val failureOnConflict = failOnConflict.toBoolean + + val rdd = MapRSpark.builder + .sparkContext(sqlContext.sparkContext) + .sparkSession(sqlContext.sparkSession) + .configuration().setTable(tableName.get) + .setCond(queryCondition) + .setColumnProjection(columns) + .build.toRDD(null) + + val schema: StructType = userSchema match { + case Some(s) => s + case None => GenerateSchema(rdd, sampleSize.map(_.toDouble).getOrElse(GenerateSchema.SAMPLE_SIZE), failureOnConflict) + } + + MapRDBRelation(tableName.get, schema, rdd, Operation)(sqlContext) + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/sql/GenerateSchema.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/GenerateSchema.scala new file mode 100644 index 0000000000000..1913a43e8194a --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/GenerateSchema.scala @@ -0,0 +1,247 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.sql + +import com.mapr.db.spark.RDD.MapRDBBaseRDD +import com.mapr.db.spark.impl.OJAIDocument +import com.mapr.db.spark.utils.MapRSpark +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types.{DataType, DataTypes, StructField, StructType} +import org.ojai.DocumentReader + +import scala.reflect.runtime.universe._ +import scala.Array._ +import java.util.Arrays.sort +import java.util.Comparator + +import com.mapr.db.spark.exceptions.SchemaMappingException +import org.apache.spark.SparkContext +import org.apache.spark.sql.catalyst.{JavaTypeInference, ScalaReflection} +import org.apache.spark.sql.catalyst.analysis.TypeCoercion +import org.apache.spark.sql.types._ + +import scala.annotation.switch + +object GenerateSchema { + + val SAMPLE_SIZE = 1000.toDouble + + private val ByteDecimal = DecimalType(3, 0) + private val ShortDecimal = DecimalType(5, 0) + private val IntDecimal = DecimalType(10, 0) + private val LongDecimal = DecimalType(20, 0) + private val FloatDecimal = DecimalType(14, 7) + private val DoubleDecimal = DecimalType(30, 15) + private val BigIntDecimal = DecimalType(38, 0) + + private def DecimalTypeforType(dataType: DataType): DecimalType = dataType match { + case ByteType => ByteDecimal + case ShortType => ShortDecimal + case IntegerType => IntDecimal + case LongType => LongDecimal + case FloatType => FloatDecimal + case DoubleType => DoubleDecimal + } + + def apply(sc: SparkContext, tableName: String, sampleSize: Double): StructType = { + apply(MapRSpark.load(sc, tableName), sampleSize) + } + + def apply(rdd: MapRDBBaseRDD[OJAIDocument], sampleSize: Double) : StructType = { + apply(rdd,sampleSize, false) + } + + def apply(rdd: MapRDBBaseRDD[OJAIDocument], sampleSize: Double, failureOnConflict: Boolean): StructType = { + + val numOfPartitions = rdd.getNumPartitions + val eachPartitionLimit = java.lang.Math.ceil(sampleSize/ numOfPartitions).toInt + val sampleData: RDD[DocumentReader] = rdd.mapPartitions(iter => iter.take(eachPartitionLimit)).map(doc => doc.asReader()) + + val resultType = sampleData.map(reader => inferTypeForField(failureOnConflict)(reader.next, reader)) + .treeAggregate[DataType](StructType(Seq()))(compatibleType(failureOnConflict), compatibleType(failureOnConflict)) + + canonicalizeType(resultType) match { + case Some(st: StructType) => st + case _ => StructType(Seq()) + } + } + + private def canonicalizeType: DataType => Option[DataType] = { + case arrayType @ ArrayType(elementType, _) => { + for { + canonicalType <- canonicalizeType(elementType) + } yield { + arrayType.copy(canonicalType) + } + } + + case StructType(fields) => { + val canonicalFields = for { + field <- fields + if field.name.nonEmpty + canonicalType <- canonicalizeType(field.dataType) + } yield { + field.copy(dataType = canonicalType) + } + + if (canonicalFields.nonEmpty) { + Some(StructType(canonicalFields)) + } else { + None + } + } + + case other => Some(other) + } + + private val structFieldComparator = new Comparator[StructField] { + override def compare(o1: StructField, o2: StructField): Int = { + o1.name.compare(o2.name) + } + } + + private def inferTypeForField(failureOnConflict: Boolean)(event: DocumentReader.EventType, reader: DocumentReader): DataType = { + (event) match { + case DocumentReader.EventType.NULL => DataTypes.NullType + + case DocumentReader.EventType.START_ARRAY => { + var elementType = DataTypes.NullType + var thisEvent = reader.next + while (thisEvent != DocumentReader.EventType.END_ARRAY) { + elementType = compatibleType(failureOnConflict)(elementType, inferTypeForField(failureOnConflict)(thisEvent, reader)) + thisEvent = reader.next + } + ArrayType(elementType) + } + + case DocumentReader.EventType.START_MAP => { + val builder = Array.newBuilder[StructField] + var thisEvent = reader.next + while (thisEvent != DocumentReader.EventType.END_MAP) { + builder += StructField(reader.getFieldName, inferTypeForField(failureOnConflict)(thisEvent, reader), nullable = true) + thisEvent = reader.next + } + val fields = builder.result() + sort(fields, structFieldComparator) + StructType(fields) + } + + case DocumentReader.EventType.BINARY => DataTypes.BinaryType + + case DocumentReader.EventType.BOOLEAN => DataTypes.BooleanType + + case DocumentReader.EventType.TIMESTAMP => DataTypes.TimestampType + + case DocumentReader.EventType.DOUBLE => DataTypes.DoubleType + + case DocumentReader.EventType.INT => DataTypes.IntegerType + + case DocumentReader.EventType.LONG => DataTypes.LongType + + case DocumentReader.EventType.STRING => DataTypes.StringType + + case DocumentReader.EventType.FLOAT => DataTypes.FloatType + + case DocumentReader.EventType.BYTE => DataTypes.ByteType + + case DocumentReader.EventType.SHORT => DataTypes.ShortType + + case DocumentReader.EventType.DECIMAL => DecimalType(reader.getDecimalPrecision, reader.getDecimalScale) + + case DocumentReader.EventType.DATE => DataTypes.DateType + + case DocumentReader.EventType.TIME => DataTypes.TimestampType + + case DocumentReader.EventType.INTERVAL => DataTypes.CalendarIntervalType + + case _ => throw new RuntimeException(s"Type ${Option(event).toString} cannot be inferred") + } + } + + private def compatibleType(failureOnConflict: Boolean)(dt1: DataType, dt2: DataType): DataType = { + TypeCoercion.findTightestCommonTypeOfTwo(dt1, dt2).getOrElse { + ((dt1, dt2)) match { + case (st1 @ StructType(fields1), st2 @ StructType(fields2)) => { + if (isInvalidType(st1)) return st1 + if (isInvalidType(st2)) return st2 + val newFields = (fields1 ++ fields2).groupBy(field => field.name).map { + case (name, fieldTypes) => try { + val dataType = fieldTypes.view.map(_.dataType).reduce(compatibleType(failureOnConflict)) + StructField(name, dataType, nullable = true) + } catch { + case e: SchemaMappingException => throw new SchemaMappingException(s"Schema cannot be inferred for the column ${name}") + } + } + StructType(newFields.toSeq.sortBy(_.name)) + } + + case (ArrayType(elementType1, containsNull1), ArrayType(elementType2, containsNull2)) => { + ArrayType(compatibleType(failureOnConflict)(elementType1, elementType2), containsNull1 || containsNull2) + } + + case (DoubleType, _: DecimalType) | (_:DecimalType, DoubleType) => DoubleType + + case (t1: DecimalType, t2: DecimalType) => { + val scale = math.max(t1.scale, t2.scale) + val range = math.max(t1.precision - t1.scale, t2.precision - t2.scale) + if (range + scale > 38) { + DoubleType + } else { + DecimalType(range + scale, scale) + } + } + + case (t1: DataType, t2: DecimalType) if(isIntegral(t1)) => compatibleType(failureOnConflict)(DecimalTypeforType(t1), t2) + + case (t1: DecimalType, t2: DataType) if(isIntegral(t2)) => compatibleType(failureOnConflict)(t1, DecimalTypeforType(t2)) + + case (t1: DataType, StringType) if (isIntegral(t1)) => StringType + + case (StringType, t1: DataType) if (isIntegral(t1)) => StringType + + case (BooleanType, StringType) | (StringType, BooleanType) => StringType + + case (t1: DataType, StringType) if (isDateOrTime(t1)) => StringType + + case (StringType, t1: DataType) if (isDateOrTime(t1)) => StringType + + case (_, _) => { if (failureOnConflict) throw new SchemaMappingException(s"Schema cannot be inferred") + StructType(StructField("InvalidType", StringType) :: Nil) + } + + } + } + } + + def reflectSchema[T <: Product: TypeTag](): Option[StructType] = { + typeOf[T] match { + case x if x == typeOf[Nothing] => None + case _ => Some(ScalaReflection.schemaFor[T].dataType.asInstanceOf[StructType]) + } + } + + + def reflectSchema[T](beanClass: Class[T]): StructType = { + JavaTypeInference.inferDataType(beanClass)._1.asInstanceOf[StructType] + } + + def isIntegral(dt: DataType): Boolean = { + if (dt == LongType || + dt == IntegerType || + dt == ByteType || + dt == ShortType || + dt == FloatType || + dt == DoubleType ) return true + return false + } + + def isDateOrTime(dt: DataType): Boolean = { + if (dt == DateType || + dt == TimestampType) return true + else return false + } + + def isInvalidType(st: StructType): Boolean = { + if (st.fieldNames.contains("InvalidType")) return true + else return false + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBDataFrameFunctions.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBDataFrameFunctions.scala new file mode 100644 index 0000000000000..2bf03b027b9aa --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBDataFrameFunctions.scala @@ -0,0 +1,24 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.sql + +import com.mapr.db.spark.condition.Predicate +import com.mapr.db.spark.utils.{LoggingTrait, MapRSpark} +import org.apache.spark.sql.{DataFrame, Row} +import org.ojai.store.DocumentMutation + +private[spark] case class MapRDBDataFrameFunctions(@transient df: DataFrame) extends LoggingTrait { + + def saveToMapRDB(tableName: String, idFieldPath : String = "_id", + createTable: Boolean = false, bulkInsert:Boolean = false): Unit = + MapRSpark.save(df, tableName,idFieldPath,createTable,bulkInsert) + + def insertToMapRDB(tableName: String, idFieldPath : String = "_id", + createTable: Boolean = false, bulkInsert:Boolean = false): Unit = + MapRSpark.insert(df, tableName,idFieldPath,createTable,bulkInsert) + + def updateToMapRDB(tableName: String, mutation: (Row) => DocumentMutation, getID: (Row) => org.ojai.Value) : Unit = + MapRSpark.update(df, tableName, mutation, getID) + + def updateToMapRDB(tableName: String, mutation: (Row) => DocumentMutation, getID: (Row) => org.ojai.Value, condition: Predicate) : Unit = + MapRSpark.update(df, tableName, mutation, getID, condition) +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBDataFrameReaderFunctions.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBDataFrameReaderFunctions.scala new file mode 100644 index 0000000000000..82afbb9b40518 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBDataFrameReaderFunctions.scala @@ -0,0 +1,58 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.sql + +import com.mapr.db.spark.utils.LoggingTrait +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{DataFrame, DataFrameReader} +import scala.reflect.runtime.universe._ + +private[spark] case class MapRDBDataFrameReaderFunctions(@transient dfr: DataFrameReader) extends LoggingTrait { + + def maprdb(): DataFrame = createDataFrame(None, None) + + /** + * Creates a [[DataFrame]] through schema inference via the `T` type, otherwise will sample the collection to + * determine the type. + * + * @tparam T The optional type of the data from MapRDB + * @return DataFrame + */ + def maprdb[T <: Product: TypeTag](): DataFrame = createDataFrame(GenerateSchema.reflectSchema[T](), None) + + def maprdb(tableName : String): DataFrame = createDataFrame(None, Option(Map("tableName" -> tableName))) + + /** + * Creates a [[DataFrame]] through schema inference via the `T` type, otherwise will sample the collection to + * determine the type. + * + * @param configuration any connection read configuration overrides. Overrides the configuration set in [[org.apache.spark.SparkConf]] + * @tparam T The optional type of the data from MapRDB + * @return DataFrame + */ + def maprdb[T <: Product: TypeTag](configuration: Map[String, String]): DataFrame = + createDataFrame(GenerateSchema.reflectSchema[T](), Some(configuration)) + + /** + * Creates a [[DataFrame]] with the set schema + * + * @param schema the schema definition + * @return DataFrame + */ + def maprdb(schema: StructType): DataFrame = createDataFrame(Some(schema), None) + + /** + * Creates a [[DataFrame]] with the set schema + * + * @param schema the schema definition + * @param configuration any custom read configuration + * @return DataFrame + */ + def maprdb(schema: StructType, configuration: Map[String, String]): DataFrame = createDataFrame(Some(schema), Some(configuration)) + + private def createDataFrame(schema: Option[StructType], readConfig: Option[Map[String, String]]): DataFrame = { + val builder = dfr.format("com.mapr.db.spark.sql.DefaultSource") + if (schema.isDefined) builder.schema(schema.get) + if (readConfig.isDefined) builder.options(readConfig.get) + builder.load() + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBDataFrameWriterFunctions.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBDataFrameWriterFunctions.scala new file mode 100644 index 0000000000000..173a46e5abebb --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBDataFrameWriterFunctions.scala @@ -0,0 +1,12 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.sql + +import com.mapr.db.spark.utils.{LoggingTrait, MapRSpark} +import org.apache.spark.sql.DataFrameWriter + +private[spark] case class MapRDBDataFrameWriterFunctions(@transient dfw: DataFrameWriter[_]) extends LoggingTrait { + + def saveToMapRDB(tableName: String, idFieldPath: String = "_id", + bulkInsert : Boolean = false): Unit = MapRSpark.save(dfw, tableName) + +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBRelation.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBRelation.scala new file mode 100644 index 0000000000000..b3763a83a78e2 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/MapRDBRelation.scala @@ -0,0 +1,249 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.sql + +import com.mapr.db.spark.RDD.MapRDBBaseRDD +import com.mapr.db.spark.condition.Predicate +import com.mapr.db.spark.field +import com.mapr.db.spark.impl.OJAIDocument +import com.mapr.db.spark.utils.LoggingTrait +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.sources.IsNotNull +import org.apache.spark.sql.{DataFrame, Row, SQLContext, SaveMode} +import org.apache.spark.sql.sources.{BaseRelation, Filter, InsertableRelation, PrunedFilteredScan} +import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.sources._ +import com.mapr.db.spark.sql.utils._ +import org.ojai.store.QueryCondition +import org.ojai.types.{ODate, OTimestamp} + +import scala.collection.mutable.ArrayBuffer + +private[spark] +case class MapRDBRelation(tableName: String, relationSchema: StructType, + rdd: MapRDBBaseRDD[OJAIDocument], Operation: String)(@transient val sqlContext: SQLContext) + extends BaseRelation with PrunedFilteredScan with InsertableRelation with LoggingTrait { + + lazy val schema = relationSchema + + override def buildScan(requiredColumns: Array[String], + filters: Array[Filter]) : RDD[Row] = { + + var optimizedRdd = rdd + val queryConditions = schema.fields.filter(!_.nullable) + .map(_.name).map(IsNotNull) ++ filters + + if (requiredColumns.nonEmpty || queryConditions.nonEmpty) { + logDebug(s"requiredColumns: ${requiredColumns.mkString(",")}, filters: ${queryConditions.mkString(",")}") + } + + val fields: ArrayBuffer[StructField] = ArrayBuffer.empty + for (elem <- requiredColumns) { fields += schema.fields(schema.fieldIndex(elem))} + + optimizedRdd = if (!requiredColumns.isEmpty) + optimizedRdd.select(requiredColumns:_*) else rdd + + optimizedRdd = if (!queryConditions.isEmpty) + optimizedRdd.where(convertToCondition(filters, false)) else optimizedRdd + + optimizedRdd.map(doc => doc.asReader()).mapPartitions(MapRSqlUtils.documentsToRow(_, StructType(fields), requiredColumns)) + } + + override def insert(data: DataFrame, overwrite: Boolean): Unit = { + + val dfw = data.write.format("com.mapr.db.spark.sql") + .option("tableName", tableName).option("sampleSize", 1000) + + overwrite match { + case true => dfw.mode(SaveMode.Overwrite).save() + case false => dfw.mode(SaveMode.Append).save() + } + } + + private def getPredicate(fieldName: String, operation: QueryCondition.Op , value: Any): Predicate = { + operation match { + case QueryCondition.Op.EQUAL => EqualsTo(fieldName, value) + case QueryCondition.Op.GREATER => Greater(fieldName, value) + case QueryCondition.Op.GREATER_OR_EQUAL => GreaterEqualsTo(fieldName, value) + case QueryCondition.Op.LESS => Lesser(fieldName, value) + case QueryCondition.Op.LESS_OR_EQUAL => LessEqualsTo(fieldName, value) + case QueryCondition.Op.NOT_EQUAL => NotEqualsTo(fieldName, value) + case _ => throw new RuntimeException(s"QueryCondition operation $operation not supported") + } + } + + private def convertToODate(dt: java.sql.Date): ODate = { + new ODate(dt.getDay) + } + + private def convertToOTimeStamp(timeStamp: java.sql.Timestamp): OTimestamp = { + new OTimestamp(timeStamp.getTime) + } + + private def EqualsTo(fieldName: String, value: Any): Predicate = { + if (value.isInstanceOf[Int]) field(fieldName) === value.asInstanceOf[Int] + else if(value.isInstanceOf[Byte]) field(fieldName) === value.asInstanceOf[Byte] + else if(value.isInstanceOf[Short]) field(fieldName) === value.asInstanceOf[Short] + else if(value.isInstanceOf[Long]) field(fieldName) === value.asInstanceOf[Long] + else if(value.isInstanceOf[Float]) field(fieldName) === value.asInstanceOf[Float] + else if(value.isInstanceOf[Double]) field(fieldName) === value.asInstanceOf[Double] + else if(value.isInstanceOf[String]) field(fieldName) === value.asInstanceOf[String] + else if(value.isInstanceOf[BigDecimal]) field(fieldName) === value.asInstanceOf[BigDecimal] + else if(value.isInstanceOf[java.sql.Date]) field(fieldName) === convertToODate(value.asInstanceOf[java.sql.Date]) + else if(value.isInstanceOf[java.sql.Timestamp]) field(fieldName) === convertToOTimeStamp(value.asInstanceOf[java.sql.Timestamp]) + else throw new RuntimeException(s"Cannot convert $value to a MapRDB predicate") + } + + private def NotEqualsTo(fieldName: String, value: Any): Predicate = { + if (value.isInstanceOf[Int]) field(fieldName).!=(value.asInstanceOf[Int]) + else if(value.isInstanceOf[Byte]) field(fieldName).!=(value.asInstanceOf[Byte]) + else if(value.isInstanceOf[Short]) field(fieldName).!=(value.asInstanceOf[Short]) + else if(value.isInstanceOf[Long]) field(fieldName).!=(value.asInstanceOf[Long]) + else if(value.isInstanceOf[Float]) field(fieldName).!=(value.asInstanceOf[Float]) + else if(value.isInstanceOf[Double]) field(fieldName).!=(value.asInstanceOf[Double]) + else if(value.isInstanceOf[String]) field(fieldName).!=(value.asInstanceOf[String]) + else if(value.isInstanceOf[BigDecimal]) field(fieldName).!=(value.asInstanceOf[BigDecimal]) + else if(value.isInstanceOf[java.sql.Date]) field(fieldName).!=(convertToODate(value.asInstanceOf[java.sql.Date])) + else if(value.isInstanceOf[java.sql.Timestamp]) field(fieldName).!=(convertToOTimeStamp(value.asInstanceOf[java.sql.Timestamp])) + else if(value.isInstanceOf[String]) field(fieldName).!=(value.asInstanceOf[String]) + else throw new RuntimeException(s"Cannot convert $value to a MapRDB predicate") + } + + private def Lesser(fieldName: String, value: Any): Predicate = { + if (value.isInstanceOf[Int]) field(fieldName) < value.asInstanceOf[Int] + else if(value.isInstanceOf[Byte]) field(fieldName) < value.asInstanceOf[Byte] + else if(value.isInstanceOf[Short]) field(fieldName) < value.asInstanceOf[Short] + else if(value.isInstanceOf[Long]) field(fieldName) < value.asInstanceOf[Long] + else if(value.isInstanceOf[Float]) field(fieldName) < value.asInstanceOf[Float] + else if(value.isInstanceOf[Double]) field(fieldName) < value.asInstanceOf[Double] + else if(value.isInstanceOf[BigDecimal]) field(fieldName) < value.asInstanceOf[BigDecimal] + else if(value.isInstanceOf[java.sql.Date]) field(fieldName) < convertToODate(value.asInstanceOf[java.sql.Date]) + else if(value.isInstanceOf[java.sql.Timestamp]) field(fieldName) < convertToOTimeStamp(value.asInstanceOf[java.sql.Timestamp]) + else if(value.isInstanceOf[String]) field(fieldName) < value.asInstanceOf[String] + else throw new RuntimeException(s"Cannot convert $value to a MapRDB predicate") + } + + private def LessEqualsTo(fieldName: String, value: Any): Predicate = { + if (value.isInstanceOf[Int]) field(fieldName) <= value.asInstanceOf[Int] + else if(value.isInstanceOf[Byte]) field(fieldName) <= value.asInstanceOf[Byte] + else if(value.isInstanceOf[Short]) field(fieldName) <= value.asInstanceOf[Short] + else if(value.isInstanceOf[Long]) field(fieldName) <= value.asInstanceOf[Long] + else if(value.isInstanceOf[Float]) field(fieldName) <= value.asInstanceOf[Float] + else if(value.isInstanceOf[Double]) field(fieldName) <= value.asInstanceOf[Double] + else if(value.isInstanceOf[BigDecimal]) field(fieldName) <= value.asInstanceOf[BigDecimal] + else if(value.isInstanceOf[java.sql.Date]) field(fieldName) <= convertToODate(value.asInstanceOf[java.sql.Date]) + else if(value.isInstanceOf[java.sql.Timestamp]) field(fieldName) <= convertToOTimeStamp(value.asInstanceOf[java.sql.Timestamp]) + else if(value.isInstanceOf[String]) field(fieldName) <= value.asInstanceOf[String] + else throw new RuntimeException(s"Cannot convert $value to a MapRDB predicate") + } + + private def Greater(fieldName: String, value: Any): Predicate = { + if (value.isInstanceOf[Int]) field(fieldName) > value.asInstanceOf[Int] + else if(value.isInstanceOf[Byte]) field(fieldName) > value.asInstanceOf[Byte] + else if(value.isInstanceOf[Short]) field(fieldName) > value.asInstanceOf[Short] + else if(value.isInstanceOf[Long]) field(fieldName) > value.asInstanceOf[Long] + else if(value.isInstanceOf[Float]) field(fieldName) > value.asInstanceOf[Float] + else if(value.isInstanceOf[Double]) field(fieldName) > value.asInstanceOf[Double] + else if(value.isInstanceOf[BigDecimal]) field(fieldName) > value.asInstanceOf[BigDecimal] + else if(value.isInstanceOf[java.sql.Date]) field(fieldName) > convertToODate(value.asInstanceOf[java.sql.Date]) + else if(value.isInstanceOf[java.sql.Timestamp]) field(fieldName) > convertToOTimeStamp(value.asInstanceOf[java.sql.Timestamp]) + else if(value.isInstanceOf[String]) field(fieldName) > value.asInstanceOf[String] + else throw new RuntimeException(s"Cannot convert $value to a MapRDB predicate") + } + + private def GreaterEqualsTo(fieldName: String, value: Any): Predicate = { + if (value.isInstanceOf[Int]) field(fieldName) >= value.asInstanceOf[Int] + else if(value.isInstanceOf[Byte]) field(fieldName) >= value.asInstanceOf[Byte] + else if(value.isInstanceOf[Short]) field(fieldName) >= value.asInstanceOf[Short] + else if(value.isInstanceOf[Long]) field(fieldName) >= value.asInstanceOf[Long] + else if(value.isInstanceOf[Float]) field(fieldName) >= value.asInstanceOf[Float] + else if(value.isInstanceOf[Double]) field(fieldName) >= value.asInstanceOf[Double] + else if(value.isInstanceOf[BigDecimal]) field(fieldName) >= value.asInstanceOf[BigDecimal] + else if(value.isInstanceOf[java.sql.Date]) field(fieldName) >= convertToODate(value.asInstanceOf[java.sql.Date]) + else if(value.isInstanceOf[java.sql.Timestamp]) field(fieldName) >= convertToOTimeStamp(value.asInstanceOf[java.sql.Timestamp]) + else if(value.isInstanceOf[String]) field(fieldName) >= value.asInstanceOf[String] + else throw new RuntimeException(s"Cannot convert $value to a MapRDB predicate") + } + + private def convertToCondition(filters: Array[Filter], inNot: Boolean): Predicate = { + val resultPredicate : Array[Predicate] = filters.map { + case EqualTo(fld, value) => { + if (!inNot) + getPredicate(fld, QueryCondition.Op.EQUAL, value) + else getPredicate(fld, QueryCondition.Op.NOT_EQUAL, value) + } + case EqualNullSafe(fld, value) => { + if (!inNot) + getPredicate(fld, QueryCondition.Op.EQUAL, value) + else getPredicate(fld, QueryCondition.Op.NOT_EQUAL, value) + } + case GreaterThan(fld, value) => { + if (!inNot) + getPredicate(fld, QueryCondition.Op.GREATER, value) + else getPredicate(fld, QueryCondition.Op.LESS_OR_EQUAL, value) + } + case GreaterThanOrEqual(fld, value) => { + if (!inNot) + getPredicate(fld, QueryCondition.Op.GREATER_OR_EQUAL, value) + else getPredicate(fld, QueryCondition.Op.LESS, value) + } + case In(fld, values) => { + if (!inNot) + field(fld) in values + else field(fld) notin values + } + case LessThan(fld, value) => { + if (!inNot) + getPredicate(fld, QueryCondition.Op.LESS, value) + else getPredicate(fld, QueryCondition.Op.GREATER_OR_EQUAL, value) + } + case LessThanOrEqual(fld, value) => { + if (!inNot) + getPredicate(fld, QueryCondition.Op.LESS_OR_EQUAL, value) + else getPredicate(fld, QueryCondition.Op.GREATER, value) + } + case IsNull(fld) => { + if (!inNot) + field(fld) notexists + else field(fld) exists + } + case IsNotNull(fld) => { + if (!inNot) + field(fld) exists + else field(fld) notexists + } + case And(leftFilter, rightFilter) => { + if (!inNot) + convertToCondition(Array(leftFilter), inNot) and convertToCondition(Array(rightFilter), inNot) + else convertToCondition(Array(leftFilter), inNot) or convertToCondition(Array(rightFilter), inNot) + } + case Or(leftFilter, rightFilter) => { + if (!inNot) + convertToCondition(Array(leftFilter), inNot) or convertToCondition(Array(rightFilter), inNot) + else convertToCondition(Array(leftFilter), inNot) and convertToCondition(Array(rightFilter), inNot) + } + case Not(filter) => { + if (!inNot) + convertToCondition(Array(filter), true) + else convertToCondition(Array(filter), false) + + } + case StringStartsWith(fld, value) => { + if (!inNot) + field(fld) like s"$value%" + else field(fld) notlike s"$value%" + } + case StringEndsWith(fld, value) => { + if (!inNot) + field(fld) like s"%$value" + else field(fld) notlike s"%$value" + } + case StringContains(fld, value) => { + if (!inNot) + field(fld) like s"%$value%" + else field(fld) notlike s"$value" + } + case _ => null + } + resultPredicate.filter(_ != null).reduceLeft[Predicate]((predicate1, predicate2) => predicate1 and predicate2) + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/sql/SparkSessionFunctions.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/SparkSessionFunctions.scala new file mode 100644 index 0000000000000..23d0d20c0f779 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/SparkSessionFunctions.scala @@ -0,0 +1,21 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.sql + +import com.mapr.db.spark.utils.MapRSpark +import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.types.StructType + +import scala.reflect.runtime.universe._ + +case class SparkSessionFunctions(@transient sparkSession: SparkSession) extends Serializable { + + def loadFromMapRDB[T <: Product : TypeTag](tableName: String, + schema : StructType = null, + sampleSize: Double = GenerateSchema.SAMPLE_SIZE): DataFrame = { + MapRSpark.builder() + .sparkSession(sparkSession) + .configuration() + .sparkContext(sparkSession.sparkContext) + .setTable(tableName).build().toDF[T](schema, sampleSize) + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/sql/package.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/package.scala new file mode 100644 index 0000000000000..fcb6912c86c78 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/package.scala @@ -0,0 +1,27 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark + +import org.apache.spark.sql._ + +package object sql { + + implicit def toSparkSessionFunctions(sqlContext: SQLContext): SparkSessionFunctions = { + toSparkSessionFunctions(sqlContext.sparkSession) + } + + implicit def toSparkSessionFunctions(sparkSession: SparkSession): SparkSessionFunctions = { + SparkSessionFunctions(sparkSession) + } + + implicit def toMaprdbReaderFunctions(dfr: DataFrameReader): MapRDBDataFrameReaderFunctions = { + MapRDBDataFrameReaderFunctions(dfr) + } + + implicit def toMaprdbWriterFunctions(dfw: DataFrameWriter[_]): MapRDBDataFrameWriterFunctions = { + MapRDBDataFrameWriterFunctions(dfw) + } + + implicit def toMapRDBDataFrame(ds: Dataset[_]): MapRDBDataFrameFunctions = { + MapRDBDataFrameFunctions(ds.toDF()) + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/sql/types/ConflictType.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/types/ConflictType.scala new file mode 100644 index 0000000000000..5b27931340b45 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/types/ConflictType.scala @@ -0,0 +1,13 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.sql.types + +import org.apache.spark.sql.types.DataType + +private[sql] class InvalidType private () extends DataType with Serializable { + def defaultSize: Int = 0 + + def asNullable: DataType = this + override def toString: String = "InvalidType" +} + +private[sql] case object InvalidType extends InvalidType diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/sql/utils/MapRSqlUtils.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/utils/MapRSqlUtils.scala new file mode 100644 index 0000000000000..3e62f21fdd22f --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/sql/utils/MapRSqlUtils.scala @@ -0,0 +1,420 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.sql.utils + +import java.nio.ByteBuffer + +import com.mapr.db.MapRDB +import com.mapr.db.rowcol.DBValueBuilderImpl +import com.mapr.db.spark.MapRDBSpark +import com.mapr.db.spark.dbclient.DBClient +import com.mapr.db.spark.exceptions.SchemaMappingException +import com.mapr.db.spark.impl.OJAIDocument +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.types._ +import org.ojai.DocumentReader +import org.ojai.types.{ODate, OInterval, OTimestamp} +import scala.language.implicitConversions +import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConverters._ +import org.ojai.store.DocumentMutation + +object MapRSqlUtils { + + def documentToRow(document: OJAIDocument, schema: StructType): Row = { + convertObject(document.getDoc.asReader(), schema) + } + + def documentsToRow(documents: Iterator[DocumentReader], schema: StructType, + requiredColumns: Array[String] = Array.empty[String]): Iterator[Row] = { + documents.flatMap(record => convertRootField(record, schema) match { + case row: Row => row :: Nil + case array: ArrayData => + if (array.numElements == 0) Nil + else array.toArray[Row](schema) + }) + } + + def convertRootField(docReader: DocumentReader, + schema: DataType): Any = { + val event = docReader.next() + (event, schema) match { + case (DocumentReader.EventType.START_ARRAY, at @ ArrayType(st, _)) => + convertArray(docReader, at) + + case (DocumentReader.EventType.START_MAP, st : StructType) => + convertObject(docReader, st) + + case _ => + convertField(event, docReader, schema, docReader.getFieldName) + } + } + + private def convertObject(documentReader: DocumentReader, + schema: StructType): Row = { + val values = ArrayBuffer.fill[Any](schema.fields.size)(null) + var event: DocumentReader.EventType = null + event = documentReader.next + while (event != null && event.compareTo(DocumentReader.EventType.END_MAP) != 0) { + val fieldName = documentReader.getFieldName + if (schema.fieldNames.contains(fieldName)) { + val index = schema.fieldIndex(fieldName) + values.update(index, convertField(event, documentReader, schema(index).dataType, fieldName)) + } + event = documentReader.next + } + new GenericRowWithSchema(values.toArray, schema.asInstanceOf[StructType]) + } + + private def convertArray(documentReader: DocumentReader, + array: ArrayType): Any = { + val values = ArrayBuffer.empty[Any] + var event: DocumentReader.EventType = null + val fieldName = documentReader.getFieldName + event = documentReader.next + while (event != null && event.compareTo(DocumentReader.EventType.END_ARRAY) != 0) { + values += convertField(event, documentReader, array.elementType, fieldName) + event = documentReader.next + } + values.toArray + } + + private def convertMap(documentReader: DocumentReader, + mapType: MapType): Any = { + val keys = ArrayBuffer.empty[String] + val values = ArrayBuffer.empty[Any] + var event: DocumentReader.EventType = null + event = documentReader.next + while (event!= null && event.compareTo(DocumentReader.EventType.END_MAP) != 0) { + keys += documentReader.getFieldName + values += convertField(event, documentReader, mapType.valueType, documentReader.getFieldName) + event = documentReader.next + } + ArrayBasedMapData.toScalaMap(keys.toArray, values.toArray) + } + + def convertField(event: DocumentReader.EventType, + documentReader: DocumentReader, + schema: DataType, fieldName: String): Any = { + ((event, schema)) match { + + case (DocumentReader.EventType.START_ARRAY, at @ ArrayType(st, _)) => { + convertArray(documentReader, at ) + } + + case (DocumentReader.EventType.START_MAP, mt @ MapType(StringType, kt, _)) => { + convertMap(documentReader, mt) + } + + case (DocumentReader.EventType.START_MAP, st: StructType) => { + convertObject(documentReader, schema.asInstanceOf[StructType]) + } + + case (DocumentReader.EventType.NULL | null, _) => { + null + } + + case (DocumentReader.EventType.STRING, _) if documentReader.getString.length < 1 => null + + case (DocumentReader.EventType.STRING, StringType) => { + documentReader.getString + } + + case (DocumentReader.EventType.STRING, BinaryType) => { + documentReader.getString.getBytes + } + + case (DocumentReader.EventType.STRING, DateType) => { + new java.sql.Date(ODate.parse(documentReader.getString).toDaysSinceEpoch) + } + + case (DocumentReader.EventType.STRING, TimestampType) => { + new java.sql.Timestamp(OTimestamp.parse(documentReader.getString).getMillis) + } + + case (DocumentReader.EventType.STRING, DoubleType) => { + documentReader.getString.toDouble + } + + case (DocumentReader.EventType.STRING, FloatType) => { + documentReader.getString.toFloat + } + + case (DocumentReader.EventType.STRING, IntegerType) => { + documentReader.getString.toInt + } + + case (DocumentReader.EventType.STRING, LongType) => { + documentReader.getString.toLong + } + + case (DocumentReader.EventType.STRING, ByteType) => { + documentReader.getString.toByte + } + + case (DocumentReader.EventType.STRING, ShortType) => { + documentReader.getString.toShort + } + + case (DocumentReader.EventType.STRING, dt: DecimalType) => { + Decimal(documentReader.getString.toFloat) + } + + case (DocumentReader.EventType.DATE, DateType) => { + new java.sql.Date(documentReader.getDate.toDate.getTime) + } + + case (DocumentReader.EventType.DATE, TimestampType) => { + new java.sql.Timestamp(documentReader.getDate.toDate.getTime) + } + + case (DocumentReader.EventType.DATE, StringType) => { + documentReader.getDate.toString + } + + case (DocumentReader.EventType.TIME, TimestampType) => { + new java.sql.Timestamp(documentReader.getTime.toDate.getTime) + } + + case (DocumentReader.EventType.TIME, StringType) => { + documentReader.getTime.toString + } + + case (DocumentReader.EventType.TIME, DateType) => { + new java.sql.Date(documentReader.getTime.getMilliSecond) + } + + case (DocumentReader.EventType.TIMESTAMP, TimestampType) => { + new java.sql.Timestamp(documentReader.getTimestampLong) + } + + case (DocumentReader.EventType.TIMESTAMP, StringType) => { + documentReader.getTimestamp.toString + } + + case (DocumentReader.EventType.TIMESTAMP, DateType) => { + new java.sql.Date(documentReader.getTimestampLong) + } + + case (DocumentReader.EventType.INT, IntegerType) => { + documentReader.getInt + } + + case (DocumentReader.EventType.INT, FloatType) => { + documentReader.getInt.toFloat + } + + case (DocumentReader.EventType.INT, StringType) => { + documentReader.getInt.toString + } + + case (DocumentReader.EventType.INT, LongType) => { + documentReader.getInt.toLong + } + + case (DocumentReader.EventType.INT, DoubleType) => { + documentReader.getInt.toDouble + } + + case (DocumentReader.EventType.INT, TimestampType) => { + new java.sql.Timestamp(documentReader.getInt) + } + + case (DocumentReader.EventType.FLOAT, FloatType) => { + documentReader.getFloat + } + + case (DocumentReader.EventType.FLOAT, DoubleType) => { + documentReader.getFloat.toDouble + } + + case (DocumentReader.EventType.FLOAT, dt: DecimalType) => { + Decimal(documentReader.getFloat) + } + + case (DocumentReader.EventType.FLOAT, st: StringType) => { + documentReader.getFloat.toString + } + + case (DocumentReader.EventType.DOUBLE, DoubleType) => { + documentReader.getDouble + } + + case (DocumentReader.EventType.DOUBLE, dt: DecimalType) => { + Decimal(documentReader.getDouble) + } + + case (DocumentReader.EventType.DOUBLE, st: StringType) => { + documentReader.getDouble.toString + } + + case (DocumentReader.EventType.LONG, LongType) => { + documentReader.getLong + } + + //Converting from LONG to Double can loose precision for some values. + case (DocumentReader.EventType.LONG, DoubleType) => { + documentReader.getLong.toDouble + } + + case (DocumentReader.EventType.LONG, TimestampType) => { + new java.sql.Timestamp(documentReader.getLong) + } + + case (DocumentReader.EventType.LONG, StringType) => { + documentReader.getLong.toString + } + + case (DocumentReader.EventType.BYTE, ByteType) => { + documentReader.getByte + } + + case (DocumentReader.EventType.BYTE, ShortType) => { + documentReader.getByte.toShort + } + + case (DocumentReader.EventType.BYTE, IntegerType) => { + documentReader.getByte.toInt + } + + case (DocumentReader.EventType.BYTE, LongType) => { + documentReader.getByte.toLong + } + + case (DocumentReader.EventType.BYTE, FloatType) => { + documentReader.getByte.toFloat + } + + case (DocumentReader.EventType.BYTE, DoubleType) => { + documentReader.getByte.toDouble + } + + case (DocumentReader.EventType.BYTE, dt: DecimalType) => { + Decimal(documentReader.getByte) + } + + case (DocumentReader.EventType.BYTE, StringType) => { + documentReader.getByte.toString + } + + case (DocumentReader.EventType.SHORT, ShortType) => { + documentReader.getShort + } + + case (DocumentReader.EventType.SHORT, IntegerType) => { + documentReader.getShort.toInt + } + + case (DocumentReader.EventType.SHORT, LongType) => { + documentReader.getShort.toLong + } + + case (DocumentReader.EventType.SHORT, FloatType) => { + documentReader.getShort.toFloat + } + + case (DocumentReader.EventType.SHORT, DoubleType) => { + documentReader.getShort.toDouble + } + + case (DocumentReader.EventType.SHORT, dt: DecimalType) => { + Decimal(documentReader.getShort) + } + + case (DocumentReader.EventType.SHORT, StringType) => { + documentReader.getShort.toString + } + + case (DocumentReader.EventType.BINARY, BinaryType) => { + documentReader.getBinary.array() + } + + case (DocumentReader.EventType.BINARY, StringType) => { + documentReader.getBinary.array().map(_.toChar) + } + + case (DocumentReader.EventType.DECIMAL, FloatType) => { + documentReader.getDecimal.floatValue() + } + + case (DocumentReader.EventType.DECIMAL, DoubleType) => { + documentReader.getDecimal.doubleValue() + } + + case (DocumentReader.EventType.DECIMAL, StringType) => { + documentReader.getDecimal.toString + } + + case (DocumentReader.EventType.DECIMAL, LongType) => { + documentReader.getDecimalValueAsLong + } + + case (DocumentReader.EventType.DECIMAL, IntegerType) => { + documentReader.getDecimalValueAsInt + } + + case (DocumentReader.EventType.BOOLEAN, BooleanType) => { + documentReader.getBoolean + } + + case (DocumentReader.EventType.BOOLEAN, StringType) => { + documentReader.getBoolean.toString + } + + case (token, dataType) => { + if (isInvalidType(dataType)) throw new SchemaMappingException(s"Schema cannot be inferred for the column {$fieldName}") + throw new SchemaMappingException(s"Failed to parse a value for data type $dataType (current token: $token)") + } + } + } + + def isInvalidType(dt: DataType) : Boolean = { + if(dt.isInstanceOf[StructType]) { + val fields = dt.asInstanceOf[StructType].fields + if (fields.size == 1 && fields.map(_.name).contains("InvalidType")) return true + } + return false + } + + def rowToDocument(row: Row): OJAIDocument = { + val document = MapRDBSpark.newDocument(DBClient().newDocument()) + row.schema.fields.zipWithIndex.foreach({ + case (field, i) if !row.isNullAt(i) => document.set(field.name, convertToDataType(row.get(i), field.dataType)) + case (field, i) if row.isNullAt(i) => document.setNull(field.name) + }) + document + } + + def convertToDataType(element: Any, elementType: DataType): org.ojai.Value = { + (elementType) match { + case (mapType: StructType) => { + val map = rowToDocument(element.asInstanceOf[Row]).getDoc + DBValueBuilderImpl.KeyValueBuilder.initFromObject(map) + } + case (arrayType: ArrayType) => { + val seq = element.asInstanceOf[Seq[Any]] + DBValueBuilderImpl.KeyValueBuilder.initFrom(seq.map(elem => convertToDataType(elem, arrayType.elementType)).asJava) + } + case (mpType: MapType) => { + val map = element.asInstanceOf[Map[String, Any]] + DBValueBuilderImpl.KeyValueBuilder.initFrom(map.asJava) + } + case (BinaryType) => DBValueBuilderImpl.KeyValueBuilder.initFrom(element.asInstanceOf[ByteBuffer]) + case (BooleanType) => DBValueBuilderImpl.KeyValueBuilder.initFrom(element.asInstanceOf[Boolean]) + case (DateType) => DBValueBuilderImpl.KeyValueBuilder.initFrom(new ODate(element.asInstanceOf[java.sql.Date])) + case (TimestampType) => DBValueBuilderImpl.KeyValueBuilder.initFrom(element.asInstanceOf[OTimestamp]) + case (NullType) => DBValueBuilderImpl.KeyValueBuilder.initFromNull() + case (DoubleType) => DBValueBuilderImpl.KeyValueBuilder.initFrom(element.asInstanceOf[Double]) + case (IntegerType) => DBValueBuilderImpl.KeyValueBuilder.initFrom(element.asInstanceOf[Integer]) + case (LongType) => DBValueBuilderImpl.KeyValueBuilder.initFrom(element.asInstanceOf[Long]) + case (StringType) => DBValueBuilderImpl.KeyValueBuilder.initFrom(element.asInstanceOf[String]) + case (CalendarIntervalType) => DBValueBuilderImpl.KeyValueBuilder.initFrom(element.asInstanceOf[OInterval]) + case (DecimalType()) => DBValueBuilderImpl.KeyValueBuilder.initFrom(element.asInstanceOf[java.math.BigDecimal]) + case (FloatType) => DBValueBuilderImpl.KeyValueBuilder.initFrom(element.asInstanceOf[Float]) + case (ShortType) => DBValueBuilderImpl.KeyValueBuilder.initFrom(element.asInstanceOf[Short]) + case (ByteType) => DBValueBuilderImpl.KeyValueBuilder.initFrom(element.asInstanceOf[Byte]) + case _ => throw new RuntimeException(s"Cannot convert ${elementType} of a row to OjaiDocument's type") + } + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/streaming/DStreamFunctions.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/streaming/DStreamFunctions.scala new file mode 100644 index 0000000000000..1bced4a2ca457 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/streaming/DStreamFunctions.scala @@ -0,0 +1,27 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.streaming + +import com.mapr.db.spark.dbclient.DBClient +import com.mapr.db.spark.utils.LoggingTrait +import org.apache.spark.SparkContext +import org.apache.spark.streaming.dstream.DStream +import org.ojai.DocumentConstants +import com.mapr.db.spark._ +import com.mapr.db.spark.writers.OJAIValue + +class DStreamFunctions[T](dStream: DStream[T])(implicit fv: OJAIValue[T]) extends Serializable with LoggingTrait { + + def sparkContext: SparkContext = dStream.context.sparkContext + + def saveToMapRDB(tableName: String, createTable: Boolean = false, + bulkInsert: Boolean = false, idFieldPath: String = DocumentConstants.ID_KEY) : Unit = { + logDebug("DStreamFunctions is called for table: "+tableName+" with bulkinsert flag set: "+bulkInsert+" and createTable:"+ createTable) + + if (createTable) { + logDebug("Table:" + tableName + " is created in DStreamFunctions") + DBClient().createTable(tableName) + } + + dStream.foreachRDD(rdd => rdd.saveToMapRDB(tableName,false,bulkInsert,idFieldPath)) + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/streaming/package.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/streaming/package.scala new file mode 100644 index 0000000000000..1ae9213ab8ff9 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/streaming/package.scala @@ -0,0 +1,11 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark + +import com.mapr.db.spark.writers.OJAIValue +import org.apache.spark.streaming.dstream.DStream + +package object streaming { + + implicit def toDStreamFunctions[T: OJAIValue](ds: DStream[T]): DStreamFunctions[T] = + new DStreamFunctions[T](ds) +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/types/DBArrayValue.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/types/DBArrayValue.scala new file mode 100644 index 0000000000000..414c9fc964471 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/types/DBArrayValue.scala @@ -0,0 +1,86 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.types + +import java.io.{Externalizable, ObjectInput, ObjectOutput} +import java.nio._ +import java.util + +import scala.collection.JavaConverters._ +import scala.collection.generic.{CanBuildFrom, GenericTraversableTemplate, SeqFactory} +import scala.collection.mutable.ListBuffer +import scala.collection.SeqLike +import scala.language.implicitConversions +import com.mapr.db.rowcol.RowcolCodec +import com.mapr.db.spark.dbclient.DBClient +import com.mapr.db.spark.utils.MapRDBUtils +import com.mapr.db.util.ByteBufs + +private[spark] object DBArrayValue extends SeqFactory[DBArrayValue] { + implicit def canBuildFrom[T]: CanBuildFrom[Coll, T, DBArrayValue[T]] = + new GenericCanBuildFrom[T] + def newBuilder[T] = new ListBuffer[T] mapResult (x => new DBArrayValue(x)) +} + +private[spark] class DBArrayValue[T]( @transient private[spark] var arr : Seq[T]) + extends Seq[T] + with GenericTraversableTemplate[T, DBArrayValue] + with SeqLike[T, DBArrayValue[T]] with Externalizable{ + + def this() { + this(null) + } + + override def companion = DBArrayValue + def iterator: Iterator[T] = new ListIterator[T](arr) + def apply(idx: Int): T = { + if (idx < 0 || idx>=length) throw new IndexOutOfBoundsException + val element = arr(idx) + if (element.isInstanceOf[java.util.List[_]]) { + new DBArrayValue(element.asInstanceOf[java.util.List[Object]].asScala).asInstanceOf[T] + } else if (element.isInstanceOf[java.util.Map[_, _]]) { + new DBMapValue(element.asInstanceOf[util.Map[String, Object]].asScala.toMap).asInstanceOf[T] + } else + element + } + + def length: Int = arr.size + + private def getval = this.arr + + override def writeExternal(objectOutput: ObjectOutput): Unit = { + val newdoc = DBClient().newDocument().set("encode", arr.map(a => a.asInstanceOf[AnyRef]).asJava) + val buff = RowcolCodec.encode(newdoc) + buff.order(ByteOrder.LITTLE_ENDIAN) + objectOutput.writeInt(buff.capacity()) + objectOutput.write(buff.array(),0,buff.capacity()) + } + + override def readExternal(objectinput: ObjectInput) : Unit = { + val buffersize = objectinput.readInt() + val buffer = ByteBufs.allocate(buffersize) + MapRDBUtils.readBytes(buffer,buffersize,objectinput) + val doc = RowcolCodec.decode(buffer) + this.arr = doc.getList("encode").asScala.map(a => a.asInstanceOf[T]) + } + + override def toString = { + this.arr.toString() + } + + override def hashCode() : Int = { + this.arr.size + } + + override def equals(other: Any) : Boolean = { + if (other.isInstanceOf[DBArrayValue[_]]) { + val that = other.asInstanceOf[DBArrayValue[_]] + val result = this.sameElements(that) + return result + } else if (other.isInstanceOf[Seq[_]]) { + val that = new DBArrayValue(other.asInstanceOf[Seq[_]]) + val result = this.arr.sameElements(that) + return result + } + return false + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/types/DBBinaryValue.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/types/DBBinaryValue.scala new file mode 100644 index 0000000000000..335414f98da2b --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/types/DBBinaryValue.scala @@ -0,0 +1,49 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.types + +import java.io.{Externalizable, ObjectInput, ObjectOutput} +import java.nio.{ByteBuffer, ByteOrder} +import com.mapr.db.impl.IdCodec +import com.mapr.db.spark.utils.MapRDBUtils +import com.mapr.db.util.ByteBufs +import com.mapr.org.apache.hadoop.hbase.util.Bytes + +private[spark] final class DBBinaryValue(@transient private[spark] var bin: ByteBuffer) extends Externalizable { + def this() { + this(null) + } + + override def writeExternal(objectOutput: ObjectOutput): Unit = { + objectOutput.writeInt(bin.capacity()) + bin.order(ByteOrder.LITTLE_ENDIAN) + objectOutput.write(bin.array()) + } + + override def readExternal(objectinput: ObjectInput) : Unit = { + val buffersize = objectinput.readInt() + val buffer = ByteBufs.allocate(buffersize) + MapRDBUtils.readBytes(buffer,buffersize,objectinput) + this.bin = buffer + } + + def getByteBuffer() = this.bin + + def array() = this.bin.array() + + private def getval = this.bin + + override def toString = { + this.bin.toString() + } + + override def hashCode() : Int = { + this.bin.array().size + } + + override def equals(other: Any) : Boolean = { + if (!other.isInstanceOf[DBBinaryValue]) return false + val prevBuf = IdCodec.encodeAsBytes(this.bin) + val curBuf = IdCodec.encodeAsBytes(other.asInstanceOf[DBBinaryValue].bin) + return Bytes.compareTo(prevBuf, 0, prevBuf.length, curBuf, 0, curBuf.length) == 0 + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/types/DBMapValue.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/types/DBMapValue.scala new file mode 100644 index 0000000000000..78e785a59a717 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/types/DBMapValue.scala @@ -0,0 +1,74 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.types + +import java.io.{Externalizable, ObjectInput, ObjectOutput} +import java.nio._ +import java.util + +import scala.language.implicitConversions +import scala.collection.JavaConverters._ +import scala.collection.MapLike +import com.mapr.db.rowcol.RowcolCodec +import com.mapr.db.spark.dbclient.DBClient +import com.mapr.db.spark.utils.MapRDBUtils +import com.mapr.db.util.ByteBufs + +private[spark] final class DBMapValue(@transient private[spark] var value: Map[String,AnyRef]) + extends Map[String,AnyRef] with MapLike[String, AnyRef, DBMapValue] with Externalizable{ + + def this() { + this(null) + } + + override def empty = new DBMapValue(Map.empty) + + private lazy val m = Map[String, AnyRef](getMap.toList: _*) + override def +[B1 >: AnyRef](kv: (String, B1)) = m + kv + override def -(k: String) = new DBMapValue(m - k) + override def iterator = new MapIterator(m) + override def get(s: String) = { + val element = m.get(s) + if (element.isDefined && element.get.isInstanceOf[java.util.List[_]]) { + Option(new DBArrayValue(element.get.asInstanceOf[java.util.List[Object]].asScala)) + } else if (element.isDefined && element.get.isInstanceOf[java.util.Map[_, _]]) { + Option(new DBMapValue(element.get.asInstanceOf[util.Map[String, Object]].asScala.toMap)) + } else { + element + } + } + + private lazy val getMap = value + + override def writeExternal(objectOutput: ObjectOutput): Unit = { + val newdoc = DBClient().newDocument().set("encode", + (value map {case (k,v) => k -> v.asInstanceOf[AnyRef]}).asJava) + val buff = RowcolCodec.encode(newdoc) + objectOutput.writeInt(buff.capacity()) + buff.order(ByteOrder.LITTLE_ENDIAN) + objectOutput.write(buff.array(),0,buff.capacity()) + } + + override def readExternal(objectinput: ObjectInput) : Unit = { + val buffersize = objectinput.readInt() + val buffer = ByteBufs.allocate(buffersize) + MapRDBUtils.readBytes(buffer,buffersize,objectinput) + this.value = RowcolCodec.decode(buffer).getMap("encode").asScala.toMap + } + + override def hashCode() : Int = { + this.keySet.size + } + + override def equals(other: Any) : Boolean = { + if (other.isInstanceOf[DBMapValue]) { + val that : DBMapValue = other.asInstanceOf[DBMapValue] + val result = this.sameElements(that) + return result + } else if (other.isInstanceOf[Map[_, _]]) { + val that: DBMapValue = new DBMapValue(other.asInstanceOf[Map[String, AnyRef]]) + val result = this.getMap.sameElements(that) + return result + } + return false + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/types/TypeIterator.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/types/TypeIterator.scala new file mode 100644 index 0000000000000..8e938c8a7abae --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/types/TypeIterator.scala @@ -0,0 +1,40 @@ +package com.mapr.db.spark.types + +import java.util +import scala.language.implicitConversions +import scala.collection.JavaConverters._ + + +class MapIterator(m: Map[String, AnyRef]) extends Iterator[scala.Tuple2[String, AnyRef]] { + val mapIterator = m.iterator + + def hasNext = mapIterator.hasNext + + def next() = { + val nextElem = mapIterator.next() + if (nextElem._2.isInstanceOf[java.util.List[_]]) { + (nextElem._1, new DBArrayValue(nextElem._2.asInstanceOf[java.util.List[Object]].asScala)) + } else if (nextElem._2.isInstanceOf[java.util.Map[_, _]]) { + (nextElem._1, new DBMapValue(nextElem._2.asInstanceOf[util.Map[String, Object]].asScala.toMap)) + } else { + nextElem + } + } +} + +class ListIterator[T](s: Seq[T]) extends Iterator[T] { + val seqIterator = s.iterator + + def hasNext = seqIterator.hasNext + + def next() = { + val nextElem = seqIterator.next() + if (nextElem.isInstanceOf[java.util.List[_]]) { + new DBArrayValue(nextElem.asInstanceOf[java.util.List[Object]].asScala).asInstanceOf[T] + } else if (nextElem.isInstanceOf[java.util.Map[_, _]]) { + new DBMapValue(nextElem.asInstanceOf[util.Map[String, Object]].asScala.toMap).asInstanceOf[T] + } else { + nextElem + } + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/utils/DefaultClass.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/utils/DefaultClass.scala new file mode 100644 index 0000000000000..5dc5e9b13dd53 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/utils/DefaultClass.scala @@ -0,0 +1,20 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.utils + +import scala.language.implicitConversions + + +object DefaultClass extends Serializable { + + sealed class DefaultType[A, B] extends Serializable + + object DefaultType extends BaseClassDefaultType { + + implicit def default[B]: DefaultType[B, B] = new DefaultType[B, B] + } + + trait BaseClassDefaultType { + + implicit def overrideDefault[A, B]: DefaultType[A, B] = new DefaultType[A, B] + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/utils/LoggingTrait.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/utils/LoggingTrait.scala new file mode 100644 index 0000000000000..0662e78f36db2 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/utils/LoggingTrait.scala @@ -0,0 +1,74 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.utils + +import org.slf4j.{Logger, LoggerFactory} + +/** + * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows + * logging messages at different levels using methods that only evaluate parameters lazily if the + * log level is enabled. + */ +private[spark] trait LoggingTrait { + + // Make the log field transient so that objects with Logging can + // be serialized and used on another machine + @transient private var log_ : Logger = null // scalastyle:ignore + + // Method to get the logger name for this object + protected def logName = { + // Ignore trailing $'s in the class names for Scala objects + this.getClass.getName.stripSuffix("$") + } + + // Method to get or create the logger for this object + protected def log: Logger = { + if (log_ == null) { // scalastyle:ignore + log_ = LoggerFactory.getLogger(logName) + } + log_ + } + + // Log methods that take only a String + protected def logInfo(msg: => String) { + if (log.isInfoEnabled) log.info(msg) + } + + protected def logDebug(msg: => String) { + if (log.isDebugEnabled) log.debug(msg) + } + + protected def logTrace(msg: => String) { + if (log.isTraceEnabled) log.trace(msg) + } + + protected def logWarning(msg: => String) { + if (log.isWarnEnabled) log.warn(msg) + } + + protected def logError(msg: => String) { + if (log.isErrorEnabled) log.error(msg) + } + + // Log methods that take Throwables (Exceptions/Errors) too + protected def logInfo(msg: => String, throwable: Throwable) { + if (log.isInfoEnabled) log.info(msg, throwable) + } + + protected def logDebug(msg: => String, throwable: Throwable) { + if (log.isDebugEnabled) log.debug(msg, throwable) + } + + protected def logTrace(msg: => String, throwable: Throwable) { + if (log.isTraceEnabled) log.trace(msg, throwable) + } + + protected def logWarning(msg: => String, throwable: Throwable) { + if (log.isWarnEnabled) log.warn(msg, throwable) + } + + protected def logError(msg: => String, throwable: Throwable) { + if (log.isErrorEnabled) log.error(msg, throwable) + } + +} + diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/utils/MapRDBUtils.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/utils/MapRDBUtils.scala new file mode 100644 index 0000000000000..c6a903a1e13c3 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/utils/MapRDBUtils.scala @@ -0,0 +1,124 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.utils + +import java.io.ObjectInput +import java.nio.ByteBuffer +import com.mapr.db.exceptions.TableNotFoundException +import com.mapr.db.exceptions.TableExistsException +import com.mapr.db.impl.ConditionNode.RowkeyRange +import com.mapr.db.spark.MapRDBSpark +import com.mapr.db.spark.codec.BeanCodec +import com.mapr.db.spark.dbclient.DBClient +import com.mapr.db.spark.types.{DBArrayValue, DBBinaryValue, DBMapValue} +import com.mapr.db.spark.writers.OJAIKey +import com.mapr.fs.jni.MapRConstants +import com.mapr.org.apache.hadoop.hbase.util.Bytes +import org.ojai.Document +import scala.collection.JavaConverters._ +import scala.reflect.ClassTag + + +private[spark] object MapRDBUtils { + def checkOrCreateTable(tableName: String, bulkMode: Boolean, createTable: Boolean, keys : Seq[org.ojai.Value]): (Boolean, Boolean) = { + var isBulkLoad: Boolean = bulkMode + if (!DBClient().tableExists(tableName)) { + if (!createTable) throw new TableNotFoundException("Table: "+ tableName + " not found") + val tabDesc = DBClient().newTableDescriptor() + tabDesc.setAutoSplit(true) + tabDesc.setPath(tableName) + tabDesc.setBulkLoad(bulkMode) + if (keys.isEmpty) + DBClient().createTable(tabDesc) + else + DBClient().createTable(tabDesc, keys.toArray) + return (true,bulkMode) + } else if (createTable ) { + throw new TableExistsException("Table: " + tableName + " already Exists") + } + + if (bulkMode == true) { + isBulkLoad = DBClient().isBulkLoad(tableName) + } + return (false, isBulkLoad) + } + + def setBulkLoad(tableName: String, bulkMode: Boolean): Unit = { + val desc = DBClient().getTableDescriptor(tableName) + desc.setBulkLoad(bulkMode) + DBClient().alterTable(desc) + } + + def containsRow(row : Array[Byte], rowkeyRange: RowkeyRange): Boolean = { + return Bytes.compareTo(row, rowkeyRange.getStartRow) >= 0 && + (Bytes.compareTo(row, rowkeyRange.getStopRow) < 0 || + Bytes.equals(rowkeyRange.getStopRow, MapRConstants.EMPTY_BYTE_ARRAY)); + } + + def toBeanClass[T](doc: Document, beanClass: Class[T]): T = { + if (beanClass.getSimpleName.equals("OJAIDocument")) + MapRDBSpark.newDocument(doc).asInstanceOf[T] + else + BeanCodec.encode[T](doc.asReader(),beanClass) + } + + def readBytes(buff: ByteBuffer, bufferSize: Int, objectInput : ObjectInput): Unit = { + val byteArray = new Array[Byte](bufferSize) + var readbytes = objectInput.read(byteArray, 0, bufferSize) + buff.put(byteArray, 0, readbytes) + var remaining = bufferSize - readbytes + + while (remaining > 0) { + val read = objectInput.read(byteArray, readbytes, remaining) + buff.put(byteArray, readbytes, read) + readbytes += read + remaining -= read + } + buff.flip() + } + + def convertToSeq(value: Seq[Any]): Seq[AnyRef] = { + if (value.isInstanceOf[DBArrayValue[_]]) { + value.asInstanceOf[DBArrayValue[_]].arr.map(_.asInstanceOf[AnyRef]) + } else { + value.map(convertToScalaCollection(_)) + } + } + + def convertToMap(value : Map[String,Any]): Map[String, AnyRef] = { + if (value.isInstanceOf[DBMapValue]) { + value.asInstanceOf[DBMapValue].value.map{ case (k,v) => k -> v.asInstanceOf[AnyRef]} + } else { + value.map{ case (k,v) => k -> convertToScalaCollection(v)} + } + } + + def convertToScalaCollection(value: Any): AnyRef = { + if (value.isInstanceOf[DBMapValue]) { + return value.asInstanceOf[DBMapValue].value.asJava.asInstanceOf[AnyRef] + } else if (value.isInstanceOf[DBArrayValue[_]]) { + return value.asInstanceOf[DBArrayValue[_]].arr.asJava.asInstanceOf[AnyRef] + } else if (value.isInstanceOf[Map[_, _]]) { + return value.asInstanceOf[Map[String,Any]].map{case (k,v) => k -> convertToScalaCollection(v)}.asJava.asInstanceOf[AnyRef] + } else if (value.isInstanceOf[Seq[Any]]) { + return value.asInstanceOf[Seq[Any]].map(convertToScalaCollection(_)).asJava.asInstanceOf[AnyRef] + } else { + return value.asInstanceOf[AnyRef] + } + } + + def getOjaiKey[T : ClassTag]() : OJAIKey[T] = { + import reflect._ + val result = classTag[T] match { + case a if a <<: classTag[String] => return OJAIKey.ojaiStringKey.asInstanceOf[OJAIKey[T]] + case a if a <<: classTag[ByteBuffer] => return OJAIKey.ojaibytebufferKey.asInstanceOf[OJAIKey[T]] + case a if a <<: classTag[DBBinaryValue] => return OJAIKey.ojaibinaryKey.asInstanceOf[OJAIKey[T]] + case _ => throw new RuntimeException("Key with type:" + classTag[T].runtimeClass + " is not supported") + } + + return result.asInstanceOf[OJAIKey[T]] + } + + implicit class ClassTagOps[T](val classTag: ClassTag[T]) extends AnyVal { + def <<:(other: ClassTag[_]) = classTag.runtimeClass.isAssignableFrom(other.runtimeClass) + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/utils/MapRSpark.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/utils/MapRSpark.scala new file mode 100644 index 0000000000000..322a3f812562e --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/utils/MapRSpark.scala @@ -0,0 +1,181 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.utils + +import com.mapr.db.MapRDB +import com.mapr.db.spark.RDD.{MapRDBTableScanRDD, RDDTYPE} +import com.mapr.db.spark.condition.{DBQueryCondition, Predicate} +import com.mapr.db.spark.configuration.SerializableConfiguration + +import scala.reflect._ +import scala.reflect.runtime.universe._ +import org.apache.spark.SparkContext +import com.mapr.db.impl.ConditionImpl +import org.apache.hadoop.conf.Configuration +import org.ojai.store.{DocumentMutation, QueryCondition} +import com.mapr.db.spark.utils.DefaultClass.DefaultType +import com.mapr.db.spark.impl.OJAIDocument +import org.apache.spark.sql._ +import com.mapr.db.spark.RDD.MapRDBBaseRDD +import com.mapr.db.spark._ +import com.mapr.db.spark.dbclient.DBClient +import com.mapr.db.spark.sql.GenerateSchema +import com.mapr.db.spark.sql.utils.MapRSqlUtils +import org.apache.spark.sql.types.StructType + +object MapRSpark { + + val defaultSource = "com.mapr.db.spark.sql.DefaultSource" + + def builder(): Builder = new Builder + + def save[D](dataset: Dataset[D], tableName: String, idFieldPath:String, createTable: Boolean, bulkInsert:Boolean): Unit = { + val documentRdd = dataset.toDF.rdd.map(MapRSqlUtils.rowToDocument(_)) + documentRdd.saveToMapRDB(tableName, createTable = createTable, bulkInsert = bulkInsert, idFieldPath = idFieldPath) + } + + def insert[D](dataset: Dataset[D], tableName: String, idFieldPath:String, createTable: Boolean, bulkInsert:Boolean): Unit = { + val documentRdd = dataset.toDF.rdd.map(MapRSqlUtils.rowToDocument(_)) + documentRdd.insertToMapRDB(tableName, createTable = createTable, bulkInsert = bulkInsert, idFieldPath = idFieldPath) + } + + def update[D](dataset: Dataset[D], tableName: String, idFieldPath:String, createTable: Boolean, bulkInsert:Boolean): Unit = { + val documentRdd = dataset.toDF.rdd.map(MapRSqlUtils.rowToDocument(_)) + documentRdd.saveToMapRDB(tableName, createTable = createTable, bulkInsert = bulkInsert, idFieldPath = idFieldPath) + } + + def update(df: DataFrame, tableName: String, mutation: (Row) => DocumentMutation,getID: (Row) => org.ojai.Value): Unit = { + val documentRdd = df.rdd + documentRdd.updateToMapRDB(tableName, mutation, getID) + } + + def update(df: DataFrame, tableName: String, mutation: (Row) => DocumentMutation,getID: (Row) => org.ojai.Value, condition: Predicate): Unit = { + val documentRdd = df.rdd + documentRdd.updateToMapRDB(tableName, mutation, getID, condition) + } + + def save(dfw: DataFrameWriter[_], tableName: String): Unit = { + dfw.format(defaultSource).option("tableName", tableName).save() + } + + def load(sc: SparkContext, tableName: String) : MapRDBBaseRDD[OJAIDocument] = { + sc.loadFromMapRDB(tableName) + } + + class Builder { + private var sparkctx : Option[SparkContext] = None + private var sparkSession : Option[SparkSession] = None + private var condition: Option[QueryCondition] = None + private var dbcondition: Option[DBQueryCondition] = None + private var tableName: Option[String] = None + private var sampleSize: Option[String] = None + private var conf: Option[SerializableConfiguration] = None + private var beanClass: Option[Class[_]] = None + private var columnProjection: Option[Seq[String]] = None + + def build(): MapRSpark = { + require(sparkctx.isDefined, "The SparkContext must be set.") + require(tableName.isDefined, "Source should be set") + require(conf.isDefined, "Configuration should be set") + val cond = condition match { + case Some(x) => Option(x) + case None => Option(DBClient().newCondition().build) + } + + new MapRSpark(sparkSession, sparkctx, conf, dbcondition, tableName, columnProjection) + } + + def configuration(conf: Configuration = new Configuration): Builder = { + val sercnf = new SerializableConfiguration(conf) + this.conf = Option(sercnf) + this + } + + def sparkContext(ctx: SparkContext): Builder = { + this.sparkctx = Option(ctx) + this.sparkSession = Option(SparkSession.builder().config(ctx.getConf).getOrCreate()) + this + } + + def sparkSession(sparkSession: SparkSession): Builder = { + this.sparkSession = Option(sparkSession) + this.sparkctx = Option(sparkSession.sparkContext) + this + } + + def setCond(cond: Option[QueryCondition]): Builder = { + this.condition = cond + this.dbcondition = if(cond.isDefined) Option(DBQueryCondition(cond.get)) else None + this + } + + def setDBCond(cond: DBQueryCondition): Builder = { + this.dbcondition = Option(cond) + this + } + + def setTable(tableName: String): Builder = { + this.tableName = Option(tableName) + this + } + + def setBeanClass(beanClass: Class[_]): Builder = { + this.beanClass = Option(beanClass) + this + } + + def setColumnProjection(columns: Option[Seq[String]]) : Builder = { + this.columnProjection = columns + this + } + } +} + +case class MapRSpark(sparkSession: Option[SparkSession], sc: Option[SparkContext], conf: Option[SerializableConfiguration], cond: Option[DBQueryCondition], + tableName: Option[String], columns: Option[Seq[String]]) { + + def toRDD[T : ClassTag](beanClass: Class[T] = null)(implicit e: T DefaultType OJAIDocument, f: RDDTYPE[T]): MapRDBTableScanRDD[T] = rdd[T](beanClass) + + def toJavaRDD[D: ClassTag](clazz: Class[D])(implicit f : RDDTYPE[D]) = toRDD[D](clazz) + + private def rdd[T : ClassTag](beanClass: Class[T])(implicit e: T DefaultType OJAIDocument, f: RDDTYPE[T]) : MapRDBTableScanRDD[T] = + new MapRDBTableScanRDD[T](sparkSession.get, sc.get, sc.get.broadcast(conf.get), columns.getOrElse(null), tableName.get, cond.getOrElse(null), beanClass) + + + def toDataFrame(schema: StructType, sampleSize: Double): DataFrame = { + var reader: DataFrameReader = sparkSession.get.read.format("com.mapr.db.spark.sql") + .schema(schema).option("tableName", this.tableName.get) + .option("sampleSize", sampleSize) + + if (cond.isDefined) reader = reader.option("QueryCondition", + new String(cond.get.condition.asInstanceOf[ConditionImpl].getDescriptor.getSerialized.array)) + + if (columns.isDefined) reader = reader.option("ColumnProjection", columns.getOrElse(Seq("")).reduce[String]((str1,str2) => str1+","+str2)) + reader.load() + } + + def toOJAIDocumentRDD(tableName: String) : MapRDBBaseRDD[OJAIDocument] = { + MapRSpark.builder.sparkSession(sparkSession.get).sparkContext(sc.get) + .configuration().setDBCond(cond.getOrElse(null)).setColumnProjection(columns) + .setTable(tableName).build().toRDD[OJAIDocument]() + } + + def toDF[T <: Product : TypeTag]() : DataFrame = { + toDF(null, GenerateSchema.SAMPLE_SIZE) + } + + def toDF[T <: Product : TypeTag](Schema: StructType) : DataFrame = { + toDF(Schema, GenerateSchema.SAMPLE_SIZE) + } + + def toDF[T <: Product : TypeTag](Schema: StructType, sampleSize: Double): DataFrame = { + var derived : StructType = null + if (Schema != null) derived = Schema + else { + derived = GenerateSchema.reflectSchema[T] match { + case Some(reflectedSchema) => reflectedSchema + case None => GenerateSchema(toOJAIDocumentRDD(tableName.get), sampleSize) + } + } + toDataFrame(derived, sampleSize) + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/writers/BulkTableWriter.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/writers/BulkTableWriter.scala new file mode 100644 index 0000000000000..8617c13fe9775 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/writers/BulkTableWriter.scala @@ -0,0 +1,26 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.writers + +import java.nio.ByteBuffer +import org.ojai.Document +import com.mapr.db.mapreduce.BulkLoadRecordWriter +import com.mapr.db.rowcol.DBValueBuilderImpl + +private[spark] case class BulkTableWriter(@transient table: BulkLoadRecordWriter) extends Writer { + + def write(doc: Document, key: ByteBuffer) = { + table.write(DBValueBuilderImpl.KeyValueBuilder.initFrom(key), doc) + } + + def write(doc: Document, key: String) = { + table.write(DBValueBuilderImpl.KeyValueBuilder.initFrom(key),doc) + } + + def write(doc: Document, key: org.ojai.Value) = { + table.write(key, doc) + } + + def close() = { + table.close(null) + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/writers/OJAIKeyWriterHelper.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/writers/OJAIKeyWriterHelper.scala new file mode 100644 index 0000000000000..8c093ff958c66 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/writers/OJAIKeyWriterHelper.scala @@ -0,0 +1,49 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.writers + +import com.mapr.db.spark.codec.BeanCodec +import com.mapr.db.spark.condition.DBQueryCondition +import com.mapr.db.spark.dbclient.DBClient +import org.ojai.{Document, Value} +import com.mapr.db.spark.impl.OJAIDocument +import org.ojai.store.DocumentMutation + +private[spark] sealed trait OJAIValue[T] extends Serializable { + type Self + def getValue(elem: T): Document + def write(doc: Document, getID: (Document) => Value, writer: Writer) + def update(mutation: DocumentMutation, getID: Value, writer: TableUpdateWriter) + def checkAndUpdate(mutation: DocumentMutation, queryCondition: DBQueryCondition, getID: Value, writer: TableCheckAndMutateWriter) +} + +private[spark] object OJAIValue extends BaseOJAIValue { + + implicit def defaultOJAIDocument[T]: OJAIValue[OJAIDocument] = new OJAIValue[OJAIDocument] { + type Self = OJAIDocument + override def getValue(elem: OJAIDocument): Document = elem.getDoc + override def write(doc: Document, getID: (Document)=> Value, writer: Writer) = writer.write(doc, getID(doc)) + override def update(mutation: DocumentMutation, getID: Value, writer: TableUpdateWriter) = writer.write(mutation, getID) + override def checkAndUpdate(mutation: DocumentMutation, queryCondition: DBQueryCondition, getID: Value, writer: TableCheckAndMutateWriter): Unit = + writer.write(mutation, queryCondition, getID) + } +} + +private[spark] trait BaseOJAIValue { + implicit def overrideDefault[T <: AnyRef]: OJAIValue[T] = new OJAIValue[T] { + type Self = AnyRef + override def getValue(elem: T): Document = BeanCodec.decode(DBClient().newDocumentBuilder(), elem) + override def write(doc: Document, getID: (Document) => Value, writer: Writer) = writer.write(doc, getID(doc)) + override def update(mutation: DocumentMutation, getID: Value, writer: TableUpdateWriter) = writer.write(mutation, getID) + override def checkAndUpdate(mutation: DocumentMutation, queryCondition: DBQueryCondition, getID: Value, writer: TableCheckAndMutateWriter): Unit = + writer.write(mutation, queryCondition, getID) + } + + def overrideJavaDefault[T <: AnyRef]: OJAIValue[T] = new OJAIValue[T] { + type Self = AnyRef + override def getValue(elem: T): Document = org.ojai.beans.BeanCodec.decode(DBClient().newDocumentBuilder(), elem) + override def write(doc: Document, getID: (Document) => Value, writer: Writer) = writer.write(doc, getID(doc)) + override def update(mutation: DocumentMutation, getID: Value, writer: TableUpdateWriter) = writer.write(mutation, getID) + override def checkAndUpdate(mutation: DocumentMutation, queryCondition: DBQueryCondition, getID: Value, writer: TableCheckAndMutateWriter): Unit = + writer.write(mutation, queryCondition, getID) + } +} diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/writers/OJAIValueWriterHelper.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/writers/OJAIValueWriterHelper.scala new file mode 100644 index 0000000000000..8924f9a441b7b --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/writers/OJAIValueWriterHelper.scala @@ -0,0 +1,46 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.writers + +import java.nio.ByteBuffer + +import com.mapr.db.spark.condition.DBQueryCondition +import com.mapr.db.spark.types.DBBinaryValue +import org.ojai.Document +import org.ojai.store.DocumentMutation + +private[spark] sealed trait OJAIKey[T] extends Serializable { + type Self + def getValue(elem: T) : Self + def write(doc: Document, key: Self, table: Writer) + def update(mutation: DocumentMutation, key: Self, table: TableUpdateWriter) + def checkAndMutate(mutation: DocumentMutation, queryCondition: DBQueryCondition, key: Self, table: TableCheckAndMutateWriter) +} + +private[spark] object OJAIKey { + implicit val ojaiStringKey = new OJAIKey[String] { + override type Self = String + override def getValue(elem: String) = elem + override def write(doc: Document, key: String, table: Writer) = table.write(doc, key) + override def update(mutation: DocumentMutation, key: String, table: TableUpdateWriter) = table.write(mutation, key) + override def checkAndMutate(mutation: DocumentMutation, queryCondition: DBQueryCondition, key: String, table: TableCheckAndMutateWriter): Unit = + table.write(mutation, queryCondition, key) + } + + implicit val ojaibytebufferKey = new OJAIKey[ByteBuffer] { + override type Self = ByteBuffer + override def getValue(elem: ByteBuffer) = elem + override def write(doc: Document, key: ByteBuffer, table: Writer) = table.write(doc,key) + override def update(mutation: DocumentMutation, key: ByteBuffer, table: TableUpdateWriter) = table.write(mutation,key) + override def checkAndMutate(mutation: DocumentMutation, queryCondition: DBQueryCondition, key: ByteBuffer, table: TableCheckAndMutateWriter): Unit = + table.write(mutation, queryCondition, key) + } + + implicit val ojaibinaryKey = new OJAIKey[DBBinaryValue] { + override type Self = ByteBuffer + override def getValue(elem: DBBinaryValue) = elem.getByteBuffer() + override def write(doc: Document, key: ByteBuffer, table: Writer) = table.write(doc,key) + override def update(mutation: DocumentMutation, key: ByteBuffer, table: TableUpdateWriter) = table.write(mutation,key) + override def checkAndMutate(mutation: DocumentMutation, queryCondition: DBQueryCondition, key: ByteBuffer, table: TableCheckAndMutateWriter): Unit = + table.write(mutation, queryCondition, key) + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/writers/TableWriter.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/writers/TableWriter.scala new file mode 100644 index 0000000000000..88e7f75061559 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/writers/TableWriter.scala @@ -0,0 +1,89 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.writers + +import java.nio.ByteBuffer + +import org.ojai.Document +import com.mapr.db.rowcol.DBValueBuilderImpl +import com.mapr.db.spark.condition.DBQueryCondition +import org.ojai.store.{DocumentMutation, DocumentStore} + +private[spark] case class TableInsertOrReplaceWriter(@transient table: DocumentStore) extends Writer { + + def write(doc: Document, key: ByteBuffer) = { + write(doc,DBValueBuilderImpl.KeyValueBuilder.initFrom(key)) + } + + def write(doc: Document, key: String) = { + write(doc, DBValueBuilderImpl.KeyValueBuilder.initFrom(key)) + } + + def write(doc: Document, key: org.ojai.Value) = { + table.insertOrReplace(doc.setId(key)) + } + + def close() = { + table.flush() + table.close() + } +} + +private[spark] case class TableInsertWriter(@transient table: DocumentStore) extends Writer { + + def write(doc: Document, key: ByteBuffer) = { + write(doc, DBValueBuilderImpl.KeyValueBuilder.initFrom(key)) + } + + def write(doc: Document, key: String) = { + write(doc, DBValueBuilderImpl.KeyValueBuilder.initFrom(key)) + } + + def write(doc: Document, key: org.ojai.Value) = { + table.insert(doc.setId(key)) + } + + def close() = { + table.flush() + table.close() + } +} + +private[spark] case class TableUpdateWriter(@transient table: DocumentStore) { + + def write(mutation: DocumentMutation, key: ByteBuffer) : Unit = { + write(mutation, DBValueBuilderImpl.KeyValueBuilder.initFrom(key)) + } + + def write(mutation: DocumentMutation, key: String) : Unit = { + write(mutation, DBValueBuilderImpl.KeyValueBuilder.initFrom(key)) + } + + def write(mutation: DocumentMutation, key: org.ojai.Value) : Unit = { + table.update(key, mutation) + } + + def close() = { + table.flush() + table.close() + } +} + +private[spark] case class TableCheckAndMutateWriter(@transient table: DocumentStore) { + + def write(mutation: DocumentMutation, queryCondition: DBQueryCondition, key: ByteBuffer) : Unit = { + write(mutation, queryCondition, DBValueBuilderImpl.KeyValueBuilder.initFrom(key)) + } + + def write(mutation: DocumentMutation, queryCondition: DBQueryCondition, key: String) : Unit = { + write(mutation, queryCondition, DBValueBuilderImpl.KeyValueBuilder.initFrom(key)) + } + + def write(mutation: DocumentMutation, queryCondition: DBQueryCondition, key: org.ojai.Value) : Unit = { + table.checkAndMutate(key, queryCondition.condition, mutation) + } + + def close() = { + table.flush() + table.close() + } +} \ No newline at end of file diff --git a/external/maprdb/src/main/scala/com/mapr/db/spark/writers/Writer.scala b/external/maprdb/src/main/scala/com/mapr/db/spark/writers/Writer.scala new file mode 100644 index 0000000000000..f1687f98b6cd3 --- /dev/null +++ b/external/maprdb/src/main/scala/com/mapr/db/spark/writers/Writer.scala @@ -0,0 +1,35 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.spark.writers + +import org.ojai.Document +import java.nio.ByteBuffer +import org.apache.hadoop.fs.Path +import com.mapr.db.mapreduce.BulkLoadRecordWriter +import com.mapr.db.spark.configuration.SerializableConfiguration +import com.mapr.db.spark.dbclient.DBClient + +private[spark] trait Writer extends Serializable { + + def write(doc: Document, key: ByteBuffer) + + def write(doc: Document, key: String) + + def write(doc: Document, key: org.ojai.Value) + + def close() +} + +private[spark] object Writer { + def initialize(tableName: String, + serializableConfiguration: SerializableConfiguration, + bulkInsert: Boolean, insertOrReplace : Boolean): Writer = { + + if (!bulkInsert) { + if (insertOrReplace) + new TableInsertOrReplaceWriter(DBClient().getTable(tableName)) + else new TableInsertWriter(DBClient().getTable(tableName)) + } + else new BulkTableWriter( + new BulkLoadRecordWriter(serializableConfiguration.value, new Path(tableName))) + } +} diff --git a/external/maprdb/src/test/scala/com/mapr/db/testCases/LoadAndSaveTests.scala b/external/maprdb/src/test/scala/com/mapr/db/testCases/LoadAndSaveTests.scala new file mode 100644 index 0000000000000..ce34d7c1f5c00 --- /dev/null +++ b/external/maprdb/src/test/scala/com/mapr/db/testCases/LoadAndSaveTests.scala @@ -0,0 +1,382 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.testCases + +import java.nio.ByteBuffer +import java.nio.charset.Charset + +import org.apache.spark.{SparkConf, SparkContext} +import org.ojai.types.ODate +import com.mapr.db.spark.field +import com.mapr.db.spark._ +import com.mapr.db.spark.types.DBBinaryValue + +import scala.language.implicitConversions +import com.mapr.db.MapRDB +import org.apache.spark.rdd.RDD + + +object LoadAndSaveTests { + val tableName = "/tmp/user_profiles_load_and_save_read" + val saveToTable = "/tmp/user_profiles_load_and_save_write" + lazy val conf = new SparkConf() + .setMaster("local[*]") + .setAppName("simpletest") + .set("spark.executor.memory", "1g") + .set("spark.driver.memory", "1g") + + def main(args: Array[String]): Unit = { + val sc = new SparkContext(conf) + tableInitialization(sc, tableName) + runTests(sc, false) + } + + def tableInitialization(sparkSession: SparkContext, tableName: String) = { + if (MapRDB.tableExists(tableName)) + MapRDB.deleteTable(tableName) + val docs = scala.collection.immutable.List("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\"," + + "\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\",\"electronics\"],\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}") + val docsrdd = sparkSession.parallelize(docs) + val ojairdd = docsrdd.map(doc => MapRDBSpark.newDocument(doc)).map(doc => { + doc.dob = Option(doc.dob) match { + case Some(a) => ODate.parse(doc.dob[String]) + case None => null + } + doc + }) + ojairdd.saveToMapRDB(tableName,createTable = true) + } + + def runTests(sparkSession: SparkContext, kryoOn: Boolean): Unit = { + testingSimpleLoadTable(sparkSession,tableName) + testingLoadTableWithSpecificColumns(sparkSession,tableName) + testingLoadTableWithWhereEQCondition(sparkSession, tableName) + testingLoadTableWithWhereEQAndSelectClause(sparkSession, tableName) + testingLoadTableWithWhereEQConditionAndSave(sparkSession, tableName) + testingSimpleSaveTable(sparkSession,tableName,saveToTable) + testingIDwithSaveToMapRDB(sparkSession, tableName, saveToTable+"id") + testingBulkSaveMode(sparkSession,tableName,saveToTable) + testingBulkSaveWithoutBulkModeSetInTable(sparkSession, tableName, saveToTable) + testingBulkSaveModeBeanClass(sparkSession,tableName,saveToTable) + testingSplitPartitioner(sparkSession, tableName, saveToTable) + testingSplitPartitionerWithBinaryData(sparkSession, tableName, saveToTable) + if (kryoOn) + testingSplitPartitionerWithByteBufferData(sparkSession, tableName, saveToTable) + } + + + def testingSimpleLoadTable(sc: SparkContext, tableName: String) = { + val cust_profiles = sc.loadFromMapRDB(tableName).collect + if (cust_profiles.map(a => a.asJsonString()).toSet.equals( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"],\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}"))) { + println("testingSimpleLoadTable succeeded") + true + } else { + println("testingSimpleLoadTable failed") + false + } + } + + def testingLoadTableWithSpecificColumns(sc: SparkContext, tableName: String) = { + val cust_profiles = sc.loadFromMapRDB(tableName).select("first_name", "address", "interests").collect + + if (cust_profiles.map(a => a.asJsonString()).toSet.equals( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"]}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"]}", + "{\"_id\":\"jdoe\",\"first_name\":\"John\"}", + "{\"_id\":\"dsimon\",\"first_name\":\"David\"}", + "{\"_id\":\"alehmann\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"]}"))) { + println("testingLoadTableWithSpecificColumns succeeded") + true + } else { + println("testingLoadTableWithSpecificColumns failed") + false + } + } + + def testingLoadTableWithWhereEQCondition(sc: SparkContext, tableName: String) = { + val cust_profiles = sc.loadFromMapRDB(tableName).where(field("address.city") === "San Jose").collect + + if (cust_profiles.map(a => a.asJsonString()).toSet.equals( + Set("{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"],\"last_name\":\"Dupont\"}"))) { + println("testingLoadTableWithWhereEQCondition succeeded") + true + } else { + println("testingLoadTableWithWhereEQCondition failed") + false + } + } + + def testingLoadTableWithWhereEQConditionAndSave(sc: SparkContext, tableName: String) = { + if (MapRDB.tableExists(tableName+"save")) + MapRDB.deleteTable(tableName+"save") + val cust_profiles = sc.loadFromMapRDB(tableName).where(field("address.city") === "San Jose") + cust_profiles.saveToMapRDB(tableName+"save", createTable= true) + val cust_profiles1 = sc.loadFromMapRDB(tableName+"save").collect + if (cust_profiles1.map(a => a.asJsonString()).toSet.equals( + Set("{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"],\"last_name\":\"Dupont\"}"))) { + println("testingLoadTableWithWhereEQConditionAndSave succeeded") + true + } else { + println("testingLoadTableWithWhereEQConditionAndSave failed") + false + } + } + + def testingLoadTableWithWhereEQAndSelectClause(sc: SparkContext, tableName: String) = { + val cust_profiles = sc.loadFromMapRDB(tableName).where(field("address.city") === "San Jose").select("address", "first_name", "interests").collect + + if (cust_profiles.map(a => a.asJsonString()).toSet.equals( + Set("{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"]}"))) { + println("testingLoadTableWithWhereEQAndSelectClause succeeded") + true + } else { + println("testingLoadTableWithWhereEQAndSelectClause failed") + false + } + } + + def testingSimpleSaveTable(sc: SparkContext, tableName: String, saveToTable: String) = { + if (MapRDB.tableExists(saveToTable)) + MapRDB.deleteTable(saveToTable) + val cust_profiles = sc.loadFromMapRDB(tableName).map(a => a.setId(a.first_name[String])).keyBy(a => a.first_name[String]). + partitionBy(MapRDBSpark.newPartitioner[String](tableName)) + cust_profiles.saveToMapRDB(saveToTable, createTable = true) + val saved_profiles = sc.loadFromMapRDB(saveToTable).collect + if (saved_profiles.map(a => (a.getIdString,a.asJsonString)).toSet.equals( + Set(("Robert","{\"_id\":\"Robert\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\",\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}"), + ("Maxime","{\"_id\":\"Maxime\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\",\"electronics\"],\"last_name\":\"Dupont\"}"), + ("John","{\"_id\":\"John\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}"), + ("David","{\"_id\":\"David\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}"), + ("Andrew","{\"_id\":\"Andrew\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}")))) { + println("testingSimpleSaveTable succeeded") + true + } else { + println("testingSimpleSaveTable failed") + saved_profiles.map(a => (a.getIdString, a.asJsonString)).foreach(println(_)) + false + } + } + + + def testingIDwithSaveToMapRDB(sc: SparkContext, tableName: String, saveToTable: String) = { + if (MapRDB.tableExists(saveToTable)) + MapRDB.deleteTable(saveToTable) + val cust_profiles = sc.loadFromMapRDB(tableName) + cust_profiles.saveToMapRDB(saveToTable, idFieldPath = "first_name", createTable = true) + val saved_profiles = sc.loadFromMapRDB(saveToTable).collect + if (saved_profiles.map(a => (a.getIdString,a.asJsonString)).toSet.equals( + Set(("Robert","{\"_id\":\"Robert\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\",\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}"), + ("Maxime","{\"_id\":\"Maxime\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\",\"electronics\"],\"last_name\":\"Dupont\"}"), + ("John","{\"_id\":\"John\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}"), + ("David","{\"_id\":\"David\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}"), + ("Andrew","{\"_id\":\"Andrew\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}")))) { + println("testingIDwithSaveToMapRDB succeeded") + true + } else { + println("testingIDwithSaveToMapRDB failed") + false + } + } + + def testingBulkSaveMode(sc: SparkContext, tableName: String, saveToTable: String) = { + if (MapRDB.tableExists(saveToTable+"bulk")) + MapRDB.deleteTable(saveToTable+"bulk") + val cust_profiles = sc.loadFromMapRDB(tableName).keyBy(a => a.first_name[String]). + repartitionAndSortWithinPartitions(MapRDBSpark.newPartitioner[String](tableName)) + cust_profiles.saveToMapRDB(saveToTable+"bulk", bulkInsert = true, createTable = true) + val saved_profiles = sc.loadFromMapRDB(saveToTable+"bulk").collect + if (saved_profiles.map(a => (a.getIdString,a.asJsonString)).toSet.equals( + Set(("Robert","{\"_id\":\"Robert\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\",\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}"), + ("Maxime","{\"_id\":\"Maxime\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\",\"electronics\"],\"last_name\":\"Dupont\"}"), + ("John","{\"_id\":\"John\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}"), + ("David","{\"_id\":\"David\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}"), + ("Andrew","{\"_id\":\"Andrew\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}")))) { + println("testingBulkSaveMode succeeded") + true + } else { + println("testingBulkSaveMode failed") + saved_profiles.map(a => (a.getIdString, a.asJsonString)).foreach(println(_)) + false + } + } + + def testingBulkSaveWithoutBulkModeSetInTable(sc: SparkContext, tableName: String, saveToTable: String) = { + if (MapRDB.tableExists(saveToTable+"bulkmodeswitch")) + MapRDB.deleteTable(saveToTable+"bulkmodeswitch") + val tabDesc = MapRDB.newTableDescriptor() + tabDesc.setAutoSplit(true) + tabDesc.setPath(saveToTable+"bulkmodeswitch") + tabDesc.setBulkLoad(false) + MapRDB.newAdmin().createTable(tabDesc) + val cust_profiles = sc.loadFromMapRDB(tableName).keyBy(a => a.first_name[String]). + repartitionAndSortWithinPartitions(MapRDBSpark.newPartitioner[String](tableName)) + cust_profiles.saveToMapRDB(saveToTable+"bulkmodeswitch", bulkInsert = true, createTable = false) + val saved_profiles = sc.loadFromMapRDB(saveToTable+"bulkmodeswitch").collect + if (saved_profiles.map(a => (a.getIdString,a.asJsonString)).toSet.equals( + Set(("Robert","{\"_id\":\"Robert\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\",\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}"), + ("Maxime","{\"_id\":\"Maxime\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\",\"electronics\"],\"last_name\":\"Dupont\"}"), + ("John","{\"_id\":\"John\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}"), + ("David","{\"_id\":\"David\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}"), + ("Andrew","{\"_id\":\"Andrew\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}")))) { + println("testingBulkSaveMode succeeded") + true + } else { + println("testingBulkSaveMode failed") + saved_profiles.map(a => (a.getIdString, a.asJsonString)).foreach(println(_)) + false + } + } + + def testingSimpleSaveModeBeanClass(sc: SparkContext, tableName: String, saveToTable: String) = { + if (MapRDB.tableExists(saveToTable+"simplebean")) + MapRDB.deleteTable(saveToTable+"simplebean") + val cust_profiles: RDD[BeanTest.User1] = sc.loadFromMapRDB[BeanTest.User1](tableName)//.map(user => BeanTest.User1(user.id,user.firstName,Option(1000),user.dob,user.interests)) + cust_profiles.saveToMapRDB(saveToTable+"simplebean", createTable = true) + val saved_profiles = sc.loadFromMapRDB(saveToTable+"simplebean").collect + if (saved_profiles.map(a => (a.getIdString, a.asJsonString())).toSet.sameElements( + Set(("rsmith","{\"_id\":\"rsmith\",\"dob\":\"1982-02-03\",\"first_name\":\"Robert\",\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}"), + ("mdupont","{\"_id\":\"mdupont\",\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\",\"electronics\"],\"last_name\":\"Dupont\"}"), + ("jdoe","{\"_id\":\"jdoe\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"interests\":null,\"last_name\":\"Doe\"}"), + ("dsimon","{\"_id\":\"dsimon\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"interests\":null,\"last_name\":\"Simon\"}"), + ("alehmann","{\"_id\":\"alehmann\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}") ))) + { + println("testingSimpleSaveModeBeanClass succeded") + true + } + else { + println("testingSimpleSaveModeBeanClass failed") + saved_profiles.map(a => (a.getIdString, a.asJsonString)).foreach(println(_)) + false + } + } + + def testingBulkSaveModeBeanClass(sc: SparkContext, tableName: String, saveToTable: String) = { + if (MapRDB.tableExists(saveToTable+"bulkbean")) + MapRDB.deleteTable(saveToTable+"bulkbean") + val cust_profiles = sc.loadFromMapRDB[BeanTest.User](tableName).keyBy(a => a.firstName). + repartitionAndSortWithinPartitions(MapRDBSpark.newPartitioner[String](tableName)) + cust_profiles.saveToMapRDB(saveToTable+"bulkbean", bulkInsert = true, createTable = true) + val saved_profiles = sc.loadFromMapRDB(saveToTable+"bulkbean").collect + if (saved_profiles.map(a => (a.getIdString, a.asJsonString())).toSet.sameElements( + Set(("Robert","{\"_id\":\"Robert\",\"dob\":\"1982-02-03\",\"first_name\":\"Robert\",\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}"), + ("Maxime","{\"_id\":\"Maxime\",\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\",\"electronics\"],\"last_name\":\"Dupont\"}"), + ("John","{\"_id\":\"John\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"interests\":null,\"last_name\":\"Doe\"}"), + ("David","{\"_id\":\"David\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"interests\":null,\"last_name\":\"Simon\"}"), + ("Andrew","{\"_id\":\"Andrew\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}") ))) + { + println("testingBulkSaveModeBeanClass succeded") + true + } + else { + println("testingBulkSaveModeBeanClass failed") + saved_profiles.map(a => (a.getIdString, a.asJsonString)).foreach(println(_)) + false + } + } + + def testingSplitPartitioner(sc: SparkContext, tableName: String, saveToTable: String) = { + if (MapRDB.tableExists(saveToTable+"splitpartitioner")) + MapRDB.deleteTable(saveToTable+"splitpartitioner") + val cust_profiles = sc.loadFromMapRDB(tableName).keyBy(a => a.`first_name`[String]). + repartitionAndSortWithinPartitions(MapRDBSpark.newPartitioner[String](Seq( "bolo","chalo","hello","zebra"))) + cust_profiles.saveToMapRDB(saveToTable+"splitpartitioner", bulkInsert = true, createTable = true) + val saved_profiles = sc.loadFromMapRDB(saveToTable+"splitpartitioner").collect + if (saved_profiles.map(a => (a.getIdString,a.asJsonString)).toSet.equals( + Set(("Robert","{\"_id\":\"Robert\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\",\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}"), + ("Maxime","{\"_id\":\"Maxime\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\",\"electronics\"],\"last_name\":\"Dupont\"}"), + ("John","{\"_id\":\"John\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}"), + ("David","{\"_id\":\"David\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}"), + ("Andrew","{\"_id\":\"Andrew\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}")))) { + println("testingSplitPartitioner succeded") + true + } + else { + println("testingSplitPartitioner failed") + saved_profiles.map(a => (a.getIdString, a.asJsonString)).foreach(println(_)) + false + } + } + + def testingSplitPartitionerWithBinaryData(sc: SparkContext, tableName: String, saveToTable: String) = { + if (MapRDB.tableExists(saveToTable+"SplitpartitionerWithBinary")) + MapRDB.deleteTable(saveToTable+"SplitpartitionerWithBinary") + + val cust_profiles = sc.loadFromMapRDB(tableName).keyBy(a => MapRDBSpark.serializableBinaryValue(ByteBuffer.wrap(a.`first_name`[String].getBytes(Charset.forName("UTF-8"))))). + repartitionAndSortWithinPartitions(MapRDBSpark.newPartitioner(Seq(MapRDBSpark.serializableBinaryValue(ByteBuffer.wrap("Bolo".getBytes( Charset.forName("UTF-8" )))), + MapRDBSpark.serializableBinaryValue(ByteBuffer.wrap("zebra".getBytes( Charset.forName("UTF-8" ))))))) + cust_profiles.saveToMapRDB(saveToTable+"SplitpartitionerWithBinary", bulkInsert = true, createTable = true) + val saved_profiles = sc.loadFromMapRDB(saveToTable+"SplitpartitionerWithBinary").collect + if (saved_profiles.map(a => a.delete("_id")).map(a => a.asJsonString).toSet.equals( + Set("{\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\",\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}", + "{\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\",\"electronics\"],\"last_name\":\"Dupont\"}", + "{\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}", + "{\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}", + "{\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}"))) { + println("testingSplitPartitionerWithBinaryData succeded") + true + } + else { + println("testingSplitPartitionerWithBinaryData failed") + saved_profiles.map(a => (a.getIdBinary, a.asJsonString)).foreach(println(_)) + false + } + } + + def testingSplitPartitionerWithByteBufferData(sc: SparkContext, tableName: String, saveToTable: String) = { + if (MapRDB.tableExists(saveToTable+"SplitpartitionerWithByteBuffer")) + MapRDB.deleteTable(saveToTable+"SplitpartitionerWithByteBuffer") + + val cust_profiles = sc.loadFromMapRDB(tableName).keyBy(a => ByteBuffer.wrap(a.`first_name`[String].getBytes(Charset.forName("UTF-8")))). + repartitionAndSortWithinPartitions(MapRDBSpark.newPartitioner[ByteBuffer](Seq(MapRDBSpark.serializableBinaryValue(ByteBuffer.wrap("Bolo".getBytes( Charset.forName("UTF-8" )))).getByteBuffer(), + MapRDBSpark.serializableBinaryValue(ByteBuffer.wrap("zebra".getBytes( Charset.forName("UTF-8" )))).getByteBuffer()))) + cust_profiles.saveToMapRDB(saveToTable+"SplitpartitionerWithByteBuffer", bulkInsert = true, createTable = true) + val saved_profiles = sc.loadFromMapRDB(saveToTable+"SplitpartitionerWithByteBuffer").collect + if (saved_profiles.map(a=> a.delete("_id")).map(a => a.asJsonString).toSet.equals( + Set("{\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\",\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}", + "{\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\",\"electronics\"],\"last_name\":\"Dupont\"}", + "{\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}", + "{\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}", + "{\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}"))) { + println("testingSplitPartitionerWithByteBufferData succeded") + true + } + else { + println("testingSplitPartitionerWithByteBufferData failed") + saved_profiles.map(a => (a.getIdBinary, a.asJsonString)).foreach(println(_)) + false + } + } +} + +object LoadAndSaveTestsWithKryo { + val tableName = "/tmp/user_profiles_load_and_save_read_kryo" + val saveToTable = "/tmp/user_profiles_load_and_save_write_kryo" + lazy val conf = new SparkConf() + .setAppName("simpletest") + .set("spark.executor.memory", "1g") + .set("spark.driver.memory", "1g") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "com.mapr.db.spark.OJAIKryoRegistrator") + + def main(args: Array[String]): Unit = { + var sc = new SparkContext(conf) + LoadAndSaveTests.tableInitialization(sc, tableName) + LoadAndSaveTests.runTests(sc, true) + } +} diff --git a/external/maprdb/src/test/scala/com/mapr/db/testCases/MaprDBSparkTests.scala b/external/maprdb/src/test/scala/com/mapr/db/testCases/MaprDBSparkTests.scala new file mode 100644 index 0000000000000..66d32797111eb --- /dev/null +++ b/external/maprdb/src/test/scala/com/mapr/db/testCases/MaprDBSparkTests.scala @@ -0,0 +1,1004 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.testCases + +import org.ojai.types.ODate +import java.nio.ByteBuffer +import java.nio.charset.Charset + +import com.fasterxml.jackson.annotation.{JsonIgnoreProperties, JsonProperty} +import com.mapr.db.MapRDB +import com.mapr.db.spark.field +import com.mapr.db.spark._ +import org.apache.spark.SparkContext +import org.apache.spark.SparkConf +import org.ojai.FieldPath +import com.mapr.db.MapRDB +import com.mapr.db.spark.impl.OJAIDocument +import com.mapr.db.testCases.BeanTest.User + +object MapRDBSparkTests { + val tableName = "/tmp/user_profiles_temp" + val saveToTable = "/tmp/user_profiles_temp_save" + lazy val conf = new SparkConf() + .setAppName("simpletest") + .set("spark.executor.memory","1g") + .set("spark.driver.memory", "1g") + .setMaster("local[*]") + def main(args: Array[String]): Unit = { + var sc = new SparkContext(conf) + tableInitialization(sc, tableName) + runTests(sc) + } + + def tableInitialization(sparkSession: SparkContext, tableName: String) = { + if (MapRDB.tableExists(tableName)) + MapRDB.deleteTable(tableName) + val docs = scala.collection.immutable.List("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\"," + + "\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\",\"electronics\"],\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}") + val docsrdd = sparkSession.parallelize(docs) + val ojairdd = docsrdd.map(doc => MapRDBSpark.newDocument(doc)).map(doc => { + doc.dob = Option(doc.dob) match { + case Some(a) => ODate.parse(doc.dob[String]) + case None => null + } + doc + }) + ojairdd.saveToMapRDB(tableName, createTable = true) + } + + def runTests(sparkSession: SparkContext): Unit = { + testEmptySet(sparkSession, tableName) + testShakespearesFamousQuote(sparkSession, tableName) + testBeanClassParsing(sparkSession, tableName) + testIntegerZipCodes(sparkSession, tableName) + testingMapRDBBeanClass(sparkSession, tableName) + testMaprDBBeanClassWithCollect(sparkSession, tableName) + testGroupByOnODate(sparkSession, tableName) + testingLoadingOfOJAIDocuments(sparkSession, tableName) + testingCoGroupWithOJAIDOCUMENT(sparkSession, tableName) + testingArrayValueWithOJAIDocument(sparkSession, tableName) + testingCoGroupArrayKey(sparkSession, tableName) + testingCoGroupOnMapValue(sparkSession, tableName) + testingCoGroupOnMapKey(sparkSession, tableName) + testingCoGroupWithMapKeyWithFilter(sparkSession, tableName) + testingAssignmentOfZipCodeToOJAIDocument(sparkSession, tableName) + testingAccessOfFieldsOfOJAIDocumentWithParametricTypes(sparkSession, tableName) + testingAccessOfFieldsOfOJAIDocumentWithParametricTypes2(sparkSession, tableName) + testingAccessOfProjectedFields(sparkSession, tableName) + testingAccessOfProjectedFieldPaths(sparkSession, tableName) + testingOfSavingTheProcessedOJAIDocuments(sparkSession, tableName) + testingMapAsaValue(sparkSession, tableName) + testingMapAsaKey(sparkSession, tableName) + testingArrayAsaValue(sparkSession, tableName) + testingArrayAsaKey(sparkSession, tableName) + testingOJAIDocumentParsingFunctionality(sparkSession, tableName) + testingMultipleDataTypesInOJAIDocument(sparkSession, tableName) +// testingMultipleDataTypesInOJAIDocumentAndTypeCasting(sparkSession, tableName) + testingSingleDataTypeDoubleInOJAIAndTypeCasting(sparkSession, tableName) + testingTupleOutputOfAnRDD(sparkSession, tableName) + testingAddingCountryInAddressField(sparkSession, tableName) + testingBinaryDataTypeInOJAIDocument(sparkSession, tableName) + testingMapTypeInOJAIDocument(sparkSession, tableName) + testingDateTypeInOJAIDocument(sparkSession, tableName) + testingSaveWithAnyObject(sparkSession, tableName) + testingFilterFunctionOnMapObject(sparkSession, tableName) + testingFilterFunctionOnArrayObject(sparkSession, tableName) + testingFilterFunctionOnArrayObjectFunctionalway(sparkSession, tableName) + testingWhereClauseOnloadFromMapRDB(sparkSession, tableName) + testingPartitionOnloadFromMapRDB(sparkSession, tableName) + testingAssignmentOfDocument(sparkSession, tableName) + testCaseWithFlatMap(sparkSession, tableName) + testingBulkJoinWithRDD(sparkSession, tableName) + testingJoinWithRDDBean(sparkSession, tableName) + testingJoinWithRDD(sparkSession, tableName) + testingJoinWithOjaiRDDBean(sparkSession, tableName) + testingUpdateMapRDBTable(sparkSession, tableName) + } + + def testEmptySet(sparkSession: SparkContext, tableName: String) = { + val lines = Array("") + val wordCounts :Seq[WordCount] = WordCount.count(sparkSession, sparkSession.parallelize(lines)).collect() + if (wordCounts.equals(Seq.empty)) { + println("testEmptySet succeeded") + true + } + else { + println("testEmptySet failed") + false + } + } + + def testShakespearesFamousQuote(sparkSession: SparkContext, tableName: String) = { + val lines = Seq("To be or not to be.", "That is the question.") + val stopWords = Set("the") + val wordCounts = WordCount.count(sparkSession, sparkSession.parallelize(lines), stopWords).collect() + if (wordCounts.sameElements(Array( + WordCount("be", 2), + WordCount("is", 1), + WordCount("not", 1), + WordCount("or", 1), + WordCount("question", 1), + WordCount("that", 1), + WordCount("to", 2)))) { + println("testShakespearesFamousQuote succeded") + true + } + else { + println("testShakespearesFamousQuote failed") + false + } + } + + def testBeanClassParsing(sparkSession: SparkContext, tableName: String) = { + + import com.fasterxml.jackson.databind.ObjectMapper + import com.fasterxml.jackson.module.scala.DefaultScalaModule + import java.io.StringWriter + + val person = BeanTest.person("fred", 25) + val mapper = new ObjectMapper() + mapper.registerModule(DefaultScalaModule) + + val out = new StringWriter + mapper.writeValue(out, person) + + val json = out.toString + + val person2 = mapper.readValue[BeanTest.person](json, classOf[BeanTest.person]) + if (person.equals(person2)) { + println("testBeanClassParsing succeded") + true + } + else { + println("testBeanClassParsing failed") + false + } + } + + def testIntegerZipCodes(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val collection = maprd.map(a => a.`address.zip`[Integer]).collect + if (collection.toSet.equals(Set(94105, 95109, null, null, null))) { + println("testIntegerZipCodes succeded") + true + } + else { + println("testIntegerZipCodes failed") + false + } + } + + def testingMapRDBBeanClass(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB[BeanTest.User](tableName) + val collection = maprd.collect() + if (collection.map(a => a.toString).toSet.equals(Set("User(rsmith,Robert,Smith,1982-02-03,List(electronics, music, sports))", + "User(mdupont,Maxime,Dupont,1982-02-03,List(sports, movies, electronics))", + "User(jdoe,John,Doe,1970-06-23,null)", + "User(dsimon,David,Simon,1980-10-13,null)", + "User(alehmann,Andrew,Lehmann,1980-10-13,List(html, css, js))"))) + { + println("testingMapRDBBeanClass succeded") + true + } + else { + println("testingMapRDBBeanClass failed") + collection.map(a=> a.toString).foreach(print(_)) + false + } + } + + def testMaprDBBeanClassWithCollect(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB[BeanTest.User](tableName) + val collection = maprd.map(a => a.toString).collect + if (collection.toSet.equals(Set("User(rsmith,Robert,Smith,1982-02-03,List(electronics, music, sports))", + "User(mdupont,Maxime,Dupont,1982-02-03,List(sports, movies, electronics))", + "User(jdoe,John,Doe,1970-06-23,null)", + "User(dsimon,David,Simon,1980-10-13,null)", + "User(alehmann,Andrew,Lehmann,1980-10-13,List(html, css, js))"))) + { + println("testMaprDBBeanClassWithCollect succeded") + true + } + else { + println("testMaprDBBeanClassWithCollect failed") + collection.map(a=> a.toString).foreach(print(_)) + false + } + } + + def testGroupByOnODate(sparkSession: SparkContext, tableName : String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val aggregatedondate = maprd.map(a => (a.dob[ODate],a)).groupByKey().map(a => (a._1, a._2.size)) + val collection = aggregatedondate.collect + + if (collection.toSet.sameElements(Set((ODate.parse("1982-02-03"),2),(ODate.parse("1980-10-13"), 2),(ODate.parse("1970-06-23"),1)))) + { + println("testGroupByOnODate succeded") + true + } + else { + println("testGroupByOnODate failed") + false + } + } + + def testingLoadingOfOJAIDocuments(sparkSession: SparkContext, tableName: String) = { + + val maprd = sparkSession.loadFromMapRDB(tableName) + val collection = maprd.collect + if (collection.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"],\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}"))) + { + println("testingLoadingOfOJAIDocuments succeded") + true + } + else { + println("testingLoadingOfOJAIDocuments failed") + false + } + } + + def testingCoGroupWithOJAIDOCUMENT(sparkSession: SparkContext, tableName: String) = { + val maprd1 = sparkSession.loadFromMapRDB(tableName) + val maprd2 = sparkSession.loadFromMapRDB(tableName) + val collection = maprd1.map(a => (a.`address.zip`[Integer], a)) + .cogroup(maprd2.map(a => (a.`address.zip`[Integer], a))) + .map(a => (a._1, a._2._1.size, a._2._2.size)) + .collect + if (collection.sameElements(Set((null, 3, 3), (94105, 1, 1), (95109, 1, 1)))) + { + println("testingCoGroupWithOJAIDocument succeded") + true + } + else { + println("testingCoGroupWithOJAIDocument failed") + false + } + } + + def testingArrayValueWithOJAIDocument(sparkSession: SparkContext, tableName: String) = { + val maprd1 = sparkSession.loadFromMapRDB(tableName) + val maprd2 = sparkSession.loadFromMapRDB(tableName) + val collection = maprd1.map(a => (a.`address.zip`[Integer], a)) + .cogroup(maprd2.map(a => (a.`address.zip`[Integer], a.interests))) + .map(a => (a._1, a._2._1.size, a._2._2.size)) + .collect + if (collection.toSet.sameElements(Set((null, 3, 3), (94105, 1, 1), (95109, 1, 1)))) + { + println("testingArrayValueWithOJAIDocument succeded") + true + } + else { + println("testingArrayValueWithOJAIDocument failed") + false + } + } + + def testingCoGroupArrayKey(sparkSession: SparkContext, tableName: String) = { + val maprd1 = sparkSession.loadFromMapRDB(tableName) + val maprd2 = sparkSession.loadFromMapRDB(tableName) + val collection = maprd1.map(a => (a.interests, a)) + .cogroup(maprd2.map(a => (a.interests, a.interests))) + .map(a => (a._1, a._2._1.size, a._2._2.size)) + .collect + if (collection.toSet.equals(Set((null, 2, 2), (Seq("html","css","js"), 1, 1), (Seq("sports", "movies", "electronics"), 1, 1), (Seq("electronics", "music", + "sports"),1,1)))) + { + println("testingCoGroupArrayKey succeded") + true + } + else { + println("testingCoGroupArrayKey failed") + false + } + } + + def testingCoGroupOnMapValue(sparkSession: SparkContext, tableName:String) = { + val maprd1 = sparkSession.loadFromMapRDB(tableName) + val maprd2 = sparkSession.loadFromMapRDB(tableName) + val collection = maprd1.map(a => (a.`address.zip`[Integer], a)) + .cogroup(maprd2.map(a => (a.`address.zip`[Integer], a.address))) + .map(a => (a._1, a._2._1.size, a._2._2.size)) + .collect + if (collection.toSet.sameElements(Set((null, 3, 3), (94105, 1, 1), (95109, 1, 1)))) + { + println("testingCoGroupOnMapValue succeded") + true + } + else { + println("testingCoGroupOnMapValue failed") + false + } + } + + def testingCoGroupOnMapKey(sparkSession: SparkContext, tableName: String) = { + val maprd1 = sparkSession.loadFromMapRDB(tableName) + val maprd2 = sparkSession.loadFromMapRDB(tableName) + val collection = maprd1.map(a => (a.address, a)).cogroup(maprd2.map(a => (a.address, a.address))).map(a => (a._1, a._2._1.size, a._2._2.size)) + .collect + if (collection.map(a => a.toString()).toSet.equals(Set("(null,3,3)","(Map(city -> San " + + "Francisco, line -> 100 Main Street, zip -> 94105.0),1,1)","(Map(city -> San Jose, line -> 1223 Broadway, zip -> 95109.0),1,1)"))) + { + println("testingCoGroupOnMapKey succeded") + true + } + else { + println("testingCoGroupOnMapKey failed") + false + } + } + + def testingCoGroupWithMapKeyWithFilter(sparkSession: SparkContext, tableName: String) = { + val maprd1 = sparkSession.loadFromMapRDB(tableName) + val maprd2 = sparkSession.loadFromMapRDB(tableName) + val collection = maprd1.map(a => (a.address, a)) + .cogroup(maprd2.map(a => (a.address, a.address))) + .map(a => (a._1, a._2._1.size, a._2._2.size)) + .filter(a => a._1!= null) + .collect + if (collection.map(a => a.toString()).toSet.equals(Set("(Map(city -> San Francisco, line -> 100 Main Street, zip -> 94105.0),1,1)","(Map(city -> San Jose, line -> 1223 " + + "Broadway, zip -> 95109.0),1,1)"))) + { + println("testingCoGroupWithMapKeyWithFilter succeded") + true + } + else { + println("testingCoGroupWithMapKeyWithFilter failed") + false + } + } + + def testingAssignmentOfZipCodeToOJAIDocument(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val collection = maprd.map(a => { a.`address.zip` = 95035 ; a }).collect + if (collection.map(a => a.`address.zip`).toSet.sameElements(Set(95035,95035,95035,95035,95035))) + { + println("testingAssignmentOfZipCodeToOJAIDocument succeded") + true + } + else { + println("testingAssignmentOfZipCodeToOJAIDocument failed") + false + } + } + + def testingAccessOfFieldsOfOJAIDocumentWithParametricTypes(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val collection = maprd.map(a => a.`address.zip`).collect + + if(collection.toSet.equals(Set(94105,95109, null, null, null))) + { + println("testingAccessOfFieldsOfOJAIDocumentWithParametricTypes succeded") + true + } + else { + println("testingAccessOfFieldsOfOJAIDocumentWithParametricTypes failed") + false + } + } + + def testingAccessOfFieldsOfOJAIDocumentWithParametricTypes2(sparkSession: SparkContext, tableName: String) = { + + val maprd = sparkSession.loadFromMapRDB(tableName) + val collection = maprd.map(a => a.`address.zip`[Integer]).collect + if (collection.toSet.equals(Set(94105,95109, null, null, null))) + { + println("testingAccessOfFieldsOfOJAIDocumentWithParametricTypes2 succeded") + true + } + else { + println("testingAccessOfFieldsOfOJAIDocumentWithParametricTypes2 failed") + false + } + } + + def testingAccessOfProjectedFields(sparkSession: SparkContext, tableName: String) = { + + val maprd = sparkSession.loadFromMapRDB(tableName).select("address", "_id", "first_name") + val collection = maprd.collect() + if (collection.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"first_name\":\"Robert\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"first_name\":\"Maxime\"}", + "{\"_id\":\"jdoe\",\"first_name\":\"John\"}", + "{\"_id\":\"dsimon\",\"first_name\":\"David\"}", + "{\"_id\":\"alehmann\",\"first_name\":\"Andrew\"}"))) + { + println("testingAccessOfProjectedFields succeded") + true + } + else { + println("testingAccessOfProjectedFields failed") + false + } + } + + def testingAccessOfProjectedFieldPaths(sparkSession: SparkContext, tableName: String) = { + + val maprd = sparkSession.loadFromMapRDB(tableName).select(FieldPath.parseFrom("address"), FieldPath.parseFrom("_id"), FieldPath.parseFrom("first_name")) + val collection = maprd.collect() + if (collection.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"first_name\":\"Robert\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"first_name\":\"Maxime\"}", + "{\"_id\":\"jdoe\",\"first_name\":\"John\"}", + "{\"_id\":\"dsimon\",\"first_name\":\"David\"}", + "{\"_id\":\"alehmann\",\"first_name\":\"Andrew\"}"))) + { + println("testingAccessOfProjectedFieldPaths succeded") + true + } + else { + println("testingAccessOfProjectedFieldPaths failed") + false + } + } + + def testingOfSavingTheProcessedOJAIDocuments(sparkSession: SparkContext, tableName: String) = { + if (MapRDB.tableExists("/tmp/temp_user_profiles")) + MapRDB.deleteTable("/tmp/temp_user_profiles") + val maprd = sparkSession.loadFromMapRDB(tableName).select("address", "id", "first_name") + maprd.saveToMapRDB("/tmp/temp_user_profiles", createTable = true) + val savedDocumentsRDD = sparkSession.loadFromMapRDB("/tmp/temp_user_profiles") + val collection = savedDocumentsRDD.collect() + if (collection.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"first_name\":\"Robert\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"first_name\":\"Maxime\"}", + "{\"_id\":\"jdoe\",\"first_name\":\"John\"}", + "{\"_id\":\"dsimon\",\"first_name\":\"David\"}", + "{\"_id\":\"alehmann\",\"first_name\":\"Andrew\"}"))) + { + println("testingOfSavingTheProcessedOJAIDocuments succeded") + true + } + else { + println("testingOfSavingTheProcessedOJAIDocuments failed") + false + } + } + + def testingMapAsaValue(sparkSession: SparkContext, tableName : String) = { + + val maprd = sparkSession.loadFromMapRDB(tableName) + val collection = maprd.map(a => a.address).collect + if (collection.map(a => { Option(a).map(a => a.asInstanceOf[Map[String, Any]]).getOrElse(null)}).toSet.equals(Set(Map("city" -> "San Francisco", "line" -> "100 Main Street", + "zip" -> 94105), Map("city" -> "San Jose", "line" -> "1223 Broadway", "zip" -> 95109), null, null, null))) { + println("testingMapAsaValue succeded") + true + } else { + println("testingMapAsaValue failed") + false + } + } + + def testingMapAsaKey(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val collection = maprd.map(a => (a.address,a.first_name)).collect + val newcollection = collection.map(a => Option(a._1).map(a => a.asInstanceOf[Map[String, AnyRef]]).getOrElse(null)).toSet + + if (newcollection.equals(Set(null, Map("city" -> "San Jose", + "line" -> "1223 Broadway", "zip" -> 95109), + Map("city" -> "San Francisco", "line" -> "100 Main Street", "zip" -> 94105), null, null))) { + println("testingMapAsaKey succeded") + true + } else { + println("testingMapAsaKey failed") + false + } + } + + def testingArrayAsaValue(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val collection: Seq[Object] = maprd.map(a => a.interests).collect + if (collection.map(a => { if (a !=null) a.asInstanceOf[Seq[AnyRef]] else null}).toSet.equals(Set(Seq("electronics","music","sports"),Seq("sports", + "movies", + "electronics"),Seq("html", "css","js"), null, null))) + { + println("testingArrayAsaValue succeded") + true + } + else { + println("testingArrayAsaValue failed") + false + } + } + + def testingArrayAsaKey(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val collection = maprd.map(a => (a.interests,a)).collect + + if (collection.map(a => { if (a._1 != null) a._1.asInstanceOf[Seq[AnyRef]] else null}).toSet.equals(Set(Seq("electronics","music","sports"),Seq("sports","movies", + "electronics"),Seq("html", "css","js"), null, null))) { + println("testingArrayAsaKey succeded") + true + } else { + println("testingArrayAsaKey failed") + false + } + } + + def testingOJAIDocumentParsingFunctionality(sparkSession: SparkContext, tableName: String) = { + if (MapRDB.tableExists("/tmp/testingOJAIDocumentParsingFunctionality")) + MapRDB.deleteTable("/tmp/testingOJAIDocumentParsingFunctionality") + val documents = sparkSession.parallelize((1 to 10).map( i => s"{\42_id\42 : \42$i\42, \42test\42: $i}")) + + val maprd = documents.map(a => MapRDBSpark.newDocument(a)) + maprd.saveToMapRDB("/tmp/testingOJAIDocumentParsingFunctionality", createTable = true) + + val saveddocs = sparkSession.loadFromMapRDB("/tmp/testingOJAIDocumentParsingFunctionality") + val collection = saveddocs.map(a => a.test).collect() + if (collection.toSet.equals(Set(1.0,2.0,3.0,4.0,5.0,6.0,7.0,8.0,9.0,10.0))) + { + println("testingOJAIDocumentParsingFunctionality succeded") + true + } + else { + println("testingOJAIDocumentParsingFunctionality failed") + false + } + } + + def testingMultipleDataTypesInOJAIDocument(sparkSession: SparkContext, tableName: String) = { + val documents = sparkSession.parallelize((1 to 10).map( i => { + if (i < 5) s"{\42test\42: $i}" + else s"{\42test\42: \42$i\42}" + })) + + val maprd = documents.map(a => MapRDBSpark.newDocument(a)) + val collection = maprd.map(a => a.test).collect + + if (collection.toSet.equals(Set("8", "9", 1.0, "5", "10", 2.0, "6", 3.0, "7", 4.0))) + { + println("testingMultipleDataTypesInOJAIDocument succeded") + true + } + else { + println("testingMultipleDataTypesInOJAIDocument failed") + false + } + } + + def testingSingleDataTypeDoubleInOJAIAndTypeCasting(sparkSession: SparkContext, tableName : String) = { + val documents = sparkSession.parallelize((1 to 10).map( i => { + s"{\42test\42: $i.$i}" + })) + + val maprd = documents.map(a => MapRDBSpark.newDocument(a)) + val collection = maprd.map(a => a.test[Integer]).collect + if (collection.toSet.equals(Set(8, 9, 1, 5, 10, 2, 6, 3, 7, 4))) + { + println("testingSingleDataTypeDoubleInOJAIAndTypeCasting succeded") + true + } + else { + println("testingSingleDataTypeDoubleInOJAIAndTypeCasting failed") + false + } + } + + def testingTupleOutputOfAnRDD(sparkSession: SparkContext, tableName : String) = { + val documents = sparkSession.loadFromMapRDB(tableName) + val collection = documents.map(a => (a.`address.city`,a.`address.zip`,a.first_name, a.last_name)).collect() + if(collection.toSet.equals(Set(("San Francisco",94105,"Robert","Smith"), + ("San Jose",95109,"Maxime","Dupont"), (null,null,"John","Doe"), (null,null,"David","Simon"), (null, null,"Andrew","Lehmann")))) + { + println("testingTupleOutputOfAnRDD succeded") + true + } + else { + println("testingTupleOutputOfAnRDD failed") + false + } + } + + def testingAddingCountryInAddressField(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val documents = maprd.map(a => { a.`address.country` = "USA"; a.`address.street` = 10; a}).map(a => a.`address.street`[Integer]).collect + if (documents.toSet.sameElements(Set(10,10, 10, 10, 10))) + { + println("testingAddingCountryInAddressField succeded") + true + } + else { + println("testingAddingCountryInAddressField failed") + false + } + } + + def testCaseWithFlatMap(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val documents = maprd.flatMap(a => Array(a.`address`)).collect + if (documents.toSet.equals(Set(Map("city" -> "San Francisco", "line" -> "100 Main Street", + "zip" -> 94105), Map("city" -> "San Jose", "line" -> "1223 Broadway", "zip" -> 95109), null, null, null))) + { + println("testingAddingCountryInAddressField succeded") + true + } + else { + println("testingAddingCountryInAddressField failed") + documents.foreach(println(_)) + false + } + } + + def testingBinaryDataTypeInOJAIDocument(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val buff: ByteBuffer = ByteBuffer.wrap("Non-Immigrant".getBytes( Charset.forName("UTF-8" ))) + val str: String = new String( buff.array(), Charset.forName("UTF-8") ) + val dBBinaryValue = MapRDBSpark.serializableBinaryValue(buff) + val map = Map("hello"-> "h", "bolo" -> "b") + val documents = maprd.map(a => { a.`address.country` = "USA"; a.user_profile = dBBinaryValue; a }) + .map(a => a.getBinarySerializable("user_profile")).collect + val chars: Seq[String] = documents.map(a=> new String(a.array(), Charset.forName("UTF-8"))) + if (chars.sameElements(Seq("Non-Immigrant","Non-Immigrant", "Non-Immigrant", "Non-Immigrant", + "Non-Immigrant"))) + { + println("testingBinaryDataTypeInOJAIDocument succeded") + true + } + else { + println("testingBinaryDataTypeInOJAIDocument failed") + false + } + } + + def testingMapTypeInOJAIDocument(sparkSession: SparkContext, tableName : String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val buff: ByteBuffer = ByteBuffer.wrap("Non-Immigrant".getBytes( Charset.forName("UTF-8" ))) + val str: String = new String( buff.array(), Charset.forName("UTF-8") ) + val dBBinaryValue = MapRDBSpark.serializableBinaryValue(buff) + val map = Map("hello"-> "h", + "bolo" -> "b") + + val documents = maprd.map(a => {a.user_profile = Seq("A","b"); a }) + .map(a => a.`user_profile`).take(5) + if (documents.map(a => a.asInstanceOf[Seq[AnyRef]]).sameElements(Array(Seq("A", "b"), Seq("A", "b"), Seq("A", "b"), Seq("A", "b"), Seq("A", "b")))) + { + println("testingMapTypeInOJAIDocument succeded") + true + } + else { + println("testingMapTypeInOJAIDocument failed") + false + } + } + + def testingDateTypeInOJAIDocument(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val buff: ByteBuffer = ByteBuffer.wrap("Non-Immigrant".getBytes( Charset.forName("UTF-8" ))) + val str: String = new String( buff.array(), Charset.forName("UTF-8") ) + val dBBinaryValue = MapRDBSpark.serializableBinaryValue(buff) + val documents = maprd.map(a => { a.`address.country` = "USA"; a.user_profile = ODate.parse("2011-10-10"); a }) + .map(a => a.user_profile).collect + if (documents.toSet.sameElements(Set(ODate.parse("2011-10-10"), ODate.parse("2011-10-10"), ODate.parse("2011-10-10"), ODate.parse("2011-10-10"), ODate.parse("2011-10-10")))) + { + println("testingDateTypeInOJAIDocument succeded") + true + } + else { + println("testingDateTypeInOJAIDocument failed") + false + } + } + + def testingSaveWithAnyObject(sparkSession: SparkContext, tableName: String) = { + if (MapRDB.tableExists("/tmp/testingSaveWithAnyObject")) + MapRDB.deleteTable("/tmp/testingSaveWithAnyObject") + val maprd = sparkSession.loadFromMapRDB[BeanTest.User](tableName) + maprd.saveToMapRDB("/tmp/testingSaveWithAnyObject", bulkInsert = true, createTable = true) + val savedocuments = sparkSession.loadFromMapRDB("/tmp/testingSaveWithAnyObject").collect + if (savedocuments.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"dob\":\"1982-02-03\",\"first_name\":\"Robert\",\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\",\"electronics\"],\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"interests\":null,\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"interests\":null,\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}") )) + { + println("testingSaveWithAnyObject succeded") + true + } + else { + println("testingSaveWithAnyObject failed") + false + } + } + + def testingFilterFunctionOnMapObject(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val useraddress = maprd.map(a => a.address[Map[String,AnyRef]]).filter(a => a!= null && a.get("city").get.equals("San Jose")).collect + if (useraddress.map(a => a.toSet).toSet.sameElements(Set(Set("city" -> "San Jose", "line" -> "1223 Broadway", "zip" -> 95109)))) + { + println("testingFilterFunctionOnMapObject succeded") + true + } + else { + println("testingFilterFunctionOnMapObject failed") + false + } + } + + def testingFilterFunctionOnArrayObject(sparkSession: SparkContext, tableName : String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val useraddress = maprd.map(a => a.interests[Seq[AnyRef]]).filter(a => a!= null && a(0) == "sports").collect + if (useraddress.map(a => a).toSet.sameElements(Set(Seq("sports", "movies", "electronics")))) + { + println("testingFilterFunctionOnArrayObject succeded") + true + } + else { + println("testingFilterFunctionOnArrayObject failed") + false + } + } + + def testingFilterFunctionOnArrayObjectFunctionalway(sparkSession: SparkContext, tableName : String) = { + val maprd = sparkSession.loadFromMapRDB(tableName) + val useraddress = maprd.map(a => a.interests[Seq[AnyRef]]).filter(a => Option(a).map(b => b(0) == "sports").getOrElse(false)).collect + if (useraddress.map(a => a).toSet.sameElements(Set(Seq("sports", "movies", "electronics")))) + { + println("testingFilterFunctionOnArrayObjectFunctionalWay succeded") + true + } + else { + println("testingFilterFunctionOnArrayObjectFunctionalWay failed") + useraddress.foreach(println(_)) + false + } + } + + def testingWhereClauseOnloadFromMapRDB(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("address.city") === "San Jose").collect + if (maprd.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"],\"last_name\":\"Dupont\"}"))) + { + println("testingWhereClauseOnloadFromMapRDB succeded") + true + } + else { + println("testingWhereClauseOnloadFromMapRDB failed") + false + } + } + + def testingPartitionOnloadFromMapRDB(sparkSession: SparkContext, tableName: String) = { + if (MapRDB.tableExists(saveToTable)) + MapRDB.deleteTable(saveToTable) + val maprd = sparkSession.loadFromMapRDB(tableName).keyBy(a => a.first_name[String]) + .repartitionAndSortWithinPartitions(MapRDBSpark.newPartitioner[String](tableName)).values.saveToMapRDB(saveToTable, createTable = true) + val collection = sparkSession.loadFromMapRDB(saveToTable).collect + if (collection.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"],\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}"))) + { + println("testingPartitionOnloadFromMapRDB succeded") + true + } + else { + println("testingPartitionOnloadFromMapRDB failed") + false + } + } + + def testingAssignmentOfDocument(sparkSession: SparkContext, tableName: String) = { + if (MapRDB.tableExists(saveToTable+"assigningdoc")) + MapRDB.deleteTable(saveToTable+"assigningdoc") + val maprd = sparkSession.loadFromMapRDB(tableName).keyBy(a => a.first_name[String]).mapValues( a => a.set("recursivedoc", MapRDBSpark.newDocument(a.asJsonString))) + .repartitionAndSortWithinPartitions(MapRDBSpark.newPartitioner[String](tableName)).values.saveToMapRDB(saveToTable+"assigningdoc", createTable= true) + val collection = sparkSession.loadFromMapRDB(saveToTable+"assigningdoc").collect + if (collection.map(a => a.delete("recursivedoc")).map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"],\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}"))) + { + println("testingAssignmentOfDocument succeded") + true + } + else { + println("testingAssignmentOfDocument failed") + collection.map(a => a.asJsonString).foreach(println(_)) + false + } + } + + def testingJoinWithRDD(sparkSession: SparkContext, tableName: String) = { + val valuesRDD = sparkSession.parallelize(Seq("jdoe","Hon","rsmith")) + val documents = valuesRDD.joinWithMapRDB(tableName).collect + if (documents.map(doc => doc._id).toSet.sameElements(Set("jdoe","rsmith"))) + { + println("testingJoinWithRDD succeded") + true + } + else { + println("testingJoinWithRDD failed") + documents.map(a => a.asJsonString).foreach(println(_)) + false + } + } + + def testingBulkJoinWithRDD(sparkSession: SparkContext, tableName: String) = { + val valuesRDD = sparkSession.parallelize(Seq("jdoe","Hon","rsmith")) + val documents = valuesRDD.bulkJoinWithMapRDB(tableName).collect + if (documents.map(doc => doc._id).toSet.sameElements(Set("jdoe","rsmith"))) + { + println("testingBulkJoinWithRDD succeded") + true + } + else { + println("testingBulkJoinWithRDD failed") + documents.map(a => a.asJsonString).foreach(println(_)) + false + } + } + + def testingJoinWithRDDBean(sparkSession: SparkContext, tableName: String) = { + val valuesRDD = sparkSession.parallelize(Seq("jdoe","Hon","rsmith")) + val documents = valuesRDD.joinWithMapRDB[User](tableName).collect + if (documents.map(doc => doc.id).toSet.sameElements(Set("jdoe","rsmith"))) + { + println("testingJoinWithRDDBean succeded") + true + } + else { + println("testingJoinWithRDDBean failed") + documents.map(a => a.toString).foreach(println(_)) + false + } + } + + def testingJoinWithOjaiRDDBean(sparkSession: SparkContext, tableName: String) = { + val valuesRDD = sparkSession.loadFromMapRDB(tableName).map(_.getString("_id")) + val documents = valuesRDD.joinWithMapRDB[User](tableName).collect + if (documents.map(doc => doc.id).toSet.sameElements(Set("jdoe","rsmith","alehmann", "mdupont", "dsimon"))) + { + println("testingJoinWithOjaiRDDBean succeded") + true + } + else { + println("testingJoinWithOjaiRDDBean failed") + documents.map(a => a.toString).foreach(println(_)) + false + } + } + + def testingUpdateMapRDBTable(sparkSession: SparkContext, tableName: String) = { + if (MapRDB.tableExists(tableName+"updateMapRDB")) + MapRDB.deleteTable(tableName+"updateMapRDB") + sparkSession.loadFromMapRDB(tableName).saveToMapRDB(tableName+"updateMapRDB", createTable = true) + val valuesRDD = sparkSession.loadFromMapRDB(tableName+"updateMapRDB") + valuesRDD.updateToMapRDB(tableName+"updateMapRDB", (doc: OJAIDocument) => { + val mutation = MapRDB.newMutation() + mutation.setOrReplace("key", doc.getIdString()) + mutation + }, (doc: OJAIDocument) => doc.getId) + val collection = sparkSession.loadFromMapRDB(tableName+"updateMapRDB").collect + if (collection.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"],\"key\":\"rsmith\",\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"],\"key\":\"mdupont\",\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"key\":\"jdoe\",\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"key\":\"dsimon\",\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"key\":\"alehmann\",\"last_name\":\"Lehmann\"}"))) + { + println("testingUpdateMapRDBTable succeded") + true + } + else { + println("testingUpdateMapRDBTable failed") + collection.map(a => a.asJsonString).foreach(println(_)) + false + } + } + + def testingCheckAndUpdateMapRDBTable(sparkSession: SparkContext, tableName: String) = { + if (MapRDB.tableExists(tableName+"checkAndUpdateMapRDB")) + MapRDB.deleteTable(tableName+"checkAndUpdateMapRDB") + sparkSession.loadFromMapRDB(tableName).saveToMapRDB(tableName+"checkAndUpdateMapRDB", createTable = true) + val valuesRDD = sparkSession.loadFromMapRDB(tableName+"checkAndUpdateMapRDB") + valuesRDD.updateToMapRDB(tableName+"checkAndUpdateMapRDB", (doc: OJAIDocument) => { + val mutation = MapRDB.newMutation() + mutation.setOrReplace("key", doc.getIdString()) + mutation + }, (doc: OJAIDocument) => doc.getId, field("_id") === "rsmith") + val collection = sparkSession.loadFromMapRDB(tableName+"checkAndUpdateMapRDB").collect + if (collection.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"],\"key\":\"rsmith\",\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"],\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}"))) + { + println("testingCheckAndUpdateMapRDBTable succeded") + true + } + else { + println("testingCheckAndUpdateMapRDBTable failed") + collection.map(a => a.asJsonString).foreach(println(_)) + false + } + } + + def testingUpdateMapRDBTablePairedRDD(sparkSession: SparkContext, tableName: String) = { + if (MapRDB.tableExists(tableName+"updateMapRDB")) + MapRDB.deleteTable(tableName+"updateMapRDB") + sparkSession.loadFromMapRDB(tableName).saveToMapRDB(tableName+"updateMapRDB", createTable = true) + val valuesRDD = sparkSession.loadFromMapRDB(tableName+"updateMapRDB").keyBy(_.getIdString) + valuesRDD.updateToMapRDB(tableName+"updateMapRDB", (doc: OJAIDocument) => { + val mutation = MapRDB.newMutation() + mutation.setOrReplace("key", doc.getIdString()) + mutation + }) + val collection = sparkSession.loadFromMapRDB(tableName+"updateMapRDB").collect + if (collection.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"],\"key\":\"rsmith\",\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"],\"key\":\"mdupont\",\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"key\":\"jdoe\",\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"key\":\"dsimon\",\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"key\":\"alehmann\",\"last_name\":\"Lehmann\"}"))) + { + println("testingUpdateMapRDBTablePairedRDD succeded") + true + } + else { + println("testingUpdateMapRDBTablePairedRDD failed") + collection.map(a => a.asJsonString).foreach(println(_)) + false + } + } +} + +object BeanTest { + case class person(name: String, age: Integer) + + @JsonIgnoreProperties(ignoreUnknown = true) + case class User (@JsonProperty("_id") id: String, @JsonProperty("first_name") firstName:String, + @JsonProperty("last_name") lastName: String, @JsonProperty("dob") dob: ODate, + @JsonProperty("interests") interests: Seq[String]) + + @JsonIgnoreProperties(ignoreUnknown = true) + case class User1 (@JsonProperty("_id") id: String, @JsonProperty("first_name") firstName:String, + @JsonProperty("last_name") lastName: Option[Int], @JsonProperty("dob") dob: ODate, + @JsonProperty("interests") interests: Seq[String]) + +} + + +object MapRDBSparkTestsWithKryo { + val tableName = "/tmp/user_profiles_temp" + val saveToTable = "/tmp/user_profiles_temp_save" + lazy val conf = new SparkConf() + .setAppName("simpletest") + .set("spark.executor.memory", "1g") + .set("spark.driver.memory", "1g") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "com.mapr.db.spark.OJAIKryoRegistrator") + + def main(args: Array[String]): Unit = { + val sc = new SparkContext(conf) + MapRDBSparkTests.tableInitialization(sc, tableName) + MapRDBSparkTests.runTests(sc) + } +} diff --git a/external/maprdb/src/test/scala/com/mapr/db/testCases/OjaiDocumentAccessTesting.scala b/external/maprdb/src/test/scala/com/mapr/db/testCases/OjaiDocumentAccessTesting.scala new file mode 100644 index 0000000000000..4876735a35302 --- /dev/null +++ b/external/maprdb/src/test/scala/com/mapr/db/testCases/OjaiDocumentAccessTesting.scala @@ -0,0 +1,773 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.testCases + +import java.util + +import org.apache.spark.rdd.RDD +import java.nio.ByteBuffer +import com.mapr.db.spark.MapRDBSpark +import com.mapr.db.spark.types.DBBinaryValue +import org.apache.spark.{SparkConf, SparkContext} +import com.mapr.db.rowcol.DBDocumentImpl +import org.ojai.types._ + + +object OjaiDocumentAccessTesting { + lazy val conf = new SparkConf() + .setAppName("simpletest") + .set("spark.executor.memory","1g") + .set("spark.driver.memory", "1g") + + def main(args: Array[String]): Unit = { + var sc = new SparkContext(conf) + runTests(sc) + } + + def runTests(sparkSession: SparkContext): Unit = { + testGetterFuncForInt(sparkSession) + testGetterFuncForByte(sparkSession) + testGetterFuncForString(sparkSession) + testGetterFuncForShort(sparkSession) + testGetterFuncForLong(sparkSession) + testGetterFuncForFloat(sparkSession) + testGetterFuncForDouble(sparkSession) + testGetterFuncForTime(sparkSession) + testGetterFuncForDate(sparkSession) + testGetterFuncForTimeStamp(sparkSession) + testGetterFuncForBinary(sparkSession) + testGetterFuncForList(sparkSession) + testGetterFuncForMap(sparkSession) + testGetterFuncForIntExpl(sparkSession) + testGetterFuncForByteExpl(sparkSession) + testGetterFuncForStringExpl(sparkSession) + testGetterFuncForShortExpl(sparkSession) + testGetterFuncForLongExpl(sparkSession) + testGetterFuncForFloatExpl(sparkSession) + testGetterFuncForDoubleExpl(sparkSession) + testGetterFuncForTimeExpl(sparkSession) + testGetterFuncForDateExpl(sparkSession) + testGetterFuncForTimeStampExpl(sparkSession) + testGetterFuncForBinaryExpl(sparkSession) + testGetterFuncForArrayExpl(sparkSession) + testGetterFuncForMapExpl(sparkSession) + testSetterFuncForInt(sparkSession) + testSetterFuncForByte(sparkSession) + testSetterFuncForString(sparkSession) + testSetterFuncForShort(sparkSession) + testSetterFuncForLong(sparkSession) + testSetterFuncForFloat(sparkSession) + testSetterFuncForDouble(sparkSession) + testSetterFuncForTime(sparkSession) + testSetterFuncForDate(sparkSession) + testSetterFuncForTimeStamp(sparkSession) + testSetterFuncForBinary(sparkSession) + testSetterFuncForBinaryWithByteArr(sparkSession) + testSetterFuncForList(sparkSession) + testSetterFuncForMap(sparkSession) + testGetterNoDataCase(sparkSession) + testSetterNullToDoc(sparkSession) + testSetterFuncForMapStringInt(sparkSession) + testNonDynamicSetterFuncForInt(sparkSession) + } + + private def createOJAIDocumentsRDD(sc: SparkContext) = { + val rec: DBDocumentImpl = new DBDocumentImpl() + rec.setId(1233456.toString) + rec.set("map.field1", 100.toByte) + rec.set("map.field2", 10000.toShort) + rec.set("map.longfield2verylongverylong", 12.345678) + rec.set("FIELD2", "VERY LONG STRING IS THIS YOU KNOW") + + rec.set("map2.field1", 100.toByte) + rec.set("map2.field2", 10000.toShort) + rec.set("map2.longfield2verylongverylong", 12.345678) + rec.set("FIELD3", "VERY LONG STRING IS THIS YOU KNOW") + val map: util.Map [java.lang.String, Object] = new util.HashMap [java.lang.String, Object]() + map.put("Name", "Aditya") + map.put("Age", new Integer(20)) + rec.set("map.map", map) + rec.set("map.boolean", false) + rec.set("map.string", "string") + rec.set("map.byte", 100.toByte) + rec.set("map.short", 10000.toShort) + rec.set("map.int", new Integer(5000)) + rec.set("map.long", 12345678999L) + rec.set("map.float", 10.1234f) + rec.set("map.double", 10.12345678910d) + // rec.set("map.interval", new OInterval(1000)) + rec.set("map.time", new OTime(1000)) + rec.set("map.date", new ODate(1000)) + rec.set("map.timestamp", new OTimestamp(1000)) + // rec.set("map.decimal", new java.math.BigDecimal("1000000000.11111111111111111111")) + + val bytes: Array[Byte] = Array.range(1,10).map(_.toByte) + rec.set("map.binary1", bytes) + rec.set("map.binary2", bytes, 1, 3) + val bbuf: ByteBuffer = ByteBuffer.allocate(100) + (1 to 100).foldLeft[ByteBuffer](bbuf)((buf, a) => buf.put(a.toByte)) + rec.set("map.binary3", bbuf) + + val values = new util.ArrayList[Object]() + values.add("Field1") + val intvalue: Integer = 500 + values.add(intvalue) + values.add(new java.lang.Double(5555.5555)) + rec.set("map.list", values) + val idstring : String = rec.getIdAsString + val doc = MapRDBSpark.newDocument(rec) + val ojairdd = sc.parallelize(List(doc)) + ojairdd + } + + /* All the following functions will test the functionality of accessing the data with corresponding data type from a ojai document*/ + def testGetterFuncForInt(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.int`).collect + if (result.sameElements(Array(5000))) + { + println("testGetterFuncForInt succeeded") + true + } + else + { + println("testGetterFuncForInt failed") + false + } + } + + def testGetterFuncForByte(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.byte`).collect + if (result.sameElements(Array(100.toByte))) + { + println("testGetterFuncForByte succeeded") + true + } + else + { + println("testGetterFuncForByte failed") + false + } + } + + def testGetterFuncForString(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.string`).collect + if (result.sameElements(Array("string"))) + { + println("testGetterFuncForString succeeded") + true + } + else + { + println("testGetterFuncForString falied") + false + } + } + + def testGetterFuncForShort(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.short`).collect + if (result.sameElements(Array(10000.toShort))) + { + println("testGetterFuncForShort succeeded") + true + } + else + { + println("testGetterFuncForShort falied") + false + } + } + + def testGetterFuncForLong(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.long`).collect + if (result.sameElements(Array(12345678999L))) + { + println("testGetterFuncForLong succeeded") + true + } + else + { + println("testGetterFuncForLong falied") + false + } + } + + def testGetterFuncForFloat(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.float`).collect + if (result.sameElements(Array(10.1234f))) + { + println("testGetterFuncForFloat succeeded") + true + } + else + { + println("testGetterFuncForFloat failed") + false + } + } + + def testGetterFuncForDouble(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.double`).collect + if (result.sameElements(Array(10.12345678910d))) + { + println("testGetterFuncForDouble succeeded") + true + } + else + { + println("testGetterFuncForDouble failed") + false + } + } + + def testGetterFuncForTime(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.time`).collect + if (result.sameElements(Array(new OTime(1000)))) + { + println("testGetterFuncForTime succeeded") + true + } + else { + println("testGetterFuncForTime failed") + false + } + } + + def testGetterFuncForDate(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.date`).collect + if (result.sameElements(Array(new ODate(1000)))) + { + println("testGetterFuncForDate succeeded") + true + } + else { + println("testGetterFuncForDate failed") + false + } + } + + def testGetterFuncForTimeStamp(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.timestamp`).collect + if (result.sameElements(Array(new OTimestamp(1000)))) + { + println("testGetterFuncForTimeStamp succeeded") + true + } + else { + println("testGetterFuncForTimeStamp failed") + false + } + } + + def testGetterFuncForBinary(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.getBinarySerializable("map.binary1")).collect + if (result.map(a => a.array().toSet).sameElements(Array(Set(1, 2, 3, 4, 5, 6, 7, 8, 9)))) + { + println("testGetterFuncForBinary succeeded") + true + } + else { + println("testGetterFuncForBinary failed") + false + } + } + + def testGetterFuncForList(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.list`).collect + if (result.map(a => a.asInstanceOf[Seq[Any]].toSet).sameElements(Array(Set("Field1",500,5555.5555)))) + { + println("testGetterFuncForList succeeded") + true + } + else { + println("testGetterFuncForList failed") + false + } + } + + def testGetterFuncForMap(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.map`).collect + if (result.map(a => a.asInstanceOf[Map[String,Any]].toSet).sameElements(Array(Set("Age" -> 20, "Name" -> "Aditya")))) + { + println("testGetterFuncForMap succeeded") + true + } + else { + println("testGetterFuncForMap failed") + false + } + } + + + + /* getters with explicit type casts*/ + + def testGetterFuncForIntExpl(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.int`[Integer]).collect + if (result.sameElements(Array(5000))) + { + println("testGetterFuncForIntExpl succeeded") + true + } + else { + println("testGetterFuncForIntExpl failed") + false + } + } + + def testGetterFuncForByteExpl(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.byte`[Byte]).collect + if (result.sameElements(Array(100.toByte))) + { + println("testGetterFuncForByteExpl succeeded") + true + } + else { + println("testGetterFuncForByteExpl failed") + false + } + } + + def testGetterFuncForStringExpl(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.string`[String]).collect + if (result.sameElements(Array("string"))) + { + println("testGetterFuncForStringExpl succeeded") + true + } + else { + println("testGetterFuncForStringExpl failed") + false + } + } + + def testGetterFuncForShortExpl(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.short`[Short]).collect + if (result.sameElements(Array(10000.toShort))) + { + println("testGetterFuncForShortExpl succeeded") + true + } + else { + println("testGetterFuncForShortExpl failed") + false + } + } + + def testGetterFuncForLongExpl(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.long`[Long]).collect + if (result.sameElements(Array(12345678999L))) + { + println("testGetterFuncForLongExpl succeeded") + true + } + else { + println("testGetterFuncForLongExpl failed") + false + } + } + + def testGetterFuncForFloatExpl(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.float`[Float]).collect + if (result.sameElements(Array(10.1234f))) + { + println("testGetterFuncForFloatExpl succeeded") + true + } + else { + println("testGetterFuncForFloatExpl failed") + false + } + } + + def testGetterFuncForDoubleExpl(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.double`[Double]).collect + if (result.sameElements(Array(10.12345678910d))) + { + println("testGetterFuncForDoubleExpl succeeded") + true + } + else { + println("testGetterFuncForDoubleExpl failed") + false + } + } + + def testGetterFuncForTimeExpl(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.time`[OTime]).collect + if (result.sameElements(Array(new OTime(1000)))) + { + println("testGetterFuncForTimeExpl succeeded") + true + } + else { + println("testGetterFuncForTimeExpl failed") + false + } + } + + def testGetterFuncForDateExpl(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.date`[ODate]).collect + if (result.sameElements(Array(new ODate(1000)))) + { + println("testGetterFuncForDateExpl succeeded") + true + } + else { + println("testGetterFuncForDateExpl failed") + false + } + } + + def testGetterFuncForTimeStampExpl(sparkSession : SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.timestamp`[OTimestamp]).collect + if (result.sameElements(Array(new OTimestamp(1000)))) + { + println("testGetterFuncForTimeStampExpl succeeded") + true + } + else { + println("testGetterFuncForTimeStampExpl failed") + false + } + } + + def testGetterFuncForBinaryExpl(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.getBinarySerializable("map.binary1")).collect + if (result.map(a => a.array().toSet).sameElements(Array(Set(1, 2, 3, 4, 5, 6, 7, 8, 9)))) + { + println("testGetterFuncForBinaryExpl succeeded") + true + } + else { + println("testGetterFuncForBinaryExpl failed") + false + } + } + + def testGetterFuncForArrayExpl(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.list`[Seq[Any]]).collect + if (result.map(a => a.toSet).sameElements(Array(Set("Field1",500,5555.5555)))) + { + println("testGetterFuncForArrayExpl succeeded") + true + } + else { + println("testGetterFuncForArrayExpl failed") + false + } + } + + def testGetterFuncForMapExpl(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`map.map`[Map[String,AnyRef]]).collect + if (result.map(a => a.toSet).sameElements(Array(Set("Age" -> 20, "Name" -> "Aditya")))) + { + println("testGetterFuncForMapExpl succeeded") + true + } + else { + println("testGetterFuncForMapExpl failed") + false + } + } + + /* testing the setters functionality for the ojai document*/ + def testSetterFuncForInt(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => { a.`map.int_new` = a.`map.int`; a } ).map(a => a.`map.int_new`).collect + if (result.sameElements(Array(5000))) + { + println("testSetterFuncForInt succeeded") + true + } + else { + println("testSetterFuncForInt failed") + false + } + } + + def testSetterFuncForByte(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => { a.`map.byte_new` = a.`map.byte`; a}).map(a => a.`map.byte_new`).collect + if (result.sameElements(Array(100.toByte))) + { + println("testSetterFuncForByte succeeded") + true + } + else { + println("testSetterFuncForByte failed") + false + } + } + + def testSetterFuncForString(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => { a.`map.string_new` = a.`map.string`; a}).map(a => a.`map.string_new`).collect + if (result.sameElements(Array("string"))) + { + println("testSetterFuncForString succeeded") + true + } + else { + println("testSetterFuncForString failed") + false + } + } + + def testSetterFuncForShort(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => {a.`map.short_new` = a.`map.short`; a}).map(a => a.`map.short_new`).collect + if (result.sameElements(Array(10000.toShort))) + { + println("testSetterFuncForShort succeeded") + true + } + else { + println("testSetterFuncForShort failed") + false + } + } + + def testSetterFuncForLong(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => { a.`map.long_new` = a.`map.long`; a}).map(a => a.`map.long_new`).collect + if (result.sameElements(Array(12345678999L))) + { + println("testSetterFuncForLong succeeded") + true + } + else { + println("testSetterFuncForLong failed") + false + } + } + + def testSetterFuncForFloat(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => { a.`map.float_new` = a.`map.float`; a}).map(a => a.`map.float_new`).collect + if (result.sameElements(Array(10.1234f))) + { + println("testSetterFuncForFloat succeeded") + true + } + else { + println("testSetterFuncForFloat failed") + false + } + } + + def testSetterFuncForDouble(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => {a.`map.double_new` = a.`map.double`; a}).map(a => a.`map.double_new`).collect + if (result.sameElements(Array(10.12345678910d))) + { + println("testSetterFuncForDouble succeeded") + true + } + else { + println("testSetterFuncForDouble failed") + false + } + } + + def testSetterFuncForTime(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => {a.`map.time_new` = a.`map.time`; a}).map(a => a.`map.time_new`).collect + if (result.sameElements(Array(new OTime(1000)))) + { + println("testSetterFuncForTime succeeded") + true + } + else { + println("testSetterFuncForTime failed") + false + } + } + + def testSetterFuncForDate(sparkSession : SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => { a.`map.date_new` = a.`map.date`; a}).map(a => a.`map.date_new`).collect + if (result.sameElements(Array(new ODate(1000)))) + { + println("testSetterFuncForDate succeeded") + true + } + else { + println("testSetterFuncForDate failed") + false + } + } + + def testSetterFuncForTimeStamp(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => { a.`map.timestamp_new` = a.`map.timestamp`; a}).map(a => a.`map.timestamp_new`).collect + if (result.sameElements(Array(new OTimestamp(1000)))) + { + println("testSetterFuncForTimeStamp succeeded") + true + } + else { + println("testSetterFuncForTimeStamp failed") + false + } + } + + def testSetterFuncForBinary(sparkSession:SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val bbuf: ByteBuffer = ByteBuffer.allocate(100) + (1 to 100).foldLeft[ByteBuffer](bbuf)((buf, a) => buf.put(a.toByte)) + val result = ojaiDocumentRDD.map(a => { a.`map.binary1_new` = a.getBinarySerializable("map.binary1"); a}).map(a => a.getBinarySerializable("map.binary1_new")).collect + if (result.map(a => a.array().toSet).sameElements(Array(Set(1, 2, 3, 4, 5, 6, 7, 8, 9)))) + { + println("testSetterFuncForBinary succeeded") + true + } + else { + println("testSetterFuncForBinary failed") + false + } + } + + def testSetterFuncForBinaryWithByteArr(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val bbuf: ByteBuffer = ByteBuffer.allocate(100) + (1 to 100).foldLeft[ByteBuffer](bbuf)((buf, a) => buf.put(a.toByte)) + val result = ojaiDocumentRDD.map(a => { a.`map.binary1_new` = a.getBinarySerializable("map.binary1").array(); a}).map(a => a.getBinarySerializable("map.binary1_new")).collect + if (result.map(a => a.array().toSet).sameElements(Array(Set(1, 2, 3, 4, 5, 6, 7, 8, 9)))) + { + println("testSetterFuncForBinaryWithByteArr succeeded") + true + } + else { + println("testSetterFuncForBinaryWithByteArr failed") + false + } + } + + def testSetterFuncForList(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => { a.`map.list_new` = Seq(1,2,3,4); a}).map(a => a.`map.list_new`).collect + if (result.map(a => a.asInstanceOf[Seq[Any]]).sameElements(Array(List(1,2,3,4)))) + { + println("testSetterFuncForList succeeded") + true + } + else { + println("testSetterFuncForList failed") + false + } + } + + def testSetterFuncForMap(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => { a.`map.map_new` = Map("one" -> 1, "two" -> 2); a}).map(a => a.`map.map_new`).collect + if (result.map(a => a.asInstanceOf[Map[String,Any]].toSet).sameElements(Array(Set("one" -> 1, "two" -> 2)))) + { + println("testSetterFuncForMap succeeded") + true + } + else { + println("testSetterFuncForMap failed") + false + } + } + + /* negative testcases where there exists no data for a particular column in the ojai document*/ + def testGetterNoDataCase(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.`getter`).collect + if (result.sameElements(Array(null))) + { + println("testGetterNoDataCase succeeded") + true + } + else { + println("testGetterNoDataCase failed") + false + } + } + + /* setting a null value to the document*/ + def testSetterNullToDoc(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => { a.`map.double_latest` = null;a }).map(a => a.`map.double_latest`).collect() + if (result.sameElements(Array(null))) + { + println("testSetterNullToDoc succeeded") + true + } + else { + println("testSetterNullToDoc failed") + false + } + } + + /* testing map setter functionality extensively*/ + def testSetterFuncForMapStringInt(sparkSession:SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => { a.`map.map_latest` = Map("India" -> "Delhi", "USA" -> "DC", "Germany" -> "Berlin"); a}).map(a => a.`map.map_latest`).collect() + if (result.map(a => a.asInstanceOf[Map[String,AnyRef]].toSet).sameElements(Array(Set("India" -> "Delhi", "USA" -> "DC", "Germany" -> "Berlin")))) + { + println("testSetterFuncForMapStringInt succeeded") + true + } + else { + println("testSetterFuncForMapStringInt failed") + false + } + } + + /* These tests are with new setter functionality*/ + def testNonDynamicSetterFuncForInt(sparkSession: SparkContext) = { + val ojaiDocumentRDD = createOJAIDocumentsRDD(sparkSession) + val result = ojaiDocumentRDD.map(a => a.set("map.int_new", a.`map.int`[Int]) ).map(a => a.`map.int_new`).collect + if (result.sameElements(Array(5000))) + { + println("testSetterFuncForInt succeeded") + true + } + else { + println("testSetterFuncForInt failed") + false + } + } +} + + +object OjaiDocumentAccessTestingWithKryo { + lazy val conf = new SparkConf() + .setAppName("simpletest") + .set("spark.executor.memory", "1g") + .set("spark.driver.memory", "1g") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "com.mapr.db.spark.OJAIKryoRegistrator") + + def main(args: Array[String]): Unit = { + var sc = new SparkContext(conf) + OjaiDocumentAccessTesting.runTests(sc) + } +} diff --git a/external/maprdb/src/test/scala/com/mapr/db/testCases/PredicateTests.scala b/external/maprdb/src/test/scala/com/mapr/db/testCases/PredicateTests.scala new file mode 100644 index 0000000000000..ebfa8391f37ff --- /dev/null +++ b/external/maprdb/src/test/scala/com/mapr/db/testCases/PredicateTests.scala @@ -0,0 +1,687 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.testCases + +import org.apache.spark.{SparkConf, SparkContext} +import org.ojai.types.{ODate, OTime} +import com.mapr.db.spark._ +import com.mapr.db.spark.field +import org.ojai.Document +import org.apache.spark.rdd.RDD +import org.ojai.exceptions.TypeException +import org.ojai.store.QueryCondition +import com.mapr.db.MapRDB + +object PredicateTests { + val tableName = "/tmp/user_profiles_predicates" + + lazy val conf = new SparkConf() + .setAppName("simpletest") + .set("spark.executor.memory", "1g") + .set("spark.driver.memory", "1g") + + def main(args: Array[String]): Unit = { + var sc = new SparkContext(conf) + tableInitialization(sc, tableName) + runTests(sc) + } + + def tableInitialization(sparkSession: SparkContext, tableName: String) = { + if (MapRDB.tableExists(tableName)) + MapRDB.deleteTable(tableName) + println("table successfully create :" + tableName) + MapRDB.createTable(tableName) + } + + def runTests(sparkSession: SparkContext): Unit = { + testingIdOnlyCondition(sparkSession, tableName) + testingSimpleGTCondition(sparkSession, tableName) + testingNotExistsCondition(sparkSession, tableName) + testingSimpleINCondition(sparkSession, tableName) + testingTYPEOFCondition(sparkSession, tableName) + testingComplexAND_INcondition(sparkSession, tableName) + testingCompositeCondition(sparkSession, tableName) + testingMatchesCondition(sparkSession, tableName) + testingLikeCondition(sparkSession, tableName) + testingThreeConditions(sparkSession, tableName) + testingORCondition(sparkSession, tableName) + testingComplexConditonWithDate(sparkSession, tableName) + testingBetweenCondition(sparkSession, tableName) + testingEqualityConditionOnSeq(sparkSession, tableName) + testingEqualtiyOnMapOfStrings(sparkSession, tableName) + testingEqualityOnMapStringInteger(sparkSession, tableName) + testingNotEqualityOnID(sparkSession, tableName) + testingNotEqualityConditionOnSeq(sparkSession, tableName) + testingNotEqualityOnMapStringInteger(sparkSession, tableName) + testingSizeOf(sparkSession, tableName) + testingSizeOfWithComplexCondition(sparkSession, tableName) + testingTypeOfWithNonExistantType(sparkSession, tableName) + testingWithQueryCondition(sparkSession, tableName) + testWithListINCondition(sparkSession, tableName) + testingSizeOfNotEquals(sparkSession, tableName) + testingINConditionOnSeqwithInSeq(sparkSession, tableName) + } + + def testingIdOnlyCondition(sparkSession : SparkContext , tableName : String) = { + val d: Document = MapRDB.newDocument.set("a.b", 350).set("s", "Holger Way") + MapRDB.getTable(tableName).insertOrReplace("k1", d) + d.set("zip-code", "95134") + MapRDB.getTable(tableName).insertOrReplace("k2", d) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("_id") === "k2").collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + if (maprd.map(doc => doc.`zip-code`[String]).sameElements(Array("95134"))) { + println("testingIdOnlyCondition succeeded") + true + } else { + println("testingIdOnlyCondition failed") + false + } + } + + def testingSimpleGTCondition(sparkSession: SparkContext, tableName: String) = { + val doc: Document = MapRDB.newDocument.set("a.b[0].boolean", false).set("a.b[1].decimal", 123.456).set("a.c.d", 5.0).set("a.c.e", "Hello").set("m", "MapR wins") + MapRDB.getTable(tableName).insertOrReplace("k1", doc) + doc.set("a.c.d", 11.0) + MapRDB.getTable(tableName).insertOrReplace("k2", doc) + doc.set("a.c.d", 8.0) + MapRDB.getTable(tableName).insertOrReplace("k3", doc) + doc.set("a.c.d", 14.0) + MapRDB.getTable(tableName).insertOrReplace("k4", doc) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("a.c.d") > 10).collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + MapRDB.getTable(tableName).delete("k3") + MapRDB.getTable(tableName).delete("k4") + if (maprd.map(doc => doc.`a.c.d`[Double]).toSet.equals(Set(11.0d,14.0d))) { + println("testingSimpleGTCondition succeeded") + true + } else { + println("testingSimpleGTCondition failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + def testingNotExistsCondition(sparkSession: SparkContext, tableName: String) = { + val doc: Document = MapRDB.newDocument.set("a.b[0].boolean", false).set("a.b[1].decimal", 123.456).set("a.c.d", 5.0).set("m", "MapR wins") + MapRDB.getTable(tableName).insertOrReplace("k1", doc) + doc.set("a.c.e", "SanJose") + MapRDB.getTable(tableName).insertOrReplace("k2", doc) + doc.set("a.c.e", "Milpitas") + MapRDB.getTable(tableName).insertOrReplace("k3", doc) + MapRDB.getTable(tableName).insertOrReplace("k4", doc) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("a.c.e") notexists).collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + MapRDB.getTable(tableName).delete("k3") + MapRDB.getTable(tableName).delete("k4") + if (maprd.map(doc => doc.getIdString()).sameElements((Array("k1")))) { + println("testingNotExistsCondition succeeded") + true + } else { + println("testingNotExistsCondition failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + def testingSimpleINCondition(sparkSession: SparkContext, tableName: String) = { + val doc: Document = MapRDB.newDocument.set("a.boolean", false).set("a.c.d", ODate.parse("2010-01-11")).set("m", "MapR wins") + MapRDB.getTable(tableName).insertOrReplace("k1", doc) + doc.set("a.c.d", ODate.parse("2011-05-21")) + MapRDB.getTable(tableName).insertOrReplace("k2", doc) + doc.set("a.c.d", ODate.parse("2005-06-21")) + MapRDB.getTable(tableName).insertOrReplace("k3", doc) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("a.c.d") in Seq(ODate.parse("2011-05-21"), ODate.parse("2013-02-22"))).collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + MapRDB.getTable(tableName).delete("k3") + if (maprd.map(doc => doc.getIdString()).sameElements(Array("k2"))) { + println("testingSimpleINCondition succeeded") + true + } else { + println("testingSimpleINCondition failed") + false + } + } + + def testingOTimeINCondition(sparkSession: SparkContext, tableName: String) = { + val doc: Document = MapRDB.newDocument.set("a.boolean", false).set("a.c.d", new OTime(1000)).set("m", "MapR wins") + MapRDB.getTable(tableName).insertOrReplace("k1", doc) + doc.set("a.c.d", new OTime(1001)) + MapRDB.getTable(tableName).insertOrReplace("k2", doc) + doc.set("a.c.d", new OTime(1002)) + MapRDB.getTable(tableName).insertOrReplace("k3", doc) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("a.c.d") in Seq(new OTime(999), new OTime(1000))).collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + MapRDB.getTable(tableName).delete("k3") + if (maprd.map(doc => doc.getIdString()).sameElements(Array("k1"))) { + println("testingOTimeINCondition succeeded") + true + } else { + println("testingOTimeINCondition failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + def testingSimpleOTime(sparkSession: SparkContext, tableName: String) = { + val doc: Document = MapRDB.newDocument.set("a.boolean", false).set("a.c.d", new OTime(1000)).set("m", "MapR wins") + MapRDB.getTable(tableName).insertOrReplace("k1", doc) + doc.set("a.c.d", new OTime(1001)) + MapRDB.getTable(tableName).insertOrReplace("k2", doc) + doc.set("a.c.d", new OTime(1002)) + MapRDB.getTable(tableName).insertOrReplace("k3", doc) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("a.c.d") === new OTime(1000)).collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + MapRDB.getTable(tableName).delete("k3") + if (maprd.map(doc => doc.getIdString()).sameElements(Array("k1"))) { + println("testingSimpleOTime succeeded") + true + } else { + println("testingSimpleOTime failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + def testingTYPEOFCondition(sparkSession: SparkContext, tableName: String) = { + val doc: Document = MapRDB.newDocument.set("a.b[0].boolean", false).set("a.b[1].decimal", 123.456).set("a.c.d", 5).set("m", "MapR wins") + MapRDB.getTable(tableName).insertOrReplace("k1", doc) + doc.set("a.c.d", "SanJose") + MapRDB.getTable(tableName).insertOrReplace("k2", doc) + doc.set("a.c.d", "Milpitas") + MapRDB.getTable(tableName).insertOrReplace("k3", doc) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("a.c.d") typeof "INT").collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + MapRDB.getTable(tableName).delete("k3") + if (maprd.map(doc => doc.getIdString()).sameElements(Array("k1"))) { + println("testingTYPEOFCondition succeeded") + true + } else { + println("testingTYPEOFCondition failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + def testingComplexAND_INcondition(sparkSession: SparkContext, tableName: String) = { + val doc: Document = MapRDB.newDocument.set("a.b[0].boolean", false).set("a.b[1].decimal", 123.456).set("a.c.d", 5).set("a.c.e", "aaa").set("m", "MapR wins") + MapRDB.getTable(tableName).insertOrReplace("k1", doc) + MapRDB.getTable(tableName).insertOrReplace("k3", doc) + doc.set("a.c.e", "xyz") + MapRDB.getTable(tableName).insertOrReplace("k2", doc) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where((field("a.c.d") in Seq(5,10)) and (field("a.c.e") notin Seq("aaa","bbb"))).collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + MapRDB.getTable(tableName).delete("k3") + if (maprd.map(doc =>doc.getIdString()).sameElements(Array("k2"))) { + println("testingComplexAND_INcondition succeeded") + true + } else { + println("testingComplexAND_INcondition failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + def testingCompositeCondition(sparkSession: SparkContext, tableName: String) = { + val doc: Document = MapRDB.newDocument.set("a.b[0].boolean", false).set("a.b[1].decimal", 123.456).set("a.c.d", 5.0).set("m", "MapR wins") + MapRDB.getTable(tableName).insertOrReplace("k1", doc) + doc.set("a.c.d", 10.0) + doc.set("a.c.e", "xyz") + MapRDB.getTable(tableName).insertOrReplace("k2", doc) + doc.set("a.c.d", 8.0) + MapRDB.getTable(tableName).insertOrReplace("k3", doc) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where((field("a.c.d") < 10) and (field("a.c.e") exists)).collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + MapRDB.getTable(tableName).delete("k3") + if (maprd.map(doc => doc.getIdString()).sameElements((Array("k3")))) { + println("testingCompositeCondition succeeded") + true + } else { + println("testingCompositeCondition failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + def testingLikeCondition(sparkSession: SparkContext, tableName: String) = { + val doc: Document = MapRDB.newDocument.set("a.b[0].boolean", false).set("a.b[1].decimal", 123.456).set("a.c.d", 5.0).set("m", "MapR wins") + MapRDB.getTable(tableName).insertOrReplace("k1", doc) + doc.set("a.c.d", 10.0) + doc.set("a.c.e", "xyz") + doc.set("m", "xyz") + MapRDB.getTable(tableName).insertOrReplace("k2", doc) + doc.set("a.c.d", 8.0) + MapRDB.getTable(tableName).insertOrReplace("k3", doc) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("m") like "%s").collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + MapRDB.getTable(tableName).delete("k3") + if (maprd.map(doc => doc.getIdString()).sameElements((Array("k1")))) { + println("testingLikeCondition succeeded") + true + } else { + println("testingLikeCondition failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + + def testingMatchesCondition(sparkSession: SparkContext, tableName: String) = { + val doc: Document = MapRDB.newDocument.set("a.b[0].boolean", false).set("a.b[1].decimal", 123.456).set("a.c.d", 5.0).set("m", "MapR wins") + MapRDB.getTable(tableName).insertOrReplace("k1", doc) + doc.set("a.c.d", 10.0) + doc.set("a.c.e", "xyz") + doc.set("m", "xyz") + MapRDB.getTable(tableName).insertOrReplace("k2", doc) + doc.set("a.c.d", 8.0) + MapRDB.getTable(tableName).insertOrReplace("k3", doc) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("m") matches ".*s").collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + MapRDB.getTable(tableName).delete("k3") + if (maprd.map(doc => doc.getIdString()).sameElements((Array("k1")))) { + println("testingMatchesCondition succeeded") + true + } else { + println("testingMatchesCondition failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + def testingThreeConditions(sparkSession: SparkContext, tableName: String) = { + val doc: Document = MapRDB.newDocument.set("a.b", 5.1).set("a.c.e", "mapR").set("m", "MapR wins").set("a.k", 25.1) + MapRDB.getTable(tableName).insertOrReplace("k1", doc) + doc.set("a.k", 10.11) + MapRDB.getTable(tableName).insertOrReplace("k2", doc) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where((field("a.c.e") exists) and ((field("a.b") >= 5) and (field("a.k") < 20))).collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + if (maprd.map(doc => doc.getIdString()).sameElements(Array("k2"))) { + println("testingThreeConditions succeeded") + true + } else { + println("testingThreeConditions failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + + false + } + } + + def testingORCondition(sparkSession: SparkContext, tableName: String) = { + val doc: Document = MapRDB.newDocument.set("a.b", 5.1).set("a.c.d", "mapR").set("m", "MapR wins").set("a.k", 25.1) + MapRDB.getTable(tableName).insertOrReplace("k1", doc) + doc.set("a.c.d", 101) + MapRDB.getTable(tableName).insertOrReplace("k2", doc) + doc.set("a.c.d", 100) + MapRDB.getTable(tableName).insertOrReplace("k3", doc) + doc.set("a.c.d", 99) + MapRDB.getTable(tableName).insertOrReplace("k4", doc) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where((field("a.c.d") > 100L) or (field("a.c.d") typeof "STRING")).collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + MapRDB.getTable(tableName).delete("k3") + MapRDB.getTable(tableName).delete("k4") + if (maprd.map(doc => 1).reduce(_+_) == 2) { + println("testingORCondition succeeded") + true + } else { + println("testingORCondition failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + def testingTypeOfWithNonExistantType(sparkSession: SparkContext, tableName: String) = { + var succeeded : Boolean = false; + val doc: Document = MapRDB.newDocument.set("a.b", 5.1).set("a.c.d", "mapR").set("m", "MapR wins").set("a.k", 25.1) + MapRDB.getTable(tableName).insertOrReplace("k1", doc) + doc.set("a.c.d", 101) + MapRDB.getTable(tableName).insertOrReplace("k2", doc) + doc.set("a.c.d", 100) + MapRDB.getTable(tableName).insertOrReplace("k3", doc) + doc.set("a.c.d", 99) + MapRDB.getTable(tableName).insertOrReplace("k4", doc) + MapRDB.getTable(tableName).flush + try { + val maprd = sparkSession.loadFromMapRDB(tableName).where((field("a.c.d") > 100L) or (field("a.c.d") typeof "STR")).collect + } catch { + case e: TypeException => println(e.getMessage) + println("testingTypeOfWithNonExistantType succeeded") + succeeded = true + } + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + MapRDB.getTable(tableName).delete("k3") + MapRDB.getTable(tableName).delete("k4") + if (succeeded) { + println("testingTypeOfWithNonExistantType succeeded") + true + } else { + println("testingTypeOfWithNonExistantType failed") + false + } + } + + def testingComplexConditonWithDate(sparkSession: SparkContext, tableName: String) = { + val doc: Document = MapRDB.newDocument.set("a.c.d", 22).set("m", "MapR wins").set("p.q", ODate.parse("2016-04-05")) + MapRDB.getTable(tableName).insertOrReplace("id1", doc) + doc.set("a.b", "xyz") + MapRDB.getTable(tableName).insertOrReplace("id2", doc) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(((field("a.b") notexists ) and (field("p.q") typeof "DATE")) and (field("a.c.d") > 20L)).collect + MapRDB.getTable(tableName).delete("id1") + MapRDB.getTable(tableName).delete("id2") + if (maprd.map(doc => doc.getIdString()).sameElements((Array("id1")))) { + println("testingComplexConditionWithDate succeeded") + true + } else { + println("testingComplexConditionWithDate failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + def testingBetweenCondition(sparkSession: SparkContext, tableName: String) = { + val doc: Document = MapRDB.newDocument.set("a.b[0].boolean", false).set("a.c.d", ODate.parse("2015-01-16")).set("a.c.e", "Hello").set("m", "MapR") + MapRDB.getTable(tableName).insertOrReplace("id1", doc) + doc.set("a.c.d", ODate.parse("2015-01-14")) + MapRDB.getTable(tableName).insertOrReplace("id2", doc) + doc.set("a.c.d", ODate.parse("2015-06-14")) + MapRDB.getTable(tableName).insertOrReplace("id3", doc) + doc.set("a.c.d", ODate.parse("2015-02-26")) + MapRDB.getTable(tableName).insertOrReplace("id4", doc) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("a.c.d") between (ODate.parse("2015-01-15"), ODate.parse("2015-05-15"))).collect + MapRDB.getTable(tableName).delete("id1") + MapRDB.getTable(tableName).delete("id2") + MapRDB.getTable(tableName).delete("id3") + MapRDB.getTable(tableName).delete("id4") + if (maprd.map(doc => doc.getIdString()).toSet.equals(Set("id1", "id4"))) { + println("testingBetweenCondition succeeded") + true + } else { + println("testingBetweenCondition failed") + maprd.map(doc => doc.getIdString()).foreach(println(_)) + false + } + } + + def testingEqualityConditionOnSeq(sparkSession: SparkContext, tableName: String) = { + val d: Document = MapRDB.newDocument.set("a.b", 111.222).set("a.x", true) + MapRDB.getTable(tableName).insertOrReplace("id1", d) + val d2: Document = d.set("a.b[0]", 12345).set("a.b[1]", "xyz") + MapRDB.getTable(tableName).insertOrReplace("id2", d2) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("a.b") === Seq(12345L, "xyz")).collect + MapRDB.getTable(tableName).delete("id1") + MapRDB.getTable(tableName).delete("id2") + if (maprd.map(doc => (doc.getIdString(),doc.`a.b`[Seq[AnyRef]].apply(0),doc.`a.b`[Seq[AnyRef]].apply(1))).toSet.equals(Set(("id2", 12345L, "xyz")))) { + println("testingEqualityConditionOnSeq succeeded") + true + } else { + println("testingEqualityConditionSeq failed") + maprd.map(doc => (doc.getIdString(),doc.getList("a.b")(0),doc.getList("a.b")(1))).foreach(println(_)) + false + } + } + + def testingINConditionOnSeqwithInSeq(sparkSession: SparkContext, tableName: String) = { + val d: Document = MapRDB.newDocument.set("a.b", 111.222).set("a.x", true) + MapRDB.getTable(tableName).insertOrReplace("id1", d) + val d2: Document = d.set("a.b1[0]", 12345).set("a.b1[1]", "xyz") + d2.set("a.c[0]", d.getList("a.b1")) + MapRDB.getTable(tableName).insertOrReplace("id2", d2) + MapRDB.getTable(tableName).flush + val list: Seq[Any] = sparkSession.loadFromMapRDB(tableName).map(doc => doc.`a.b1`).collect + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("a.c") === Seq(list(1))).collect + MapRDB.getTable(tableName).delete("id1") + MapRDB.getTable(tableName).delete("id2") + val values = maprd.map(doc => doc.getIdString()) + if (values.toSet.equals(Set("id2"))) { + println("testingINConditionOnSeqwithInSeq succeeded") + true + } else { + println("testingINConditionOnSeqwithInSeq failed") + maprd.map(doc => (doc.getIdString(),doc.getList("a.b")(0),doc.getList("a.b")(1))).foreach(println(_)) + false + } + } + + def testingNotEqualityConditionOnSeq(sparkSession: SparkContext, tableName: String) = { + val d: Document = MapRDB.newDocument.set("a.b", 111.222).set("a.x", true) + MapRDB.getTable(tableName).insertOrReplace("id1", d) + val d2: Document = d.set("a.b[0]", 12345).set("a.b[1]", "xyz") + MapRDB.getTable(tableName).insertOrReplace("id2", d2) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("a.b") != Seq(12345L, "xyz")).collect + MapRDB.getTable(tableName).delete("id1") + MapRDB.getTable(tableName).delete("id2") + if (maprd.map(doc => (doc.getIdString(),doc.`a.b`[Double])).toSet.equals(Set(("id1",111.222)))) { + println("testingNotEqualityConditionOnSeq succeeded") + true + } else { + println("testingNotEqualityConditionSeq failed") + maprd.map(doc => (doc.getIdString(),doc.getList("a.b")(0),doc.getList("a.b")(1))).foreach(println(_)) + false + } + } + + def testingNotEqualityOnID(sparkSession : SparkContext , tableName : String) = { + val d: Document = MapRDB.newDocument.set("a.b", 350).set("s", "Holger Way") + MapRDB.getTable(tableName).insertOrReplace("k1", d) + d.set("zip-code", "95134") + MapRDB.getTable(tableName).insertOrReplace("k2", d) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("_id") != "k1").collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + if (maprd.map(doc => doc.`zip-code`[String]).sameElements(Array("95134"))) { + println("testingIdOnlyCondition succeeded") + true + } else { + println("testingIdOnlyCondition failed") + false + } + } + + def testingEqualtiyOnMapOfStrings(sparkSession: SparkContext, tableName: String) = { + val m: java.util.Map[String, String] = new java.util.HashMap[String, String] + m.put("k", "kite") + m.put("m", "map") + val doc: Document = MapRDB.newDocument.set("a", m).set("b.c.d", ODate.parse("2013-03-22")) + MapRDB.getTable(tableName).insertOrReplace("id1", doc) + val d2: Document = doc.set("a.m", "not map").setArray("x.y", Array[Int](4, 44)) + MapRDB.getTable(tableName).insertOrReplace("id2", d2) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("a") === Map("k" -> "kite", "m" -> "map")).collect + MapRDB.getTable(tableName).delete("id1") + MapRDB.getTable(tableName).delete("id2") + if (maprd.map(doc => doc.getIdString()).sameElements(Array("id1"))) { + println("testingEqualityOnMapOfString succeeded") + true + } else { + println("testingEqualityOnMapOfStrings failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + def testingEqualityOnMapStringInteger(sparkSession: SparkContext, tableName: String) = { + val m: java.util.Map[String, Integer] = new java.util.HashMap[String, Integer] + m.put("k", 100) + m.put("m", 120) + val doc: Document = MapRDB.newDocument.set("a", m).set("b.c.d", ODate.parse("2013-03-22")) + MapRDB.getTable(tableName).insertOrReplace("id1", doc) + val d2: Document = doc.set("a.m", "not map").setArray("x.y", Array[Int](4, 44)) + MapRDB.getTable(tableName).insertOrReplace("id2", d2) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("a") === Map("k" -> 100, "m" -> 120)).collect + MapRDB.getTable(tableName).delete("id1") + MapRDB.getTable(tableName).delete("id2") + if (maprd.map(doc => doc.getIdString()).sameElements(Array("id1"))) { + println("testingEqualityOnMapStringInteger succeeded") + true + } else { + println("testingEqualityOnMapStringInteger failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + def testingNotEqualityOnMapStringInteger(sparkSession: SparkContext, tableName: String) = { + val m: java.util.Map[String, Integer] = new java.util.HashMap[String, Integer] + m.put("k", 100) + m.put("m", 120) + val doc: Document = MapRDB.newDocument.set("a", m).set("b.c.d", ODate.parse("2013-03-22")) + MapRDB.getTable(tableName).insertOrReplace("id1", doc) + val d2: Document = doc.set("a.m", "not map").setArray("x.y", Array[Int](4, 44)) + MapRDB.getTable(tableName).insertOrReplace("id2", d2) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(field("a") != Map("k" -> 100, "m" -> 120)).collect + MapRDB.getTable(tableName).delete("id1") + MapRDB.getTable(tableName).delete("id2") + if (maprd.map(doc => doc.getIdString()).sameElements(Array("id2"))) { + println("testingNotEqualityOnMapStringInteger succeeded") + true + } else { + println("testingNotEqualityOnMapStringInteger failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + + def testingSizeOf(sparkSession: SparkContext, tableName: String) = { + val m: java.util.Map[String, Integer] = new java.util.HashMap[String, Integer] + m.put("k", 100) + m.put("m", 120) + val doc: Document = MapRDB.newDocument.set("a", m).set("b.c.d", ODate.parse("2013-03-22")) + MapRDB.getTable(tableName).insertOrReplace("id1", doc) + val d2: Document = doc.set("a.m", "not map").setArray("x.y", Array[Int](4, 44)) + MapRDB.getTable(tableName).insertOrReplace("id2", d2) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(sizeOf(field("a.m")) > 4).collect + MapRDB.getTable(tableName).delete("id1") + MapRDB.getTable(tableName).delete("id2") + if (maprd.map(doc => doc.getIdString()).sameElements(Array("id2"))) { + println("testingSizeOf succeeded") + true + } else { + println("testingSizeOf failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + def testingSizeOfWithComplexCondition(sparkSession: SparkContext, tableName: String) = { + val m: java.util.Map[String, Integer] = new java.util.HashMap[String, Integer] + m.put("k", 100) + m.put("m", 120) + val doc: Document = MapRDB.newDocument.set("a", m).set("b.c.d", ODate.parse("2013-03-22")) + MapRDB.getTable(tableName).insertOrReplace("id1", doc) + val d2: Document = doc.set("a.m", "not map").setArray("x.y", Array[Int](4, 44)) + MapRDB.getTable(tableName).insertOrReplace("id2", d2) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where((sizeOf(field("a")) > 1) and (field("a") != Map("k" -> 100, "m" -> 120))).collect + MapRDB.getTable(tableName).delete("id1") + MapRDB.getTable(tableName).delete("id2") + if (maprd.map(doc => doc.getIdString()).sameElements(Array("id2"))) { + println("testingSizeOfWithComplexCondition succeeded") + true + } else { + println("testingSizeOfWithComplexCondition failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } + + def testingWithQueryCondition(sparkSession: SparkContext, tableName: String) = { + val d: Document = MapRDB.newDocument.set("a.b", 350).set("s", "Holger Way") + MapRDB.getTable(tableName).insertOrReplace("k1", d) + d.set("zip-code", "95134") + MapRDB.getTable(tableName).insertOrReplace("k2", d) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(MapRDB.newCondition().is("_id", QueryCondition.Op.EQUAL, "k2").build()).collect + MapRDB.getTable(tableName).delete("k1") + MapRDB.getTable(tableName).delete("k2") + if (maprd.map(doc => doc.`zip-code`[String]).sameElements(Array("95134"))) { + println("testingWithQueryCondition succeeded") + true + } else { + println("testingWithQueryCondition failed") + false + } + } + + def testWithListINCondition(sparkSession: SparkContext, tableName: String) = { + val maprd = sparkSession.loadFromMapRDB("/tmp/bug26114").where(field("level0_a0_bool") in List(-4998511681057838802L, 0.22987476349110803, -1722097687, 0.6030484, false)).collect + if (maprd.size == 0) { + println("testWithListINCondition succeeded") + true + } else { + println("testWithListINCondition failed") + false + } + } + + def testingSizeOfNotEquals(sparkSession: SparkContext, tableName: String) = { + val m: java.util.Map[String, Integer] = new java.util.HashMap[String, Integer] + m.put("k", 100) + m.put("m", 120) + val doc: Document = MapRDB.newDocument.set("a", m).set("b.c.d", ODate.parse("2013-03-22")) + MapRDB.getTable(tableName).insertOrReplace("id1", doc) + val d2: Document = doc.set("a.m", "not map").setArray("x.y", Array[Int](4, 44)) + MapRDB.getTable(tableName).insertOrReplace("id2", d2) + MapRDB.getTable(tableName).flush + val maprd = sparkSession.loadFromMapRDB(tableName).where(sizeOf(field("a.m")) != 4).collect + MapRDB.getTable(tableName).delete("id1") + MapRDB.getTable(tableName).delete("id2") + if (maprd.map(doc => doc.getIdString()).sameElements(Array("id2"))) { + println("testingSizeOfNotEquals succeeded") + true + } else { + println("testingSizeOfNotEquals failed") + maprd.map(doc => doc.asJsonString()).foreach(println(_)) + false + } + } +} + + +object PredicateTestsWithKryo { + val tableName = "/tmp/user_profiles_predicates" + lazy val conf = new SparkConf() + .setAppName("simpletest") + .set("spark.executor.memory", "1g") + .set("spark.driver.memory", "1g") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "com.mapr.db.spark.OJAIKryoRegistrator") + + def main(args: Array[String]): Unit = { + var sc = new SparkContext(conf) + PredicateTests.tableInitialization(sc, tableName) + PredicateTests.runTests(sc) + } +} diff --git a/external/maprdb/src/test/scala/com/mapr/db/testCases/SparkSqlAcessTests.scala b/external/maprdb/src/test/scala/com/mapr/db/testCases/SparkSqlAcessTests.scala new file mode 100644 index 0000000000000..190ddf71e5d9b --- /dev/null +++ b/external/maprdb/src/test/scala/com/mapr/db/testCases/SparkSqlAcessTests.scala @@ -0,0 +1,2679 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.testCases + +import org.apache.spark.{SparkConf, SparkException} +import org.ojai.Document +import java.math.BigDecimal +import java.sql.Timestamp +import com.mapr.db.rowcol.DBDocumentImpl +import com.mapr.db.spark._ +import com.mapr.db.spark.exceptions.SchemaMappingException +import org.apache.spark.sql.SparkSession +import org.ojai.types.{ODate, OTime, OTimestamp} +import com.mapr.org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator +import com.mapr.db.MapRDB + +object SparkSqlAccessTests { + lazy val conf = new SparkConf() + .setAppName("simpletest") + .set("spark.executor.memory","1g") + .set("spark.driver.memory", "1g") + + lazy val spark = SparkSession.builder().appName("SparkSqlTest").config(conf).getOrCreate() + + val tableName="/tmp/SparkSqlOjaiConnectorAccessTesting" + + def main(args: Array[String]): Unit = { + SparkSqlAccessTests.tableInitialization(tableName) + SparkSqlAccessTests.runTests(spark) + } + + def tableInitialization(tableName: String): Unit = { + if (MapRDB.tableExists(tableName)) + MapRDB.deleteTable(tableName) + val table = MapRDB.createTable(tableName) + table.insertOrReplace(getNullRecord()) + table.insertOrReplace(getBooleanRecord()) + table.insertOrReplace(getStringRecord()) + table.insertOrReplace(getByteRecord()) + table.insertOrReplace(getShortRecord()) + table.insertOrReplace(getIntRecord()) + table.insertOrReplace(getLongRecord()) + table.insertOrReplace(getFloatRecord()) + table.insertOrReplace(getDoubleRecord()) + //table.insertOrReplace(getDecimalRecord()) + table.insertOrReplace(getDateRecord()) + table.insertOrReplace(getTimeRecord()) + table.insertOrReplace(getTimeStampRecord()) + table.insertOrReplace(getBinaryRecord()) + table.insertOrReplace(getMapRecord()) + table.insertOrReplace(getArrayRecord()) + table.flush() + table.close() + } + + private def getBooleanRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + rec.setId("boolean") + rec.set("null", true) + rec.set("boolean", true) + rec.set("string", true) + rec.set("byte", true) + rec.set("short", true) + rec.set("int", true) + rec.set("long", true) + rec.set("float", true) + rec.set("double", true) + rec.set("decimal", true) + rec.set("date", true) + rec.set("time", true) + rec.set("timestamp", true) + rec.set("binary", true) + rec.set("map", true) + rec.set("array", true) + return rec + } + + private def getNullRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + rec.setId("null") + rec.setNull("null") + rec.setNull("boolean") + rec.setNull("string") + rec.setNull("byte") + rec.setNull("short") + rec.setNull("int") + rec.setNull("long") + rec.setNull("float") + rec.setNull("double") + rec.setNull("decimal") + rec.setNull("date") + rec.setNull("time") + rec.setNull("timestamp") + rec.setNull("binary") + rec.setNull("map") + rec.setNull("array") + return rec + } + + private def getStringRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + rec.setId("string") + rec.set("null", "hello") + rec.set("boolean", "hello") + rec.set("string", "hello") + rec.set("byte", "hello") + rec.set("short", "hello") + rec.set("int", "hello") + rec.set("long", "hello") + rec.set("float", "hello") + rec.set("double", "hello") + rec.set("decimal", "hello") + rec.set("date", "hello") + rec.set("time", "hello") + rec.set("timestamp", "hello") + rec.set("binary", "hello") + rec.set("map", "hello") + rec.set("array", "hello") + return rec + } + + private def getByteRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + rec.setId("byte") + rec.set("null", 100.toByte) + rec.set("boolean", 100.toByte) + rec.set("string", 100.toByte) + rec.set("byte", 100.toByte) + rec.set("short", 100.toByte) + rec.set("int", 100.toByte) + rec.set("long", 100.toByte) + rec.set("float", 100.toByte) + rec.set("double", 100.toByte) + rec.set("decimal", 100.toByte) + rec.set("date", 100.toByte) + rec.set("time", 100.toByte) + rec.set("timestamp", 100.toByte) + rec.set("binary", 100.toByte) + rec.set("map", 100.toByte) + rec.set("array", 100.toByte) + return rec + } + + + private def getShortRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + rec.setId("short") + rec.set("null", 10000.toShort) + rec.set("boolean", 10000.toShort) + rec.set("string", 10000.toShort) + rec.set("byte", 10000.toShort) + rec.set("short", 10000.toShort) + rec.set("int", 10000.toShort) + rec.set("long", 10000.toShort) + rec.set("float", 10000.toShort) + rec.set("double", 10000.toShort) + rec.set("decimal", 10000.toShort) + rec.set("date", 10000.toShort) + rec.set("time", 10000.toShort) + rec.set("timestamp", 10000.toShort) + rec.set("binary", 10000.toShort) + rec.set("map", 10000.toShort) + rec.set("array", 10000.toShort) + return rec + } + + private def getIntRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + rec.setId("int") + rec.set("null", new Integer(5000)) + rec.set("boolean", new Integer(5000)) + rec.set("string", new Integer(5000)) + rec.set("byte", new Integer(5000)) + rec.set("short", new Integer(5000)) + rec.set("int", new Integer(5000)) + rec.set("long", new Integer(5000)) + rec.set("float", new Integer(5000)) + rec.set("double", new Integer(5000)) + rec.set("decimal", new Integer(5000)) + rec.set("date", new Integer(5000)) + rec.set("time", new Integer(5000)) + rec.set("timestamp", new Integer(5000)) + rec.set("binary", new Integer(5000)) + rec.set("map", new Integer(5000)) + rec.set("array", new Integer(5000)) + return rec + } + + private def getLongRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + rec.setId("long") + rec.set("null", 12345678999L) + rec.set("boolean", 12345678999L) + rec.set("string", 12345678999L) + rec.set("byte", 12345678999L) + rec.set("short", 12345678999L) + rec.set("int", 12345678999L) + rec.set("long", 12345678999L) + rec.set("float", 12345678999L) + rec.set("double", 12345678999L) + rec.set("decimal", 12345678999L) + rec.set("date", 12345678999L) + rec.set("time", 12345678999L) + rec.set("timestamp", 12345678999L) + rec.set("binary", 12345678999L) + rec.set("map", 12345678999L) + rec.set("array", 12345678999L) + return rec + } + + private def getFloatRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + rec.setId("float") + rec.set("null", 10.1234f) + rec.set("boolean", 10.1234f) + rec.set("string", 10.1234f) + rec.set("byte", 10.1234f) + rec.set("short", 10.1234f) + rec.set("int", 10.1234f) + rec.set("long", 10.1234f) + rec.set("float", 10.1234f) + rec.set("double", 10.1234f) + rec.set("decimal", 10.1234f) + rec.set("date", 10.1234f) + rec.set("time", 10.1234f) + rec.set("timestamp", 10.1234f) + rec.set("binary", 10.1234f) + rec.set("map", 10.1234f) + rec.set("array", 10.1234f) + return rec + } + + private def getDoubleRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + rec.setId("double") + rec.set("null", 10.12345678910d) + rec.set("boolean", 10.12345678910d) + rec.set("string", 10.12345678910d) + rec.set("byte", 10.12345678910d) + rec.set("short", 10.12345678910d) + rec.set("int", 10.12345678910d) + rec.set("long", 10.12345678910d) + rec.set("float", 10.12345678910d) + rec.set("double", 10.12345678910d) + rec.set("decimal", 10.12345678910d) + rec.set("date", 10.12345678910d) + rec.set("time", 10.12345678910d) + rec.set("timestamp", 10.12345678910d) + rec.set("binary", 10.12345678910d) + rec.set("map", 10.12345678910d) + rec.set("array", 10.12345678910d) + return rec + } + + + private def getDecimalRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + rec.setId("decimal") + rec.set("null", new BigDecimal(1.5)) + rec.set("boolean", new BigDecimal(1.5)) + rec.set("string", new BigDecimal(1.5)) + rec.set("byte", new BigDecimal(1.5)) + rec.set("short", new BigDecimal(1.5)) + rec.set("int", new BigDecimal(1.5)) + rec.set("long", new BigDecimal(1.5)) + rec.set("float", new BigDecimal(1.5)) + rec.set("double", new BigDecimal(1.5)) + rec.set("decimal", new BigDecimal(1.5)) + rec.set("date", new BigDecimal(1.5)) + rec.set("time", new BigDecimal(1.5)) + rec.set("timestamp", new BigDecimal(1.5)) + rec.set("binary", new BigDecimal(1.5)) + rec.set("map", new BigDecimal(1.5)) + rec.set("array", new BigDecimal(1.5)) + return rec + } + + + private def getDateRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + rec.setId("date") + rec.set("null", ODate.fromDaysSinceEpoch(1000)) + rec.set("boolean", ODate.fromDaysSinceEpoch(1000)) + rec.set("string", ODate.fromDaysSinceEpoch(1000)) + rec.set("byte", ODate.fromDaysSinceEpoch(1000)) + rec.set("short", ODate.fromDaysSinceEpoch(1000)) + rec.set("int", ODate.fromDaysSinceEpoch(1000)) + rec.set("long", ODate.fromDaysSinceEpoch(1000)) + rec.set("float", ODate.fromDaysSinceEpoch(1000)) + rec.set("double", ODate.fromDaysSinceEpoch(1000)) + rec.set("decimal", ODate.fromDaysSinceEpoch(1000)) + rec.set("date", ODate.fromDaysSinceEpoch(1000)) + rec.set("time", ODate.fromDaysSinceEpoch(1000)) + rec.set("timestamp", ODate.fromDaysSinceEpoch(1000)) + rec.set("binary", ODate.fromDaysSinceEpoch(1000)) + rec.set("map", ODate.fromDaysSinceEpoch(1000)) + rec.set("array", ODate.fromDaysSinceEpoch(1000)) + return rec + } + + private def getTimeRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + rec.setId("time") + rec.set("null", OTime.fromMillisOfDay(1000)) + rec.set("boolean", OTime.fromMillisOfDay(1000)) + rec.set("string", OTime.fromMillisOfDay(1000)) + rec.set("byte", OTime.fromMillisOfDay(1000)) + rec.set("short", OTime.fromMillisOfDay(1000)) + rec.set("int", OTime.fromMillisOfDay(1000)) + rec.set("long", OTime.fromMillisOfDay(1000)) + rec.set("float", OTime.fromMillisOfDay(1000)) + rec.set("double", OTime.fromMillisOfDay(1000)) + rec.set("decimal", OTime.fromMillisOfDay(1000)) + rec.set("date", OTime.fromMillisOfDay(1000)) + rec.set("time", OTime.fromMillisOfDay(1000)) + rec.set("timestamp", OTime.fromMillisOfDay(1000)) + rec.set("binary", OTime.fromMillisOfDay(1000)) + rec.set("map", OTime.fromMillisOfDay(1000)) + rec.set("array", OTime.fromMillisOfDay(1000)) + return rec + } + + private def getTimeStampRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + rec.setId("timestamp") + rec.set("null", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("boolean", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("string", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("byte", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("short", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("int", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("long", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("float", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("double", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("decimal", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("date", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("time", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("timestamp", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("binary", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("map", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + rec.set("array", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)) + return rec + } + + private def getBinaryRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + val bytes: Array[Byte] = Array.range(1,10).map(_.toByte) + rec.setId("binary") + rec.set("null", bytes) + rec.set("boolean", bytes) + rec.set("string", bytes) + rec.set("byte", bytes) + rec.set("short", bytes) + rec.set("int", bytes) + rec.set("long", bytes) + rec.set("float", bytes) + rec.set("double", bytes) + rec.set("decimal", bytes) + rec.set("date", bytes) + rec.set("time", bytes) + rec.set("timestamp", bytes) + rec.set("binary", bytes) + rec.set("map", bytes) + rec.set("array", bytes) + return rec + } + + private def getMapRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + val map: java.util.Map [java.lang.String, Object] = new java.util.HashMap [java.lang.String, Object]() + map.put("Name", "AAA") + map.put("Age", new Integer(20)) + rec.setId("map") + rec.set("null", map) + rec.set("boolean", map) + rec.set("string", map) + rec.set("byte", map) + rec.set("short", map) + rec.set("int", map) + rec.set("long", map) + rec.set("float", map) + rec.set("double", map) + rec.set("decimal", map) + rec.set("date", map) + rec.set("time", map) + rec.set("timestamp", map) + rec.set("binary", map) + rec.set("map", map) + rec.set("array", map) + return rec + } + + private def getArrayRecord(): Document = { + val rec: DBDocumentImpl = new DBDocumentImpl() + val values = new java.util.ArrayList[Object]() + values.add("Field1") + val intvalue: Integer = 500 + values.add(intvalue) + values.add(new java.lang.Double(5555.5555)) + rec.setId("array") + rec.set("null", values) + rec.set("boolean", values) + rec.set("string", values) + rec.set("byte", values) + rec.set("short", values) + rec.set("int", values) + rec.set("long", values) + rec.set("float", values) + rec.set("double", values) + rec.set("decimal", values) + rec.set("date", values) + rec.set("time", values) + rec.set("timestamp", values) + rec.set("binary", values) + rec.set("map", values) + rec.set("array", values) + return rec + } + + def testingBooleanVsNull(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where((field("_id") === "boolean") or (field("_id") === "null")).toDF + if (booleanAndNullDF.collect.map(row => row.isNullAt(row.fieldIndex("boolean")) match { + case true => null + case false => row.getBoolean(row.fieldIndex("boolean")) + }).toSet.equals(Set(true, null))) { + println("testingBooleanVsNull Succeeded") + return true + } + else { + println("testingBooleanVsNull Failed") + return false + } + } + + def testingStringVsNull(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "string" or field("_id") === "null").toDF + if (booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("string"))).toSet.equals(Set("hello", null))) { + println("testingStringVsNull Succeeded") + return true + } + else { + println("testingStringVsNull Failed") + return false + } + } + + def testingByteVsNull(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "byte" or field("_id") === "null").toDF + if (booleanAndNullDF.collect.map(row => row.isNullAt(row.fieldIndex("byte")) match { case true => null case false => row.getByte(row.fieldIndex("byte")) }).toSet.equals(Set(100.toByte, null))) { + println("testingByteVsNull Succeeded") + return true + } + else { + println("testingByteVsNull Failed") + return false + } + } + + def testingShortVsNull(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "short" or field("_id") === "null").toDF + if (booleanAndNullDF.collect.map(row => row.isNullAt(row.fieldIndex("short")) match { case true => null case false => row.getShort(row.fieldIndex("short")) }).toSet.equals(Set(10000.toShort, null))) { + println("testingShortVsNull Succeeded") + return true + } + else { + println("testingShortVsNull Failed") + return false + } + } + + def testingIntVsNull(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "int" or field("_id") === "null").toDF + if (booleanAndNullDF.collect.map(row => row.isNullAt(row.fieldIndex("int")) match { case true => null case false => row.getInt(row.fieldIndex("int")) }).toSet.equals(Set(5000.toInt, null))) { + println("testingIntVsNull Succeeded") + return true + } + else { + println("testingIntVsNull Failed") + return false + } + } + + def testingLongVsNull(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "long" or field("_id") === "null").toDF + if (booleanAndNullDF.collect.map(row => row.isNullAt(row.fieldIndex("long")) match { case true => null case false => row.getLong(row.fieldIndex("long")) }).toSet.equals(Set(12345678999L, null))) { + println("testingLongVsNull succeeded") + return true + } + else { + println("testingLongVsNull Failed") + return false + } + } + + def testingFloatVsNull(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "float" or field("_id") === "null").toDF + if (booleanAndNullDF.collect.map(row => row.isNullAt(row.fieldIndex("float")) match { case true => null case false => row.getFloat(row.fieldIndex("float"))}).toSet.equals(Set(10.1234f, null))) { + println("testingFloatVsNull Succeeded") + return true + } + else { + println("testingFloatVsNull Failed") + return false + } + } + + def testingDoubleVsNull(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "double" or field("_id") === "null").toDF + if (booleanAndNullDF.collect.map(row => row.isNullAt(row.fieldIndex("double")) match { case true => null case false => row.getDouble(row.fieldIndex("double"))}).toSet.equals(Set(10.12345678910d, null))) { + println("testingDoubleVsNull Succeeded") + return true + } + else { + println("testingDoubleVsNull Failed") + return false + } + } + + def testingDateVsNull(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "date" or field("_id") === "null").toDF + if (booleanAndNullDF.collect.map(row => row.getDate(row.fieldIndex("date"))).toSet.equals(Set(new java.sql.Date(ODate.fromDaysSinceEpoch(1000).toDate.getTime), null))) { + println("testingDateVsNull Succeeded") + return true + } + else { + println("testingDateVsNull Failed") + return false + } + } + + def testingTimeVsNull(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "time" or field("_id") === "null").toDF + if (booleanAndNullDF.collect.map(row => row.getTimestamp(row.fieldIndex("time"))).toSet.equals(Set(new java.sql.Timestamp(OTime.fromMillisOfDay(1000).toDate.getTime), null))) { + println("testingTimeVsNull Succeeded") + return true + } + else { + println("testingTimeVsNull Failed") + return false + } + } + + def testingTimeStampVsNull(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "timestamp" or field("_id") === "null").toDF + if (booleanAndNullDF.collect.map(row => row.getTimestamp(row.fieldIndex("timestamp"))).toSet.equals(Set(new Timestamp(OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr).getMillis), null))) { + println("testingTimeStampVsNull Succeeded") + return true + } + else { + println("testingTimeSTampVsNull Failed") + return false + } + } + + def testingBinaryVsNull(spark: SparkSession, tableName: String): Boolean = { + val bytes: Array[Byte] = Array.range(1,10).map(_.toByte) + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "binary" or field("_id") === "null").toDF + if (new ByteArrayComparator().compare(booleanAndNullDF.collect.map(row => row.get(row.fieldIndex("binary")).asInstanceOf[Array[Byte]]).filter(_ != null).toSeq(0), bytes) == 0) { + println("testingBinaryVsNull Succeeded") + return true + } + else { + println("testingBinaryVsNull Failed") + return false + } + } + + def testingMapVsNull(spark: SparkSession, tableName: String): Boolean = { + val map = Map("Name" -> "AAA", "Age" -> 20.toInt) + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "map" or field("_id") === "null").toDF + if (booleanAndNullDF.collect.map(row => row.getStruct(row.fieldIndex("map"))).filter(_!=null) + .map(row => row.schema.fieldNames.zip(row.toSeq).toMap) + .toSeq(0).toSeq.toSet.equals(map.toSeq.toSet)) { + println("testingMapVsNull Succeeded") + return true + } + else { + println(booleanAndNullDF.collect.map(row => row.getMap(row.fieldIndex("map"))).filter(_!=null).toSeq(0).asInstanceOf[Map[String,Any]]) + println("testingMapVsNull Failed") + return false + } + } + + def testingArrayVsNull(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "array" or field("_id") === "null").toDF + if (booleanAndNullDF.collect.map(row => row.getSeq(row.fieldIndex("array"))).filter(_!=null).toSeq(0).asInstanceOf[Seq[Any]].sameElements(Seq("Field1","500","5555.5555"))) { + println("testingArrayVsNull Succeeded") + return true + } + else { + println(booleanAndNullDF.collect.map(row => row.getSeq(row.fieldIndex("array"))).filter(_!=null).toSeq(0)) + println("testingArrayVsNull Failed") + return false + } + } + + def testingBooleanVsString(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "boolean" or field("_id") === "string").toDF + if (booleanAndNullDF.collect.map(row => row.isNullAt(row.fieldIndex("boolean")) match { + case true => null + case false => row.getString(row.fieldIndex("boolean")) + }).toSet.equals(Set("true", "hello"))) { + println("testingBooleanVsString Succeeded") + return true + } + else { + println("testingBooleanVsString Failed") + return false + } + } + + def testingBooleanVsByte(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "boolean" or field("_id") === "byte").toDF.collect + println("testingBooleanVsByte Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingBooleanVsByte Succeeded") + return true + } + case ex: Throwable => { + println("testingBooleanVsByte Failed") + return false + } + } + } + case ex: Throwable => { + println("testingBooleanVsByte Failed") + return false + } + } + } + + def testingBooleanVsShort(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "boolean" or field("_id") === "short").toDF.collect + println("testingBooleanVsShort Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingBooleanVsShort Succeeded") + return true + } + case ex: Throwable => { + println("testingBooleanVsShort Failed") + return false + } + } + } + case ex: Throwable => { + println("testingBooleanVsShort Failed") + return false + } + } + } + + def testingBooleanVsInt(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "boolean" or field("_id") === "int").toDF.collect() + println("testingBooleanVsInt Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingBooleanVsInt Succeeded") + return true + } + case ex: Throwable => { + println("testingBooleanVsInt Failed") + return false + } + } + } + case ex: Throwable => { + println("testingBooleanVsInt Failed") + return false + } + } + } + + def testingBooleanVsLong(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "boolean" or field("_id") === "long").toDF.collect + println("testingBooleanVsLong Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingBooleanVsLong Succeeded") + return true + } + case ex: Throwable => { + println("testingBooleanVsLong Failed") + return false + } + } + } + case ex: Throwable => { + println("testingBooleanVsLong Failed") + return false + } + } + } + + def testingBooleanVsFloat(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "boolean" or field("_id") === "float").toDF.collect + println("testingBooleanVsFloat Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingBooleanVsFloat Succeeded") + return true + } + case ex: Throwable => { + println("testingBooleanVsFloat Failed") + return false + } + } + } + case ex: Throwable => { + println("testingBooleanVsFloat Failed") + return false + } + } + } + + def testingBooleanVsDouble(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "boolean" or field("_id") === "double").toDF.collect + println("testingBooleanVsDouble Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingBooleanVsDouble Succeeded") + return true + } + case ex: Throwable => { + println("testingBooleanVsDouble Failed") + return false + } + } + } + case ex: Throwable => { + println("testingBooleanVsDouble Failed") + return false + } + } + } + + def testingBooleanVsDate(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "boolean" or field("_id") === "date").toDF.collect + println("testingBooleanVsDate Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingBooleanVsDate Succeeded") + return true + } + case ex: Throwable => { + println("testingBooleanVsDate Failed") + return false + } + } + } + case ex: Throwable => { + println("testingBooleanVsDate Failed") + return false + } + } + } + + def testingBooleanVsTime(spark: SparkSession, tableName: String): Boolean = { + try{ + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "boolean" or field("_id") === "time").toDF.collect + println("testingBooleanVsTime Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingBooleanVsTime Succeeded") + return true + } + case ex: Throwable => { + println("testingBooleanVsTime Failed") + return false + } + } + } + case ex: Throwable => { + println("testingBooleanVsTime Failed") + return false + } + } + } + + def testingBooleanVsTimeStamp(spark: SparkSession, tableName: String): Boolean = { + try{ + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "boolean" or field("_id") === "timestamp").toDF.collect + println("testingBooleanVsTimeStamp Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingBooleanVsTimeStamp Succeeded") + return true + } + case ex: Throwable => { + println("testingBooleanVsTimeStamp Failed") + return false + } + } + } + case ex: Throwable => { + println("testingBooleanVsTimeStamp Failed") + return false + } + } + } + + def testingBooleanVsBinary(spark: SparkSession, tableName: String): Boolean = { + try{ + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "boolean" or field("_id") === "binary").toDF.collect + println("testingBooleanVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingBooleanVsBinary Succeeded") + return true + } + case ex: Throwable => { + println("testingBooleanVsBinary Failed") + return false + } + } + } + case ex: Throwable => { + println("testingBooleanVsBinary Failed") + return false + } + } + } + + def testingBooleanVsMap(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "boolean" or field("_id") === "map").toDF.collect + println("testingBooleanVsMap Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingBooleanVsMap Succeeded") + return true + } + case ex: Throwable => { + println("testingBooleanVsMap Failed") + return false + } + } + } + case ex: Throwable => { + println("testingBooleanVsMap Failed") + return false + } + } + } + + def testingBooleanVsArray(spark: SparkSession, tableName: String): Boolean = { + try{ + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "boolean" or field("_id") === "array").toDF.collect + println("testingBooleanVsArray Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingBooleanVsArray Succeeded") + return true + } + case ex: Throwable => { + println("testingBooleanVsArray Failed") + return false + } + } + } + case ex: Throwable => { + println("testingBooleanVsArray Failed") + return false + } + } + } + + def testingStringVsByte(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "string" or field("_id") === "byte").toDF + if (booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("string"))).toSet.equals(Set("hello", "100"))) { + println("testingStringVsByte Succeeded") + return true + } + else { + println("testingStringVsByte Failed") + return false + } + } + + def testingStringVsShort(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "string" or field("_id") === "short").toDF + if (booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("string"))).toSet.equals(Set("hello", "10000"))) { + println("testingStringVsShort Succeeded") + return true + } + else { + println("testingStringVsShort Failed") + return false + } + } + + def testingStringVsInt(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "string" or field("_id") === "int").toDF + if (booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("string"))).toSet.equals(Set("hello", "5000"))) { + println("testingStringVsInt Succeeded") + return true + } + else { + println("testingStringVsInt Failed") + return false + } + } + + def testingStringVsLong(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "string" or field("_id") === "long").toDF + if (booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("string"))).toSet.equals(Set("hello", "12345678999"))) { + println("testingStringVsLong Succeeded") + return true + } + else { + println("testingStringVsLong Failed") + return false + } + } + + def testingStringVsFloat(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "string" or field("_id") === "float").toDF + if (booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("string"))).toSet.equals(Set("hello", "10.1234"))) { + println("testingStringVsFloat Succeeded") + return true + } + else { + println("testingStringVsFloat Failed") + return false + } + } + + def testingStringVsDouble(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "string" or field("_id") === "double").toDF + if (booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("string"))).toSet.equals(Set("hello", "10.1234567891"))) { + println("testingStringVsDouble Succeeded") + return true + } + else { + booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("string"))).foreach(println) + println("testingStringVsDouble Failed") + return false + } + } + + def testingStringVsDate(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "string" or field("_id") === "date").toDF + if (booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("string"))).toSet.equals(Set("hello", ODate.fromDaysSinceEpoch(1000).toDateStr))) { + println("testingStringVsDate Succeeded") + return true + } + else { + println("testingStringVsDate Failed") + return false + } + } + + def testingStringVsTime(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "string" or field("_id") === "time").toDF + if (booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("string"))).toSet.equals(Set("hello", OTime.fromMillisOfDay(1000).toString))) { + println("testingStringVsTime Succeeded") + return true + } + else { + println("testingStringVsTime Failed") + return false + } + } + + def testingStringVsTimeStamp(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "string" or field("_id") === "timestamp").toDF + if (booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("string"))).toSet.equals(Set("hello", OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr).toString))) { + println("testingStringVsTimeStamp Succeeded") + return true + } + else { + println("testingStringVsTimeStamp Failed") + return false + } + } + + def testingStringVsBinary(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "string" or field("_id") === "binary").toDF.collect + println("testingStringVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingStringVsBinary Succeeded") + return true + } + case ex: Throwable => { + println("testingStringVsBinary Failed") + return false + } + } + } + case ex: Throwable => { + println("testingStringVsBinary Failed") + return false + } + } + } + + def testingStringVsMap(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "string" or field("_id") === "map").toDF.collect + println("testingStringVsMap Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingStringVsMap Succeeded") + return true + } + case ex: Throwable => { + println("testingStringVsMap Failed") + return false + } + } + } + case ex: Throwable => { + println("testingStringVsMap Failed") + return false + } + } + } + + def testingStringVsArray(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "string" or field("_id") === "array").toDF.collect + println("testingStringVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingStringVsArray Succeeded") + return true + } + case ex: Throwable => { + println("testingStringVsArray Failed") + return false + } + } + } + case ex: Throwable => { + println("testingStringVsArray Failed") + return false + } + } + } + + def testingByteVsShort(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "byte" or field("_id") === "short").toDF + if (booleanAndNullDF.collect.map(row => row.getShort(row.fieldIndex("byte"))).toSet.equals(Set(100, 10000))) { + println("testingByteVsShort Succeeded") + return true + } + else { + println("testingByteVsShort Failed") + return false + } + } + + def testingByteVsInt(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "byte" or field("_id") === "int").toDF + if (booleanAndNullDF.collect.map(row => row.getInt(row.fieldIndex("byte"))).toSet.equals(Set(100, 5000))) { + println("testingByteVsInt Succeeded") + return true + } + else { + println("testingByteVsInt Failed") + return false + } + } + + def testingByteVsLong(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "byte" or field("_id") === "long").toDF + if (booleanAndNullDF.collect.map(row => row.getLong(row.fieldIndex("byte"))).toSet.equals(Set(100L, 12345678999L))) { + println("testingByteVsLong Succeeded") + return true + } + else { + println("testingByteVsLong Failed") + return false + } + } + + def testingByteVsFloat(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "byte" or field("_id") === "float").toDF + if (booleanAndNullDF.collect.map(row => row.getFloat(row.fieldIndex("byte"))).toSet.equals(Set(100f, 10.1234f))) { + println("testingByteVsFloat Succeeded") + return true + } + else { + booleanAndNullDF.collect.map(row => row.getFloat(row.fieldIndex("byte"))).foreach(println) + println("testingByteVsFloat Failed") + return false + } + } + + def testingByteVsDouble(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "byte" or field("_id") === "double").toDF + if (booleanAndNullDF.collect.map(row => row.getDouble(row.fieldIndex("byte"))).toSet.equals(Set(100, 10.1234567891d))) { + println("testingByteVsDouble Succeeded") + return true + } + else { + booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("byte"))).foreach(println) + println("testingByteVsDouble Failed") + return false + } + } + + def testingByteVsDate(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "byte" or field("_id") === "date").toDF.collect + println("testingByteVsDate Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingByteVsDate Succeeded") + return true + } + case ex: Throwable => { + println("testingByteVsDate Failed") + return false + } + } + } + case ex: Throwable => { + println("testingByteVsDate Failed") + return false + } + } + } + + def testingByteVsTime(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "byte" or field("_id") === "time").toDF.collect + println("testingByteVsTime Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingByteVsTime Succeeded") + return true + } + case ex: Throwable => { + println("testingByteVsTime Failed") + return false + } + } + } + case ex: Throwable => { + println("testingByteVsTime Failed") + return false + } + } + } + + def testingByteVsTimeStamp(spark: SparkSession, tableName: String): Boolean = { + try{ + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "byte" or field("_id") === "timestamp").toDF.collect + println("testingByteVsTimeStamp Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingByteVsTimestamp Succeeded") + return true + } + case ex: Throwable => { + println("testingByteVsTimestamp Failed") + return false + } + } + } + case ex: Throwable => { + println("testingByteVsTimeStamp Failed") + return false + } + } + } + + def testingByteVsBinary(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "byte" or field("_id") === "binary").toDF.collect + println("testingByteVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingByteVsBinary Succeeded") + return true + } + case ex: Throwable => { + println("testingByteVsBinary Failed") + return false + } + } + } + case ex: Throwable => { + println("testingByteVsBinary Failed") + return false + } + } + } + + def testingByteVsMap(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "byte" or field("_id") === "map").toDF.collect + println("testingByteVsMap Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingByteVsMap Succeeded") + return true + } + case ex: Throwable => { + println("testingByteVsMap Failed") + return false + } + } + } + case ex: Throwable => { + println("testingByteVsMap Failed") + return false + } + } + } + + def testingByteVsArray(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "byte" or field("_id") === "array").toDF.collect + println("testingByteVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingByteVsArray Succeeded") + return true + } + case ex: Throwable => { + println("testingByteVsArray Failed") + return false + } + } + } + case ex: Throwable => { + println("testingByteVsArray Failed") + return false + } + } + } + + def testingShortVsInt(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "short" or field("_id") === "int").toDF + if (booleanAndNullDF.collect.map(row => row.getInt(row.fieldIndex("byte"))).toSet.equals(Set(10000, 5000))) { + println("testingShortVsInt Succeeded") + return true + } + else { + println("testingShortVsInt Failed") + return false + } + } + + def testingShortVsLong(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "short" or field("_id") === "long").toDF + if (booleanAndNullDF.collect.map(row => row.getLong(row.fieldIndex("short"))).toSet.equals(Set(10000, 12345678999L))) { + println("testingShortVsLong Succeeded") + return true + } + else { + booleanAndNullDF.collect.map(row => row.getLong(row.fieldIndex("short"))).foreach(println) + println("testingShortVsLong Failed") + return false + } + } + + def testingShortVsFloat(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "short" or field("_id") === "float").toDF + if (booleanAndNullDF.collect.map(row => row.getFloat(row.fieldIndex("short"))).toSet.equals(Set(10000f, 10.1234f))) { + println("testingShortVsFloat Succeeded") + return true + } + else { + booleanAndNullDF.collect.map(row => row.getFloat(row.fieldIndex("short"))).foreach(println) + println("testingShortVsFloat Failed") + return false + } + } + + def testingShortVsDouble(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "short" or field("_id") === "double").toDF + if (booleanAndNullDF.collect.map(row => row.getDouble(row.fieldIndex("short"))).toSet.equals(Set(10000, 10.1234567891d))) { + println("testingShortVsDouble Succeeded") + return true + } + else { + booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("short"))).foreach(println) + println("testingShortVsDouble Failed") + return false + } + } + + def testingShortVsDate(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "short" or field("_id") === "date").toDF.collect + println("testingShortVsDate Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingShortVsDate Succeeded") + return true + } + case ex: Throwable => { + println("testingShortVsDate Failed") + return false + } + } + } + case ex: Throwable => { + println("testingShortVsDate Failed") + return false + } + } + } + + def testingShortVsTime(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "short" or field("_id") === "time").toDF.collect + println("testingShortVsTime Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingShortVsTime Succeeded") + return true + } + case ex: Throwable => { + println("testingShortVsTime Failed") + return false + } + } + } + case ex: Throwable => { + println("testingShortVsTime Failed") + return false + } + } + } + + def testingShortVsTimeStamp(spark: SparkSession, tableName: String): Boolean = { + try{ + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "short" or field("_id") === "timestamp").toDF.collect + println("testingShortVsTimeStamp Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingShortVsTimestamp Succeeded") + return true + } + case ex: Throwable => { + println("testingShortVsTimestamp Failed") + return false + } + } + } + case ex: Throwable => { + println("testingShortVsTimeStamp Failed") + return false + } + } + } + + def testingShortVsBinary(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "short" or field("_id") === "binary").toDF.collect + println("testingShortVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingShortVsBinary Succeeded") + return true + } + case ex: Throwable => { + println("testingShortVsBinary Failed") + return false + } + } + } + case ex: Throwable => { + println("testingShortVsBinary Failed") + return false + } + } + } + + def testingShortVsMap(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "short" or field("_id") === "map").toDF.collect + println("testingShortVsMap Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingShortVsMap Succeeded") + return true + } + case ex: Throwable => { + println("testingShortVsMap Failed") + return false + } + } + } + case ex: Throwable => { + println("testingShortVsMap Failed") + return false + } + } + } + + def testingShortVsArray(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "short" or field("_id") === "array").toDF.collect + println("testingShortVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingShortVsArray Succeeded") + return true + } + case ex: Throwable => { + println("testingShortVsArray Failed") + return false + } + } + } + case ex: Throwable => { + println("testingShortVsArray Failed") + return false + } + } + } + + def testingIntVsLong(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "int" or field("_id") === "long").toDF + if (booleanAndNullDF.collect.map(row => row.getLong(row.fieldIndex("int"))).toSet.equals(Set(5000L, 12345678999L))) { + println("testingIntVsLong Succeeded") + return true + } + else { + booleanAndNullDF.collect.map(row => row.getLong(row.fieldIndex("int"))).foreach(println) + println("testingIntVsLong Failed") + return false + } + } + + def testingIntVsFloat(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "int" or field("_id") === "float").toDF + if (booleanAndNullDF.collect.map(row => row.getFloat(row.fieldIndex("int"))).toSet.equals(Set(5000f, 10.1234f))) { + println("testingIntVsFloat Succeeded") + return true + } + else { + booleanAndNullDF.collect.map(row => row.getFloat(row.fieldIndex("int"))).foreach(println) + println("testingIntVsFloat Failed") + return false + } + } + + def testingIntVsDouble(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "int" or field("_id") === "double").toDF + if (booleanAndNullDF.collect.map(row => row.getDouble(row.fieldIndex("int"))).toSet.equals(Set(5000d, 10.1234567891d))) { + println("testingSIntVsDouble Succeeded") + return true + } + else { + booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("int"))).foreach(println) + println("testingIntVsDouble Failed") + return false + } + } + + def testingIntVsDate(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "int" or field("_id") === "date").toDF.collect + println("testingIntVsDate Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingIntVsDate Succeeded") + return true + } + case ex: Throwable => { + println("testingIntVsDate Failed") + return false + } + } + } + case ex: Throwable => { + println("testingIntVsDate Failed") + return false + } + } + } + + def testingIntVsTime(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "int" or field("_id") === "time").toDF.collect + println("testingIntVsTime Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingIntVsTime Succeeded") + return true + } + case ex: Throwable => { + println("testingIntVsTime Failed") + return false + } + } + } + case ex: Throwable => { + println("testingIntVsTime Failed") + return false + } + } + } + + def testingIntVsTimeStamp(spark: SparkSession, tableName: String): Boolean = { + try{ + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "int" or field("_id") === "timestamp").toDF.collect + println("testingIntVsTimeStamp Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingIntVsTimestamp Succeeded") + return true + } + case ex: Throwable => { + println("testingIntVsTimestamp Failed") + return false + } + } + } + case ex: Throwable => { + println("testingIntVsTimeStamp Failed") + return false + } + } + } + + def testingIntVsBinary(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "int" or field("_id") === "binary").toDF.collect + println("testingIntVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingIntVsBinary Succeeded") + return true + } + case ex: Throwable => { + println("testingIntVsBinary Failed") + return false + } + } + } + case ex: Throwable => { + println("testingIntVsBinary Failed") + return false + } + } + } + + def testingIntVsMap(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "int" or field("_id") === "map").toDF.collect + println("testingIntVsMap Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingIntVsMap Succeeded") + return true + } + case ex: Throwable => { + println("testingIntVsMap Failed") + return false + } + } + } + case ex: Throwable => { + println("testingShortVsMap Failed") + return false + } + } + } + + def testingIntVsArray(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "int" or field("_id") === "array").toDF.collect + println("testingIntVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingIntVsArray Succeeded") + return true + } + case ex: Throwable => { + println("testingIntVsArray Failed") + return false + } + } + } + case ex: Throwable => { + println("testingIntVsArray Failed") + return false + } + } + } + + def testingLongVsFloat(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "long" or field("_id") === "float").toDF.collect + println("testingLongVsFloat Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingLongVsFloat Succeeded") + return true + } + case ex: Throwable => { + println("testingLongVsFloat Failed") + return false + } + } + } + case ex: Throwable => { + println("testingLongVsFloat Failed") + return false + } + } + } + + def testingLongVsDouble(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "long" or field("_id") === "double").toDF + if (booleanAndNullDF.collect.map(row => row.getDouble(row.fieldIndex("long"))).toSet.equals(Set(12345678999d, 10.1234567891d))) { + println("testingLongVsDouble Succeeded") + return true + } + else { + booleanAndNullDF.collect.map(row => row.getString(row.fieldIndex("long"))).foreach(println) + println("testingLongVsDouble Failed") + return false + } + } + + def testingLongVsDate(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "long" or field("_id") === "date").toDF.collect + println("testingLongVsDate Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingLongVsDate Succeeded") + return true + } + case ex: Throwable => { + println("testingLongVsDate Failed") + return false + } + } + } + case ex: Throwable => { + println("testingLongVsDate Failed") + return false + } + } + } + + def testingLongVsTime(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "long" or field("_id") === "time").toDF.collect + println("testingLongVsTime Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingLongVsTime Succeeded") + return true + } + case ex: Throwable => { + println("testingLongVsTime Failed") + return false + } + } + } + case ex: Throwable => { + println("testingLongVsTime Failed") + return false + } + } + } + + def testingLongVsTimeStamp(spark: SparkSession, tableName: String): Boolean = { + try{ + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "long" or field("_id") === "timestamp").toDF.collect + println("testingLongVsTimeStamp Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingLongVsTimestamp Succeeded") + return true + } + case ex: Throwable => { + println("testingLongVsTimestamp Failed") + return false + } + } + } + case ex: Throwable => { + println("testingLongVsTimeStamp Failed") + return false + } + } + } + + def testingLongVsBinary(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "long" or field("_id") === "binary").toDF.collect + println("testingLongVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingLongVsBinary Succeeded") + return true + } + case ex: Throwable => { + println("testingLongVsBinary Failed") + return false + } + } + } + case ex: Throwable => { + println("testingLongVsBinary Failed") + return false + } + } + } + + def testingLongVsMap(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "long" or field("_id") === "map").toDF.collect + println("testingLongVsMap Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingLongVsMap Succeeded") + return true + } + case ex: Throwable => { + println("testingLongVsMap Failed") + return false + } + } + } + case ex: Throwable => { + println("testingLongVsMap Failed") + return false + } + } + } + + def testingLongVsArray(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "long" or field("_id") === "array").toDF.collect + println("testingLongVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingLongVsArray Succeeded") + return true + } + case ex: Throwable => { + println("testingLongVsArray Failed") + return false + } + } + } + case ex: Throwable => { + println("testingLongVsArray Failed") + return false + } + } + } + + def testingFloatVsDouble(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "float" or field("_id") === "double").toDF + if (booleanAndNullDF.collect.map(row => row.getDouble(row.fieldIndex("float"))).toSet.equals(Set(10.12339973449707d, 10.1234567891d))) { + println("testingFloatVsDouble Succeeded") + return true + } + else { + booleanAndNullDF.collect.map(row => row.getDouble(row.fieldIndex("float"))).foreach(println) + println("testingFloatVsDouble Failed") + return false + } + } + + def testingFloatVsDate(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "float" or field("_id") === "date").toDF.collect + println("testingFloatVsDate Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingFloatVsDate Succeeded") + return true + } + case ex: Throwable => { + println("testingFloatVsDate Failed") + return false + } + } + } + case ex: Throwable => { + println("testingFloatVsDate Failed") + return false + } + } + } + + def testingFloatVsTime(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "float" or field("_id") === "time").toDF.collect + println("testingFloatVsTime Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingFloatVsTime Succeeded") + return true + } + case ex: Throwable => { + println("testingFloatVsTime Failed") + return false + } + } + } + case ex: Throwable => { + println("testingFloatVsTime Failed") + return false + } + } + } + + def testingFloatVsTimeStamp(spark: SparkSession, tableName: String): Boolean = { + try{ + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "float" or field("_id") === "timestamp").toDF.collect + println("testingFloatVsTimeStamp Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingFloatVsTimestamp Succeeded") + return true + } + case ex: Throwable => { + println("testingFloatVsTimestamp Failed") + return false + } + } + } + case ex: Throwable => { + println("testingFloatVsTimeStamp Failed") + return false + } + } + } + + def testingFloatVsBinary(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "float" or field("_id") === "binary").toDF.collect + println("testingFloatVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingFloatVsBinary Succeeded") + return true + } + case ex: Throwable => { + println("testingFloatVsBinary Failed") + return false + } + } + } + case ex: Throwable => { + println("testingFloatsBinary Failed") + return false + } + } + } + + def testingFloatVsMap(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "float" or field("_id") === "map").toDF.collect + println("testingFloatVsMap Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingFloatVsMap Succeeded") + return true + } + case ex: Throwable => { + println("testingFloatVsMap Failed") + return false + } + } + } + case ex: Throwable => { + println("testingFloatVsMap Failed") + return false + } + } + } + + def testingFloatVsArray(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "float" or field("_id") === "array").toDF.collect + println("testingFloatVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingFloatVsArray Succeeded") + return true + } + case ex: Throwable => { + println("testingFloatVsArray Failed") + return false + } + } + } + case ex: Throwable => { + println("testingFloatVsArray Failed") + return false + } + } + } + + + def testingDoubleVsDate(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "double" or field("_id") === "date").toDF.collect + println("testingDoubleVsDate Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingDoubleVsDate Succeeded") + return true + } + case ex: Throwable => { + println("testingDoubleVsDate Failed") + return false + } + } + } + case ex: Throwable => { + println("testingDoubleVsDate Failed") + return false + } + } + } + + def testingDoubleVsTime(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "double" or field("_id") === "time").toDF.collect + println("testingDoubleVsTime Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingDoubleVsTime Succeeded") + return true + } + case ex: Throwable => { + println("testingDoubleVsTime Failed") + return false + } + } + } + case ex: Throwable => { + println("testingDoubleVsTime Failed") + return false + } + } + } + + def testingDoubleVsTimeStamp(spark: SparkSession, tableName: String): Boolean = { + try{ + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "double" or field("_id") === "timestamp").toDF.collect + println("testingDoubleVsTimeStamp Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingDoubleVsTimestamp Succeeded") + return true + } + case ex: Throwable => { + println("testingDoubleVsTimestamp Failed") + return false + } + } + } + case ex: Throwable => { + println("testingDoubleVsTimeStamp Failed") + return false + } + } + } + + def testingDoubleVsBinary(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "double" or field("_id") === "binary").toDF.collect + println("testingDoubleVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingDoubleVsBinary Succeeded") + return true + } + case ex: Throwable => { + println("testingDoubleVsBinary Failed") + return false + } + } + } + case ex: Throwable => { + println("testingDoubleBinary Failed") + return false + } + } + } + + def testingDoubleVsMap(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "double" or field("_id") === "map").toDF.collect + println("testingDoubleVsMap Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingDoubleVsMap Succeeded") + return true + } + case ex: Throwable => { + println("testingDoubleVsMap Failed") + return false + } + } + } + case ex: Throwable => { + println("testingDoubleVsMap Failed") + return false + } + } + } + + def testingDoubleVsArray(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "double" or field("_id") === "array").toDF.collect + println("testingDoubleVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingDoubleVsArray Succeeded") + return true + } + case ex: Throwable => { + println("testingDoubleVsArray Failed") + return false + } + } + } + case ex: Throwable => { + println("testingDoubleVsArray Failed") + return false + } + } + } + + def testingDateVsTime(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "date" or field("_id") === "time").toDF + if (booleanAndNullDF.collect.map(row => new OTimestamp(row.getTimestamp(row.fieldIndex("time")).getTime)).toSet.equals( + Set(OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr), new OTimestamp(OTime.fromMillisOfDay(1000).toDate.getTime)))) { + println("testingDateVsTime Succeeded") + return true + } + else { + booleanAndNullDF.collect.map(row => new OTimestamp(row.getTimestamp(row.fieldIndex("time")))).foreach(println) + Set(OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr), new OTimestamp(OTime.fromMillisOfDay(1000).toDate.getTime)).foreach(println) + println("testingDateVsTime Failed") + return false + } + } + + def testingDateVsTimeStamp(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "date" or field("_id") === "timestamp").toDF + if (booleanAndNullDF.collect.map(row => new OTimestamp(row.getTimestamp(row.fieldIndex("timestamp")))).toSet.equals( + Set(OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr), OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)))) { + println("testingDateVsTimeStamp Succeeded") + return true + } + else { + booleanAndNullDF.collect.map(row => new OTimestamp(row.getTimestamp(row.fieldIndex("timestamp")))).foreach(println) + Set(OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr), OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)).foreach(println) + println("testingDateVsTimeStamp Failed") + return false + } + } + + def testingDateVsBinary(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "date" or field("_id") === "binary").toDF.collect + println("testingDateVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingDateVsBinary Succeeded") + return true + } + case ex: Throwable => { + println("testingDateVsBinary Failed") + return false + } + } + } + case ex: Throwable => { + println("testingDateBinary Failed") + return false + } + } + } + + def testingDateVsMap(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "date" or field("_id") === "map").toDF.collect + println("testingDateVsMap Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingDateVsMap Succeeded") + return true + } + case ex: Throwable => { + println("testingDateVsMap Failed") + return false + } + } + } + case ex: Throwable => { + println("testingDateVsMap Failed") + return false + } + } + } + + def testingDateVsArray(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "date" or field("_id") === "array").toDF.collect + println("testingDateVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingDateVsArray Succeeded") + return true + } + case ex: Throwable => { + println("testingDateVsArray Failed") + return false + } + } + } + case ex: Throwable => { + println("testingDateVsArray Failed") + return false + } + } + } + + def testingTimeVsTimeStamp(spark: SparkSession, tableName: String): Boolean = { + val booleanAndNullDF = spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "time" or field("_id") === "timestamp").toDF + if (booleanAndNullDF.collect.map(row => new OTimestamp(row.getTimestamp(row.fieldIndex("time")))).toSet.equals( + Set(OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr), new OTimestamp(OTime.fromMillisOfDay(1000).toDate.getTime)))) { + println("testingTimeVsTimeStamp Succeeded") + return true + } + else { + booleanAndNullDF.collect.map(row => new OTimestamp(row.getTimestamp(row.fieldIndex("time")))).foreach(println) + //Set(OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr), OTimestamp.parse(ODate.fromDaysSinceEpoch(1000).toDateStr)).foreach(println) + println("testingTimeVsTimeStamp Failed") + return false + } + } + + def testingTimeVsBinary(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "time" or field("_id") === "binary").toDF.collect + println("testingTimeVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingTimeVsBinary Succeeded") + return true + } + case ex: Throwable => { + println("testingTimeVsBinary Failed") + return false + } + } + } + case ex: Throwable => { + println("testingTimeBinary Failed") + return false + } + } + } + + def testingTimeVsMap(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "time" or field("_id") === "map").toDF.collect + println("testingTimeVsMap Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingTimeVsMap Succeeded") + return true + } + case ex: Throwable => { + println("testingTimeVsMap Failed") + return false + } + } + } + case ex: Throwable => { + println("testingTimeVsMap Failed") + return false + } + } + } + + def testingTimeVsArray(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "time" or field("_id") === "array").toDF.collect + println("testingTimeVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingTimeVsArray Succeeded") + return true + } + case ex: Throwable => { + println("testingTimeVsArray Failed") + return false + } + } + } + case ex: Throwable => { + println("testingTimeVsArray Failed") + return false + } + } + } + + def testingTimeStampVsBinary(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "timestamp" or field("_id") === "binary").toDF.collect + println("testingTimeStampVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingTimeStampVsBinary Succeeded") + return true + } + case ex: Throwable => { + println("testingTimeStampVsBinary Failed") + return false + } + } + } + case ex: Throwable => { + println("testingTimeStampBinary Failed") + return false + } + } + } + + def testingTimeStampVsMap(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "timestamp" or field("_id") === "map").toDF.collect + println("testingTimeStampVsMap Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingTimeStampVsMap Succeeded") + return true + } + case ex: Throwable => { + println("testingTimeStampVsMap Failed") + return false + } + } + } + case ex: Throwable => { + println("testingTimeStampVsMap Failed") + return false + } + } + } + + def testingTimeStampVsArray(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "timestamp" or field("_id") === "array").toDF.collect + println("testingTimeStampVsBinary Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingTimeStampVsArray Succeeded") + return true + } + case ex: Throwable => { + println("testingTimeStampVsArray Failed") + return false + } + } + } + case ex: Throwable => { + println("testingTimeStampVsArray Failed") + return false + } + } + } + + def testingBinaryVsMap(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "binary" or field("_id") === "map").toDF.collect + println("testingBinaryVsMap Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingBinaryVsMap Succeeded") + return true + } + case ex: Throwable => { + println("testingBinaryVsMap Failed") + return false + } + } + } + case ex: Throwable => { + println("testingBinaryVsMap Failed") + return false + } + } + } + + def testingBinaryVsArray(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "binary" or field("_id") === "array").toDF.collect + println("testingBinaryVsArray Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingBinaryVsArray Succeeded") + return true + } + case ex: Throwable => { + println("testingBinaryVsArray Failed") + return false + } + } + } + case ex: Throwable => { + println("testingBinaryVsArray Failed") + return false + } + } + } + + def testingArrayVsMap(spark: SparkSession, tableName: String): Boolean = { + try { + spark.sparkContext.loadFromMapRDB(tableName).where(field("_id") === "array" or field("_id") === "map").toDF.collect + println("testingArrayVsMap Failed") + return false + } catch { + case ex: SparkException => { + ex.getCause match { + case ex: SchemaMappingException => { + println("testingArrayVsMap Succeeded") + return true + } + case ex: Throwable => { + println("testingArrayVsMap Failed") + return false + } + } + } + case ex: Throwable => { + println("testingArrayVsMap Failed") + return false + } + } + } + + def testingCustomer(spark: SparkSession, tableName: String): Boolean = { + spark.sqlContext.read.format("com.mapr.db.spark.sql.DefaultSource").option("tableName", "/tmp/customer").load().select("features").show(10) + println("testingCustomer Failed") + return true + } + + def runTests(sparkSession: SparkSession): Unit = { + testingBooleanVsNull(sparkSession, tableName) + testingStringVsNull(sparkSession, tableName) + testingByteVsNull(sparkSession, tableName) + testingShortVsNull(sparkSession, tableName) + testingIntVsNull(sparkSession, tableName) + testingLongVsNull(sparkSession, tableName) + testingFloatVsNull(sparkSession, tableName) + testingDoubleVsNull(sparkSession, tableName) + testingDateVsNull(sparkSession, tableName) + testingTimeVsNull(sparkSession, tableName) + testingTimeStampVsNull(sparkSession, tableName) + testingBinaryVsNull(sparkSession, tableName) + testingMapVsNull(sparkSession, tableName) + testingArrayVsNull(sparkSession, tableName) + testingBooleanVsString(sparkSession, tableName) + testingBooleanVsByte(sparkSession, tableName) + testingBooleanVsShort(sparkSession, tableName) + testingBooleanVsInt(sparkSession, tableName) + testingBooleanVsLong(sparkSession, tableName) + testingBooleanVsFloat(sparkSession, tableName) + testingBooleanVsDouble(sparkSession, tableName) + testingBooleanVsDate(sparkSession, tableName) + testingBooleanVsTime(sparkSession, tableName) + testingBooleanVsTimeStamp(sparkSession, tableName) + testingBooleanVsBinary(sparkSession, tableName) + testingBooleanVsMap(sparkSession, tableName) + testingBooleanVsArray(sparkSession, tableName) + testingStringVsByte(sparkSession, tableName) + testingStringVsShort(sparkSession, tableName) + testingStringVsInt(sparkSession, tableName) + testingStringVsLong(sparkSession, tableName) + testingStringVsFloat(sparkSession, tableName) + testingStringVsDouble(sparkSession, tableName) + testingStringVsDate(sparkSession, tableName) + testingStringVsTime(sparkSession, tableName) + testingStringVsTimeStamp(sparkSession, tableName) + testingStringVsBinary(sparkSession, tableName) + testingStringVsMap(sparkSession, tableName) + testingStringVsArray(sparkSession, tableName) + testingByteVsShort(sparkSession, tableName) + testingByteVsInt(sparkSession, tableName) + testingByteVsLong(sparkSession, tableName) + testingByteVsFloat(sparkSession, tableName) + testingByteVsLong(sparkSession, tableName) + testingByteVsFloat(sparkSession, tableName) + testingByteVsDouble(sparkSession, tableName) + testingByteVsDate(sparkSession, tableName) + testingByteVsTime(sparkSession, tableName) + testingByteVsTimeStamp(sparkSession, tableName) + testingByteVsBinary(sparkSession, tableName) + testingByteVsMap(sparkSession, tableName) + testingByteVsArray(sparkSession, tableName) + testingShortVsInt(sparkSession, tableName) + testingShortVsLong(sparkSession, tableName) + testingShortVsFloat(sparkSession, tableName) + testingShortVsLong(sparkSession, tableName) + testingShortVsFloat(sparkSession, tableName) + testingShortVsDouble(sparkSession, tableName) + testingShortVsDate(sparkSession, tableName) + testingShortVsTime(sparkSession, tableName) + testingShortVsTimeStamp(sparkSession, tableName) + testingShortVsBinary(sparkSession, tableName) + testingShortVsMap(sparkSession, tableName) + testingShortVsArray(sparkSession, tableName) + testingIntVsLong(sparkSession, tableName) + testingIntVsFloat(sparkSession, tableName) + testingIntVsDouble(sparkSession, tableName) + testingIntVsDate(sparkSession, tableName) + testingIntVsTime(sparkSession, tableName) + testingIntVsTimeStamp(sparkSession, tableName) + testingIntVsBinary(sparkSession, tableName) + testingIntVsMap(sparkSession, tableName) + testingIntVsArray(sparkSession, tableName) + testingLongVsFloat(sparkSession, tableName) + testingLongVsDouble(sparkSession, tableName) + testingLongVsDate(sparkSession, tableName) + testingLongVsTime(sparkSession, tableName) + testingLongVsTimeStamp(sparkSession, tableName) + testingLongVsBinary(sparkSession, tableName) + testingLongVsMap(sparkSession, tableName) + testingLongVsArray(sparkSession, tableName) + testingFloatVsDouble(sparkSession, tableName) + testingFloatVsDate(sparkSession, tableName) + testingFloatVsTime(sparkSession, tableName) + testingFloatVsTimeStamp(sparkSession, tableName) + testingFloatVsBinary(sparkSession, tableName) + testingFloatVsMap(sparkSession, tableName) + testingFloatVsArray(sparkSession, tableName) + testingDoubleVsDate(sparkSession, tableName) + testingDoubleVsTime(sparkSession, tableName) + testingDoubleVsTimeStamp(sparkSession, tableName) + testingDoubleVsBinary(sparkSession, tableName) + testingDoubleVsMap(sparkSession, tableName) + testingDoubleVsArray(sparkSession, tableName) + testingDateVsTime(sparkSession, tableName) + testingDateVsTimeStamp(sparkSession, tableName) + testingDateVsBinary(sparkSession, tableName) + testingDateVsArray(sparkSession, tableName) + testingDateVsMap(sparkSession, tableName) + testingTimeVsTimeStamp(sparkSession, tableName) + testingTimeVsBinary(sparkSession, tableName) + testingTimeVsMap(sparkSession, tableName) + testingTimeVsArray(sparkSession, tableName) + testingTimeStampVsBinary(sparkSession, tableName) + testingTimeStampVsMap(sparkSession, tableName) + testingTimeStampVsArray(sparkSession, tableName) + testingBinaryVsArray(sparkSession, tableName) + testingBinaryVsMap(sparkSession, tableName) + testingArrayVsMap(sparkSession, tableName) + } +} + +object SparkSqlAccessTestsWithKryo { + val tableName="/tmp/SparkSqlOjaiConnectorAccessTesting" + lazy val conf = new SparkConf() + .setAppName("SparkSqlAccessTestsWithKryo") + .set("spark.executor.memory", "1g") + .set("spark.driver.memory", "1g") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "com.mapr.db.spark.OJAIKryoRegistrator") + + lazy val spark = SparkSession.builder().appName("SparkSqlAccessTestsWithKryo").config(conf).getOrCreate() + + + def main(args: Array[String]): Unit = { + SparkSqlAccessTests.tableInitialization(tableName) + SparkSqlAccessTests.runTests(spark) + } +} diff --git a/external/maprdb/src/test/scala/com/mapr/db/testCases/SparkSqlLoadAndSaveTests.scala b/external/maprdb/src/test/scala/com/mapr/db/testCases/SparkSqlLoadAndSaveTests.scala new file mode 100644 index 0000000000000..01f3bca258d89 --- /dev/null +++ b/external/maprdb/src/test/scala/com/mapr/db/testCases/SparkSqlLoadAndSaveTests.scala @@ -0,0 +1,446 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.testCases + +import java.util.Collections + +import com.mapr.db.MapRDB +import org.apache.spark.SparkConf +import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import com.mapr.db.spark.sql._ +import com.mapr.db.testCases.BeanTest1.User1 +import org.apache.spark.sql.types._ +import com.mapr.db.MapRDB +import com.mapr.db.rowcol.{DBValueBuilder, DBValueBuilderImpl} +import com.mapr.db.spark._ +import org.ojai.types.ODate + + +object SparkSqlLoadAndSaveTests { + lazy val conf = new SparkConf() + .setAppName("SparkSqlLoadAndSaveTests") + .set("spark.executor.memory", "1g") + .set("spark.driver.memory", "1g") + .setMaster("local[*]") + + lazy val spark = SparkSession.builder().appName("SparkSqlLoadAndSaveTests").config(conf).getOrCreate() + + val tableName = "/tmp/SparkSqlOjaiConnectorLoadAndSaveTesting" + val saveTableName = "/tmp/SparkSqlOjaiConnectorLoadAndSaveTesting_save" + + def main(args: Array[String]): Unit = { + MapRDBSparkTests.tableInitialization(spark.sparkContext, tableName) + SparkSqlLoadAndSaveTests.runTests(spark) + } + + def testingLoadBeanClass(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val users : DataFrame = spark.loadFromMapRDB[User1](tableName) + val andrewProfile : Array[String] = users.filter($"first_name" === "Andrew").collect.map(r => r.getString(r.fieldIndex("first_name"))) + if (andrewProfile.toSet.equals(Set("Andrew"))) { + println("testingLoadBeanClass succeeded") + return true + } + else { + println("testingLoadBeanClass failed") + return false + } + } + + def testingLoadExplicitSchema(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val addressSchema = StructType(StructField("Pin", IntegerType) :: + StructField("city", StringType) :: + StructField("street", StringType) :: Nil) + + val schema = StructType(StructField("_id", StringType) :: + StructField("first_name", StringType) :: + StructField("last_name", StringType) :: + StructField("address", addressSchema) :: + StructField("interests", ArrayType(StringType)) :: Nil) + + val users : DataFrame = spark.loadFromMapRDB(tableName, schema) + val andrewProfile : Array[String] = users.filter($"first_name" === "Andrew").collect.map(r => r.getString(r.fieldIndex("first_name"))) + if (andrewProfile.toSet.equals(Set("Andrew"))) { + println("testingLoadExplicitSchema succeeded") + return true + } + else { + println("testingLoadExplicitSchema failed") + return false + } + } + + def testingLoadInferSchema(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val users : DataFrame = spark.loadFromMapRDB(tableName) + val andrewProfile : Array[String] = users.filter($"first_name" === "Andrew").collect.map(r => r.getString(r.fieldIndex("first_name"))) + if (andrewProfile.toSet.equals(Set("Andrew"))) { + println("testingLoadBeanClass succeeded") + return true + } + else { + println("testingLoadBeanClass failed") + return false + } + } + + def testingLoadFromDFReader(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val users : DataFrame = spark.read.maprdb(tableName) + val andrewProfile : Array[String] = users.filter($"first_name" === "Andrew").collect.map(r => r.getString(r.fieldIndex("first_name"))) + if (andrewProfile.toSet.equals(Set("Andrew"))) { + println("testingLoadFromDFReader succeeded") + return true + } + else { + println("testingLoadFromDFReader failed") + return false + } + } + + def testingLoadFromDFReaderLoad(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val users : DataFrame = spark.read.format("com.mapr.db.spark.sql").option("tableName", tableName).option("sampleSize", 100).load() + val andrewProfile : Array[String] = users.filter($"first_name" === "Andrew").collect.map(r => r.getString(r.fieldIndex("first_name"))) + if (andrewProfile.toSet.equals(Set("Andrew"))) { + println("testingLoadFromDFReaderLoad succeeded") + return true + } + else { + println("testingLoadFromDFReaderLoad failed") + return false + } + } + + def testingLoadFromDFWriterWithOperationOption(spark: SparkSession, tableName: String): Boolean = { + if (MapRDB.tableExists(tableName+"writer_save")) + MapRDB.deleteTable(tableName+"writer_save") + import spark.implicits._ + val users : DataFrame = spark.read.option("sampleSize", 100).maprdb(tableName) + users.write.option("Operation", "Insert").saveToMapRDB(tableName+"writer_save") + val collection = spark.sparkContext.loadFromMapRDB(tableName+"writer_save").collect + if (collection.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"],\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"address\":null,\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"interests\":null,\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"address\":null,\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"interests\":null,\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"address\":null,\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}"))) + { + println("testingLoadFromDFWriterWithOperationOption succeded") + true + } + else { + println("testingLoadFromDFWriterWithOperationOption failed") + collection.map(a => a.asJsonString).foreach(println(_)) + false + } + } + + def testingUpdateToMapRDB(spark: SparkSession, tableName: String): Boolean = { + if (MapRDB.tableExists(tableName+"writer_save")) + MapRDB.deleteTable(tableName+"writer_save") + import spark.implicits._ + val users : DataFrame = spark.read.option("sampleSize", 100).maprdb(tableName) + users.write.saveToMapRDB(tableName+"writer_save") + users.updateToMapRDB(tableName+"writer_save", (row: Row) => { + val mutation = MapRDB.newMutation() + mutation.setOrReplace("key", row.getString(row.fieldIndex("_id"))) + mutation + }, (row: Row) => DBValueBuilderImpl.KeyValueBuilder.initFrom(row.getString(row.fieldIndex("_id")))) + val collection = spark.sparkContext.loadFromMapRDB(tableName+"writer_save").collect + if (collection.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"],\"key\":\"rsmith\",\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"],\"key\":\"mdupont\",\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"address\":null,\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"interests\":null,\"key\":\"jdoe\",\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"address\":null,\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"interests\":null,\"key\":\"dsimon\",\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"address\":null,\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"key\":\"alehmann\",\"last_name\":\"Lehmann\"}"))) + { + println("testingUpdateToMapRDB succeded") + true + } + else { + println("testingUpdateToMapRDB failed") + collection.map(a => a.asJsonString).foreach(println(_)) + false + } + } + + def testingCheckAndUpdateToMapRDB(spark: SparkSession, tableName: String): Boolean = { + if (MapRDB.tableExists(tableName+"writer_save")) + MapRDB.deleteTable(tableName+"writer_save") + import spark.implicits._ + val users : DataFrame = spark.read.option("sampleSize", 100).maprdb(tableName) + users.write.saveToMapRDB(tableName+"writer_save") + users.updateToMapRDB(tableName+"writer_save", (row: Row) => { + val mutation = MapRDB.newMutation() + mutation.setOrReplace("key", row.getString(row.fieldIndex("_id"))) + mutation + }, (row: Row) => DBValueBuilderImpl.KeyValueBuilder.initFrom(row.getString(row.fieldIndex("_id"))), field("_id") === "rsmith") + val collection = spark.sparkContext.loadFromMapRDB(tableName+"writer_save").collect + if (collection.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"],\"key\":\"rsmith\",\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"],\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"address\":null,\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"interests\":null,\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"address\":null,\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"interests\":null,\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"address\":null,\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}"))) + { + println("testingCheckAndUpdateToMapRDB succeded") + true + } + else { + println("testingCheckAndUpdateToMapRDB failed") + collection.map(a => a.asJsonString).foreach(println(_)) + false + } + } + + def testingUpdateToMapRDBAddToArray(spark: SparkSession, tableName: String): Boolean = { + if (MapRDB.tableExists(tableName+"writer_modify_array")) + MapRDB.deleteTable(tableName+"writer_modify_array") + import spark.implicits._ + val docs = scala.collection.immutable.List("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\"," + + "\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\",\"electronics\"],\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}") + val docsrdd = spark.sparkContext.parallelize(docs) + val ojairdd = docsrdd.map(doc => MapRDBSpark.newDocument(doc)).map(doc => { + doc.dob = Option(doc.dob) match { + case Some(a) => ODate.parse(doc.dob[String]) + case None => null + } + doc + }).map( doc => { + if (doc.getList("interests") == null) + doc.set("interests", Seq()) + else doc + }).saveToMapRDB(tableName+"writer_modify_array", createTable = true) + val users : DataFrame = spark.read.option("sampleSize", 100).maprdb(tableName) + users.updateToMapRDB(tableName+"writer_modify_array", (row: Row) => { + val mutation = MapRDB.newMutation() + mutation.append("interests", Collections.singletonList("cpp")) + mutation + }, (row: Row) => DBValueBuilderImpl.KeyValueBuilder.initFrom(row.getString(row.fieldIndex("_id")))) + val collection = spark.sparkContext.loadFromMapRDB(tableName+"writer_modify_array").collect + if (collection.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\",\"cpp\"],\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\",\"cpp\"],\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"interests\":[\"cpp\"],\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"interests\":[\"cpp\"],\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\",\"cpp\"],\"last_name\":\"Lehmann\"}"))) + { + println("testingUpdateToMapRDBAddToArray succeded") + true + } + else { + println("testingUpdateToMapRDBAddToArray failed") + collection.map(a => a.asJsonString).foreach(println(_)) + false + } + } + + def testingLoadFromDFReaderWithOperationOption(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val users : DataFrame = spark.read.option("sampleSize", 100).option("Operation","Insert").maprdb(tableName) + val andrewProfile : Array[String] = users.filter($"first_name" === "Andrew").collect.map(r => r.getString(r.fieldIndex("first_name"))) + if (andrewProfile.toSet.equals(Set("Andrew"))) { + println("testingLoadFromDFReaderWithOperationOption succeeded") + return true + } + else { + println("testingLoadFromDFReaderWithOperationOption failed") + return false + } + } + + def tesitngLoadFromDFReaderWithSampleOption(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val users : DataFrame = spark.read.format("com.mapr.db.spark.sql").option("tableName", tableName).option("sampleSize", 1000).maprdb(tableName) + val andrewProfile : Array[String] = users.filter($"first_name" === "Andrew").collect.map(r => r.getString(r.fieldIndex("first_name"))) + if (andrewProfile.toSet.equals(Set("Andrew"))) { + println("tesitngLoadFromDFReaderWithSampleOption succeeded") + return true + } + else { + println("tesitngLoadFromDFReaderWithSampleOption failed") + return false + } + } + + def testingLoadFromDFReaderWithFailOnConflict(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val users : DataFrame = spark.read.option("sampleSize", 1000).option("FailOnConflict","true").maprdb(tableName) + val andrewProfile : Array[String] = users.filter($"first_name" === "Andrew").collect.map(r => r.getString(r.fieldIndex("first_name"))) + if (andrewProfile.toSet.equals(Set("Andrew"))) { + println("testingLoadFromDFReaderWithFailOnConflict succeeded") + return true + } + else { + println("testingLoadFromDFReaderWithFailOnConflict failed") + return false + } + } + + def testingSaveFromSparkSession(spark: SparkSession, tableName: String): Boolean = { + if (MapRDB.tableExists(saveTableName)) + MapRDB.deleteTable(saveTableName) + import spark.implicits._ + val users : DataFrame = spark.read.option("sampleSize", 1000).option("FailOnConflict","true").maprdb(tableName) + users.saveToMapRDB(saveTableName, createTable = true) + val savedUsers : DataFrame = spark.read.format("com.mapr.db.spark.sql").option("tableName", tableName).option("sampleSize", 1000).option("FailOnConflict","true").maprdb(saveTableName) + val andrewProfile : Array[String] = savedUsers.filter($"first_name" === "Andrew").collect.map(r => r.getString(r.fieldIndex("first_name"))) + if (andrewProfile.toSet.equals(Set("Andrew"))) { + println("testingSaveFromSparkSession succeeded") + return true + } + else { + println("testingSaveFromSparkSession failed") + return false + } + } + + def testingSaveFromDFWSession(spark: SparkSession, tableName: String): Boolean = { + if (MapRDB.tableExists(saveTableName)) + MapRDB.deleteTable(saveTableName) + import spark.implicits._ + val users : DataFrame = spark.read.option("sampleSize", 1000).maprdb(tableName) + users.saveToMapRDB(saveTableName, createTable = true) + val savedUsers : DataFrame = spark.read.option("sampleSize", 1000).maprdb(saveTableName) + val andrewProfile : Array[String] = savedUsers.filter($"first_name" === "Andrew").collect.map(r => r.getString(r.fieldIndex("first_name"))) + if (andrewProfile.toSet.equals(Set("Andrew"))) { + println("testingSaveFromDFWSession succeeded") + return true + } + else { + println("testingSaveFromDFWSession failed") + return false + } + } + + def testingSaveWithBulkLoad(spark: SparkSession, tableName: String): Boolean = { + if (MapRDB.tableExists(saveTableName)) + MapRDB.deleteTable(saveTableName) + import spark.implicits._ + val users : DataFrame = spark.read.option("sampleSize", 1000).maprdb(tableName) + users.saveToMapRDB(saveTableName, createTable = true , bulkInsert = true) + val savedUsers : DataFrame = spark.read.option("sampleSize", 1000).maprdb(saveTableName) + val andrewProfile : Array[String] = savedUsers.filter($"first_name" === "Andrew").collect.map(r => r.getString(r.fieldIndex("first_name"))) + if (andrewProfile.toSet.equals(Set("Andrew"))) { + println("testingSaveWithBulkLoad succeeded") + return true + } + else { + println("testingSaveWithBulkLoad failed") + return false + } + } + + def testingSaveWithComplexDocument(spark: SparkSession): Boolean = { + if (MapRDB.tableExists(saveTableName+"complex")) + MapRDB.deleteTable(saveTableName+"complex") + + if (MapRDB.tableExists(saveTableName+"complex_save")) + MapRDB.deleteTable(saveTableName+"complex_save") + + import com.mapr.db.spark._ + val listOfDocs = List("{\"_id\":\"2DT3201\",\"brand\":\"Careen\",\"category\":\"Pedals\",\"features\":[\"Low-profile design\",\"Floating SH11 cleats included\"],\"name\":\" Allegro SPD-SL 6800\",\"price\":112.99,\"product_ID\":\"2DT3201\",\"specifications\":{\"color\":\"black\",\"weight_per_pair\":\"260g\"},\"type\":\"Components\"}", + "{\"_id\":\"3ML6758\",\"brand\":\"Careen\",\"category\":\"Jersey\",\"features\":[\"Wicks away moisture.\",\"SPF-30\",\"Reflects light at night.\"],\"name\":\"Trikot 24-LK\",\"price\":76.99,\"product_ID\":\"3ML6758\",\"specifications\":{\"colors\":[\"white\",\"navy\",\"green\"],\"sizes\":[\"S\",\"M\",\"L\",\"XL\",\"XXL\"]},\"type\":\"Clothing\"}", + "{\"_id\":\"4GGC859\",\"brand\":\"Careen\",\"category\":\"Bicycle\",\"name\":\"Thresher 1000\",\"price\":2949.99,\"product_ID\":\"4GGC859\",\"specifications\":{\"frameset\":{\"fork\":\"Gabel 2\",\"frame\":\"Carbon Enduro\"},\"groupset\":{\"brake\":\"Bremse FullStop\",\"chainset\":\"Kette 230\"},\"size\":\"55cm\",\"wheel_size\":\"700c\",\"wheelset\":{\"tyres\":\"Reifen Pro\",\"wheels\":\"Rad Schnell 10\"}},\"type\":\"Road bicycle\"}") + val schema = StructType(StructField("_id",StringType,true) :: + StructField("brand",StringType,true) :: + StructField("category",StringType,true) :: + StructField("features",ArrayType(StringType,true),true) :: + StructField("name",StringType,true) :: + StructField("price",DoubleType,true) :: + StructField("product_ID",StringType,true) :: + StructField("specifications",StructType(StructField("color",StringType,true) :: + StructField("colors",ArrayType(StringType,true),true) :: + StructField("frameset",StructType(StructField("fork",StringType,true) :: + StructField("frame",StringType,true) :: Nil),true) :: + StructField("groupset",StructType(StructField("brake",StringType,true) :: + StructField("chainset",StringType,true) :: Nil),true) :: + StructField("size",StringType,true) :: + StructField("sizes",ArrayType(StringType,true),true) :: + StructField("weight_per_pair", StringType, true) :: + StructField("wheel_size", StringType, true) :: + StructField("wheelset", StructType(StructField("tyres", StringType, true) :: + StructField("wheels", StringType, true) :: Nil)) :: Nil)) :: + StructField("type", StringType, true) :: Nil) + +// val listOfRows = listOfDocs.map(doc => MapRDBSpark.newDocument(doc)).map(doc => MapRDBSpark.docToRow(doc, schema)) +// val docs = listOfRows.map(row => MapRDBSpark.rowToDoc(row)).map(_.asJsonString()) +// return true + spark.sparkContext.parallelize(listOfDocs).map(MapRDBSpark.newDocument(_)).saveToMapRDB(saveTableName+ "complex", createTable = true) + val df = spark.read.format("com.mapr.db.spark.sql.DefaultSource").option("tableName", saveTableName+ "complex").load(); +// df.printSchema() + df.write.format("com.mapr.db.spark.sql.DefaultSource").option("tableName", saveTableName+ "complex_save").save() + val collection = spark.sparkContext.loadFromMapRDB(saveTableName+"complex_save").collect + if (collection.map(_.asJsonString()).toSet.equals(Set("{\"_id\":\"2DT3201\",\"brand\":\"Careen\",\"category\":\"Pedals\",\"features\":[\"Low-profile design\",\"Floating SH11 cleats included\"],\"name\":\" Allegro SPD-SL 6800\",\"price\":112.99,\"product_ID\":\"2DT3201\",\"specifications\":{\"color\":\"black\",\"colors\":null,\"frameset\":null,\"groupset\":null,\"size\":null,\"sizes\":null,\"weight_per_pair\":\"260g\",\"wheel_size\":null,\"wheelset\":null},\"type\":\"Components\"}", + "{\"_id\":\"3ML6758\",\"brand\":\"Careen\",\"category\":\"Jersey\",\"features\":[\"Wicks away moisture.\",\"SPF-30\",\"Reflects light at night.\"],\"name\":\"Trikot 24-LK\",\"price\":76.99,\"product_ID\":\"3ML6758\",\"specifications\":{\"color\":null,\"colors\":[\"white\",\"navy\",\"green\"],\"frameset\":null,\"groupset\":null,\"size\":null,\"sizes\":[\"S\",\"M\",\"L\",\"XL\",\"XXL\"],\"weight_per_pair\":null,\"wheel_size\":null,\"wheelset\":null},\"type\":\"Clothing\"}", + "{\"_id\":\"4GGC859\",\"brand\":\"Careen\",\"category\":\"Bicycle\",\"features\":null,\"name\":\"Thresher 1000\",\"price\":2949.99,\"product_ID\":\"4GGC859\",\"specifications\":{\"color\":null,\"colors\":null,\"frameset\":{\"fork\":\"Gabel 2\",\"frame\":\"Carbon Enduro\"},\"groupset\":{\"brake\":\"Bremse FullStop\",\"chainset\":\"Kette 230\"},\"size\":\"55cm\",\"sizes\":null,\"weight_per_pair\":null,\"wheel_size\":\"700c\",\"wheelset\":{\"tyres\":\"Reifen Pro\",\"wheels\":\"Rad Schnell 10\"}},\"type\":\"Road bicycle\"}"))) { + println("testingSaveWithComplexDocument succeded") + true + } + else { + println("testingSaveWithComplexDocument failed") + collection.map(a => a.asJsonString).foreach(println(_)) + false + } + } + + def runTests(sparkSession: SparkSession): Unit = { + testingLoadExplicitSchema(spark, tableName) + testingLoadBeanClass(spark, tableName) + testingLoadInferSchema(spark, tableName) + testingLoadFromDFReader(spark, tableName) + testingLoadFromDFReaderLoad(spark, tableName) + testingLoadFromDFReaderWithOperationOption(spark, tableName) + tesitngLoadFromDFReaderWithSampleOption(spark, tableName) + testingLoadFromDFReaderWithFailOnConflict(spark, tableName) + testingSaveFromSparkSession(spark, tableName) + testingSaveFromDFWSession(spark, tableName) + testingSaveWithBulkLoad(spark, tableName) + testingSaveWithComplexDocument(spark) + } +} + +object SparkSqlLoadAndSaveTestsWithKryo { + + val tableName = "/tmp/SparkSqlOjaiConnectorLoadAndSaveTesting" + val saveTableName = "/tmp/SparkSqlOjaiConnectorLoadAndSaveTesting_save" + + lazy val conf = new SparkConf() + .setAppName("SparkSqlLoadAndSaveTestsWithKryo") + .set("spark.executor.memory", "1g") + .set("spark.driver.memory", "1g") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "com.mapr.db.spark.OJAIKryoRegistrator") + + lazy val spark = SparkSession.builder().appName("SparkSqlLoadAndSaveTestsWithKryo").config(conf).getOrCreate() + + + def main(args: Array[String]): Unit = { + MapRDBSparkTests.tableInitialization(spark.sparkContext,tableName) + SparkSqlLoadAndSaveTests.runTests(spark) + } +} + + +object BeanTest1 { + case class person(name: String, age: Integer) + + case class User1 (_id: String, first_name:String, + last_name: String, dob: java.sql.Date, + interests: Seq[String]) + +} diff --git a/external/maprdb/src/test/scala/com/mapr/db/testCases/SparkSqlPushDownTests.scala b/external/maprdb/src/test/scala/com/mapr/db/testCases/SparkSqlPushDownTests.scala new file mode 100644 index 0000000000000..a9d8531f165ea --- /dev/null +++ b/external/maprdb/src/test/scala/com/mapr/db/testCases/SparkSqlPushDownTests.scala @@ -0,0 +1,429 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.testCases + +import com.mapr.db.spark.field +import org.apache.spark.SparkConf +import org.apache.spark.sql.{AnalysisException, SparkSession} +import com.mapr.db.spark._ +import com.mapr.db.spark.sql._ +import org.apache.spark.sql.functions._ + + +object SparkSqlPushDownTests { + lazy val conf = new SparkConf() + .setAppName("SparkSqlFilterTests") + .set ("spark.executor.memory", "1g") + .set ("spark.driver.memory", "1g") + + lazy val spark = SparkSession.builder ().appName ("SparkSqlFilterTests").config (conf).getOrCreate () + + val tableName = "/tmp/SparkSqlOjaiConnectorFilterTesting" + + def main (args: Array[String] ): Unit = { + MapRDBSparkTests.tableInitialization (spark.sparkContext,tableName) + SparkSqlPushDownTests.runTests(spark) + } + + def testFilterPushDownOnIdColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter($"_id" === "rsmith").queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("EqualTo(_id,rsmith)")) { + println("testFilterPushDownOnIdColumn succeeded") + return true + } + else { + println("testFilterPushDownOnIdColumn failed") + return false + } + } + + + def testFilterPushDownOnNonIDColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter($"first_name" === "Andrew").queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("EqualTo(first_name,Andrew)")) { + println("testFilterPushDownOnNonIDColumn succeeded") + return true + } + else { + println("testFilterPushDownOnNonIDColumn failed") + return false + } + } + + def testFilterPushDownOnMapColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter($"address.city" === "San Jose").queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (!(pushDownHappened && subStr.contains("EqualTo(address.city,San Jose)"))) { + if (pushDownHappened == false) println("Filter is not pushed down") + println("testFilterPushDownOnMapColumn succeeded") + return true + } + else { + println("testFilterPushDownOnMapColumn failed") + return false + } + } + + def testFilterPushDownOnArrayColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter($"interests"(0) === "San Jose").queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (!(pushDownHappened && subStr.contains("EqualTo(interests(0),San Jose)"))) { + if (pushDownHappened == false) println("Filter is not pushed down") + println("testFilterPushDownOnArrayColumn succeeded") + return true + } + else { + println("testFilterPushDownOnArrayColumn failed") + return false + } + } + + def testGTFilterPushDownOnNonIDColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter($"first_name" > "Andrew").queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("GreaterThan(first_name,Andrew)")) { + println("testGTFilterPushDownOnNonIDColumn succeeded") + return true + } + else { + println(subStr) + println("testGTFilterPushDownOnNonIDColumn failed") + return false + } + } + + def testLTFilterPushDownOnNonIDColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter($"first_name" < "Andrew").queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("LessThan(first_name,Andrew)")) { + println("testLTFilterPushDownOnNonIDColumn succeeded") + return true + } + else { + println(subStr) + println("testLTFilterPushDownOnNonIDColumn failed") + return false + } + } + + def testLTEFilterPushDownOnNonIDColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter($"first_name" <= "Andrew").queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("LessThanOrEqual(first_name,Andrew)")) { + println("testLTEFilterPushDownOnNonIDColumn succeeded") + return true + } + else { + println(subStr) + println("testLTEFilterPushDownOnNonIDColumn failed") + return false + } + } + + def testGTEFilterPushDownOnNonIDColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter($"first_name" >= "Andrew").queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("GreaterThanOrEqual(first_name,Andrew)")) { + println("testGTEFilterPushDownOnNonIDColumn succeeded") + return true + } + else { + println(subStr) + println("testGTEFilterPushDownOnNonIDColumn failed") + return false + } + } + + def testComplexOrFilterPushDownOnNonIDColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter($"first_name" >= "Andrew" or $"first_name" <= "Andrew").queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("Or(GreaterThanOrEqual(first_name,Andrew),LessThanOrEqual(first_name,Andrew))")) { + println("testComplexOrFilterPushDownOnNonIDColumn succeeded") + return true + } + else { + println(subStr) + println("testComplexOrFilterPushDownOnNonIDColumn failed") + return false + } + } + + def testComplexAndFilterPushDownOnNonIDColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter($"first_name" >= "Andrew" and $"last_name" === "Lehmann").queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("GreaterThanOrEqual(first_name,Andrew)") && subStr.contains("EqualTo")) { + println("testComplexAndFilterPushDownOnNonIDColumn succeeded") + return true + } + else { + println(subStr) + println(subStr.substring(70), subStr.length) + println("testComplexAndFilterPushDownOnNonIDColumn failed") + return false + } + } + + def testStartsWithPushDownOnNonIDColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter($"first_name".startsWith("And")).queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("StringStartsWith(first_name,And)")) { + println("testStartsWithPushDownOnNonIDColumn succeeded") + return true + } + else { + println(subStr) + //println(subStr.substring(70), subStr.length) + println("testStartsWithPushDownOnNonIDColumn failed") + return false + } + } + + def testEndsWithPushDownOnNonIDColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter($"first_name".endsWith("rew")).queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("StringEndsWith(first_name,rew)")) { + println("testEndsWithPushDownOnNonIDColumn succeeded") + return true + } + else { + println(subStr) + //println(subStr.substring(70), subStr.length) + println("testEndsWithPushDownOnNonIDColumn failed") + return false + } + } + + def testContainsPushDownOnNonIDColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter($"first_name".contains("dre")).queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("StringContains(first_name,dre)")) { + println("testContainsPushDownOnNonIDColumn succeeded") + return true + } + else { + println(subStr) + //println(subStr.substring(70), subStr.length) + println("testContainsPushDownOnNonIDColumn failed") + return false + } + } + + def testINPushDownOnNonIDColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter($"first_name".isin("Andrew")).queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("In(first_name, [Andrew]")) { + println("testINPushDownOnNonIDColumn succeeded") + return true + } + else { + println(subStr) + //println(subStr.substring(70), subStr.length) + println("testINPushDownOnNonIDColumn failed") + return false + } + } + + def testComplexNotOrFilterPushDownOnNonIDColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter(not($"first_name" >= "Andrew" or $"first_name" <= "Andrew")).queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("LessThan(first_name,Andrew)") && subStr.contains("GreaterThan(first_name,Andrew)")) { + println("testComplexNotOrFilterPushDownOnNonIDColumn succeeded") + return true + } + else { + println(subStr) + println("testComplexNotOrFilterPushDownOnNonIDColumn failed") + return false + } + } + + def testComplexNotAndFilterPushDownOnNonIDColumn(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).filter(not($"first_name" >= "Andrew" and $"last_name" === "Lehmann")).queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("Or(LessThan(first_name,Andrew),Not(EqualTo(last_name,Lehmann)))")) { + println("testComplexNotAndFilterPushDownOnNonIDColumn succeeded") + return true + } + else { + println(subStr) + println(subStr.substring(70), subStr.length) + println("testComplexNotAndFilterPushDownOnNonIDColumn failed") + return false + } + } + + def testProjectionPushDown(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).select("first_name","last_name","_id").queryExecution.sparkPlan.toString() + val subStr = getPushedColumns(executionPlan) + if (subStr.contains("first_name") && subStr.contains("last_name") && subStr.contains("_id") && subStr.split(",").length == 3 ) { + println("testProjectionPushDown succeeded") + return true + } + else { + println(subStr) + subStr.split(",").foreach(println) + println("testProjectionPushDown failed") + return false + } + } + + def testProjectionPushDownNestedFields(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.loadFromMapRDB(tableName).select("first_name","last_name","_id","address").queryExecution.executedPlan.toString() + val subStr = getPushedColumns(executionPlan) + if (subStr.contains("first_name") && subStr.contains("last_name") && subStr.contains("_id") && subStr.contains("address") && !subStr.contains("city") && subStr.split(",").length == 4 ) { + println("testProjectionPushDownNestedFields succeeded") + return true + } + else { + println(subStr) + subStr.split(",").foreach(println) + println("testProjectionPushDownNestedFields failed") + return false + } + } + + def testProjectionPDOnDFWithRDDSelection(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.sparkContext.loadFromMapRDB(tableName).select("first_name","last_name","_id","address").toDF.select("first_name","last_name").queryExecution.executedPlan.toString() + val subStr = getPushedColumns(executionPlan) + if (subStr.contains("first_name") && subStr.contains("last_name") && !subStr.contains("_id") && !subStr.contains("address") && !subStr.contains("city") && subStr.split(",").length == 2 ) { + println("testProjectionOnDFWithRDDSelection succeeded") + return true + } + else { + println(subStr) + subStr.split(",").foreach(println) + println("testProjectionOnDFWithRDDSelection failed") + return false + } + } + + def testProjectionPDOnDFWithRDDSelectionErrorCondition(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + var executionPlan: String = null + try { + executionPlan = spark.sparkContext.loadFromMapRDB(tableName).select("first_name", "last_name", "_id", "address").toDF.select("first_name", "last_name", "dob").queryExecution.executedPlan.toString() + } catch { + case ex: AnalysisException => { + println("testProjectionPDOnDFWithRDDSelectionErrorCondition succeeded") + return true + } + } + val subStr = getPushedColumns(executionPlan) + if (subStr.contains("first_name") && subStr.contains("last_name") && !subStr.contains("_id") && !subStr.contains("address") && !subStr.contains("city") && subStr.split(",").length == 2 ) { + println("testProjectionPDOnDFWithRDDSelectionErrorCondition succeeded") + return true + } + else { + println(subStr) + subStr.split(",").foreach(println) + println("testProjectionPDOnDFWithRDDSelectionErrorCondition failed") + return false + } + } + + def testFilterPDOnDFWithRDDFilter(spark: SparkSession, tableName: String): Boolean = { + import spark.implicits._ + val executionPlan = spark.sparkContext.loadFromMapRDB(tableName).where(field("first_name") <= "Andrew").toDF.filter($"first_name" >= "Andrew").queryExecution.sparkPlan.toString() + val pushDownHappened = executionPlan.contains("PushedFilters:") + val subStr = if (pushDownHappened) getPrunedFilters(executionPlan) else "" + if (pushDownHappened && subStr.contains("GreaterThanOrEqual(first_name,Andrew)")) { + println("testFilterPDOnDFWithRDDFilter succeeded") + return true + } + else { + println(subStr) + println("testFilterPDOnDFWithRDDFilter failed") + return false + } + } + + def runTests(sparkSession: SparkSession): Unit = { + testFilterPushDownOnIdColumn(spark, tableName) + testFilterPushDownOnNonIDColumn(spark, tableName) + testFilterPushDownOnMapColumn(spark, tableName) + testFilterPushDownOnArrayColumn(spark, tableName) + testLTFilterPushDownOnNonIDColumn(spark, tableName) + testLTEFilterPushDownOnNonIDColumn(spark, tableName) + testGTFilterPushDownOnNonIDColumn(spark, tableName) + testGTEFilterPushDownOnNonIDColumn(spark, tableName) + testComplexOrFilterPushDownOnNonIDColumn(spark, tableName) + testComplexAndFilterPushDownOnNonIDColumn(spark, tableName) + testStartsWithPushDownOnNonIDColumn(spark, tableName) + testEndsWithPushDownOnNonIDColumn(spark, tableName) + testContainsPushDownOnNonIDColumn(spark, tableName) + testINPushDownOnNonIDColumn(spark, tableName) + testProjectionPushDown(spark, tableName) + testProjectionPushDownNestedFields(spark, tableName) + testProjectionPDOnDFWithRDDSelection(spark, tableName) + testFilterPDOnDFWithRDDFilter(spark, tableName) + testProjectionPDOnDFWithRDDSelectionErrorCondition(spark, tableName) + } + + def getPrunedFilters(executionPlan: String): String = { + val startIndex: Int = executionPlan.indexOf("PushedFilters:") + val endIndex: Int = executionPlan.indexOf("ReadSchema") + return executionPlan.substring(startIndex, endIndex) + } + + def getPushedColumns(executionPlan: String): String = { + val startIndex: Int = executionPlan.indexOf("MapRDBBaseRDD") + val colStartIndex: Int = executionPlan.substring(startIndex).indexOf('[') + startIndex + val colEndIndex: Int = if (executionPlan.substring(startIndex).indexOf(']') != -1) executionPlan.substring(startIndex).indexOf(']') + startIndex + else executionPlan.substring(startIndex).indexOf("ReadSchema") + startIndex + return executionPlan.substring(colStartIndex, colEndIndex) + } +} + +object SparkSqlPushDownTestsWithKryo { + val tableName = "/tmp/SparkSqlOjaiConnectorFilterTesting" + lazy val conf = new SparkConf() + .setAppName("SparkSqlFilterTestsWithKryo") + .set("spark.executor.memory", "1g") + .set("spark.driver.memory", "1g") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "com.mapr.db.spark.OJAIKryoRegistrator") + + lazy val spark = SparkSession.builder().appName("SparkSqlFilterTestsWithKryo").config(conf).getOrCreate() + + + def main(args: Array[String]): Unit = { + MapRDBSparkTests.tableInitialization(spark.sparkContext,tableName) + SparkSqlPushDownTests.runTests(spark) + } +} \ No newline at end of file diff --git a/external/maprdb/src/test/scala/com/mapr/db/testCases/SparkStreamingTests.scala b/external/maprdb/src/test/scala/com/mapr/db/testCases/SparkStreamingTests.scala new file mode 100644 index 0000000000000..ba88131d59690 --- /dev/null +++ b/external/maprdb/src/test/scala/com/mapr/db/testCases/SparkStreamingTests.scala @@ -0,0 +1,83 @@ +/* Copyright (c) 2015 & onwards. MapR Tech, Inc., All rights reserved */ +package com.mapr.db.testCases + +import com.mapr.db.spark._ +import com.mapr.db.spark.dbclient.DBClient +import org.apache.spark.streaming.dstream.ConstantInputDStream +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.streaming.{Seconds, StreamingContext} +import com.mapr.db.spark.streaming._ +import org.apache.spark.sql.SparkSession + +object SparkStreamingTests { + lazy val conf = new SparkConf() + .setAppName("SparkStreamingTests") + .set ("spark.executor.memory", "1g") + .set ("spark.driver.memory", "1g") + + def functionToCreateContext(spark: SparkContext)(): StreamingContext = { + val ssc = new StreamingContext(spark, Seconds(10)) + ssc + } + lazy val ssc = StreamingContext.getOrCreate("/tmp/", + functionToCreateContext(SparkSession.builder().appName("SparkStreaming").config(conf).getOrCreate().sparkContext)) + + val tableName = "/tmp/SparkSqlOjaiConnectorStreamingTesting" + + def main (args: Array[String] ): Unit = { + MapRDBSparkTests.tableInitialization (ssc.sparkContext,tableName) + SparkStreamingTests.runTests(ssc.sparkContext) + } + + def runTests(sparkSession: SparkContext): Unit = { + testSavingDStreamToMapRDBTable(ssc, tableName) + } + + def testSavingDStreamToMapRDBTable(spark: StreamingContext, tableName: String): Boolean = { + if (DBClient().tableExists(tableName+"output")) DBClient().deleteTable(tableName+"output") + val rdd = spark.sparkContext.loadFromMapRDB(tableName) + val dstream = new ConstantInputDStream(spark, rdd) + dstream.saveToMapRDB(tableName+"output", createTable = true) + + spark.start() + Thread.sleep(10000) + val collection = ssc.sparkContext.loadFromMapRDB(tableName+"output").collect + if (collection.map(a => a.asJsonString()).toSet.sameElements( + Set("{\"_id\":\"rsmith\",\"address\":{\"city\":\"San Francisco\",\"line\":\"100 Main Street\",\"zip\":94105},\"dob\":\"1982-02-03\",\"first_name\":\"Robert\"," + + "\"interests\":[\"electronics\",\"music\",\"sports\"],\"last_name\":\"Smith\"}", + "{\"_id\":\"mdupont\",\"address\":{\"city\":\"San Jose\",\"line\":\"1223 Broadway\",\"zip\":95109},\"dob\":\"1982-02-03\",\"first_name\":\"Maxime\",\"interests\":[\"sports\",\"movies\"," + + "\"electronics\"],\"last_name\":\"Dupont\"}", + "{\"_id\":\"jdoe\",\"dob\":\"1970-06-23\",\"first_name\":\"John\",\"last_name\":\"Doe\"}", + "{\"_id\":\"dsimon\",\"dob\":\"1980-10-13\",\"first_name\":\"David\",\"last_name\":\"Simon\"}", + "{\"_id\":\"alehmann\",\"dob\":\"1980-10-13\",\"first_name\":\"Andrew\",\"interests\":[\"html\",\"css\",\"js\"],\"last_name\":\"Lehmann\"}"))) { + println("testSavingDStreamToMapRDBTable succeeded") + return true + } else { + println("testSavingDStreamToMapRDBTable failed") + collection.foreach(doc => println(doc.asJsonString())) + return false + } + } +} + +object SparkStreamingTestsWithKryo { + val tableName = "/tmp/SparkSqlOjaiConnectorStreamingTestingKryo" + lazy val conf = new SparkConf() + .setAppName("SparkStreamingTestsWithKryo") + .set("spark.executor.memory", "1g") + .set("spark.driver.memory", "1g") + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "com.mapr.db.spark.OJAIKryoRegistrator") + + def functionToCreateContext(spark: SparkContext)(): StreamingContext = { + val ssc = new StreamingContext(spark, Seconds(10)) + ssc + } + lazy val ssc = StreamingContext.getOrCreate("/tmp/", + functionToCreateContext(SparkSession.builder().appName("SparkStreaming").config(conf).getOrCreate().sparkContext)) + + def main (args: Array[String] ): Unit = { + MapRDBSparkTests.tableInitialization (ssc.sparkContext,tableName) + SparkStreamingTests.runTests(ssc.sparkContext) + } +} diff --git a/external/maprdb/src/test/scala/com/mapr/db/testCases/WordCount.scala b/external/maprdb/src/test/scala/com/mapr/db/testCases/WordCount.scala new file mode 100644 index 0000000000000..c28ddd6ebd6c1 --- /dev/null +++ b/external/maprdb/src/test/scala/com/mapr/db/testCases/WordCount.scala @@ -0,0 +1,64 @@ +package com.mapr.db.testCases + +import org.apache.spark.SparkContext +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.{Duration, StreamingContext, Time} + +case class WordCount(word: String, count: Int) + +object WordCount { + + type WordHandler = (RDD[WordCount], Time) => Unit + + def count(sc: SparkContext, lines: RDD[String]): RDD[WordCount] = count(sc, lines, Set()) + + def count(sc: SparkContext, lines: RDD[String], stopWords: Set[String]): RDD[WordCount] = { + val stopWordsVar = sc.broadcast(stopWords) + + val words = prepareWords(lines, stopWordsVar) + + val coll = words.collect() + val wordCounts = words.map(word => (word, 1)).reduceByKey(_ + _).map { + case (word: String, count: Int) => WordCount(word, count) + } + + val sortedWordCounts = wordCounts.sortBy(_.word) + + sortedWordCounts + } + + def count(ssc: StreamingContext, + lines: DStream[String], + windowDuration: Duration, + slideDuration: Duration) + (handler: WordHandler): Unit = count(ssc, lines, windowDuration, slideDuration, Set())(handler) + + def count(ssc: StreamingContext, + lines: DStream[String], + windowDuration: Duration, + slideDuration: Duration, + stopWords: Set[String]) + (handler: WordHandler): Unit = { + + val sc = ssc.sparkContext + val stopWordsVar = sc.broadcast(stopWords) + + val words = lines.transform(prepareWords(_, stopWordsVar)) + + val wordCounts = words.map(x => (x, 1)).reduceByKeyAndWindow(_ + _, _ - _, windowDuration, slideDuration).map { + case (word: String, count: Int) => WordCount(word, count) + } + + wordCounts.foreachRDD((rdd: RDD[WordCount], time: Time) => { + handler(rdd.sortBy(_.word), time) + }) + } + + private def prepareWords(lines: RDD[String], stopWords: Broadcast[Set[String]]): RDD[String] = { + val out = lines.collect() + lines.flatMap(_.split("\\s")).flatMap(_.split(" ")).map(_.stripSuffix(".")).map(_.toLowerCase) + .filter(a => !stopWords.value.contains(a)).filter(!_.isEmpty) + } +} diff --git a/external/maprdb/src/test/scala/com/mapr/db/tests/AllTests.scala b/external/maprdb/src/test/scala/com/mapr/db/tests/AllTests.scala new file mode 100644 index 0000000000000..f861167f36867 --- /dev/null +++ b/external/maprdb/src/test/scala/com/mapr/db/tests/AllTests.scala @@ -0,0 +1,1123 @@ +package com.mapr.db.tests + + +import org.scalatest._ +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import com.mapr.db.testCases._ +import scala.language.implicitConversions +import org.junit.experimental.categories.{Categories, Category} + + +@RunWith(classOf[JUnitRunner]) +class AllTests extends FlatSpec with InitClusterDisableKryoSerialization with GivenWhenThen with Matchers { + + "Empty set" should "be counted" in { + assert (com.mapr.db.testCases.MapRDBSparkTests.testEmptySet(sc,tableName)) + } + + "Shakespeare most famous quote" should "be counted" in { + assert( com.mapr.db.testCases.MapRDBSparkTests.testShakespearesFamousQuote(sc, tableName)) + } + + "bean class parsing" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testBeanClassParsing(sc, tableName)) + } + + "integer collection of zip codes" should "be tested " in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testIntegerZipCodes(sc, tableName)) + } + + "maprdb bean class" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingMapRDBBeanClass(sc, tableName)) + } + + "maprdb bean collect class" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testMaprDBBeanClassWithCollect(sc, tableName)) + } + + "groupby key on ODate " should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testGroupByOnODate(sc, tableName)) + } + + "loading of ojai documents into rdd" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingLoadingOfOJAIDocuments(sc, tableName)) + } + + "cogroup test on ojai document rdd" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingCoGroupWithOJAIDOCUMENT(sc, tableName)) + } + + "cogroup on array value test on ojai document rdd" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingArrayValueWithOJAIDocument(sc, tableName)) + } + + "cogroup on array key test on ojai document rdd" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingCoGroupArrayKey(sc, tableName)) + } + + "cogroup on map value test on ojai document rdd" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingCoGroupOnMapValue(sc, tableName)) + } + + "cogroup on map key test on ojai document rdd" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingCoGroupOnMapKey(sc, tableName)) + } + + "cogroup on map key test on ojai document rdd with filter" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingCoGroupWithMapKeyWithFilter(sc, tableName)) + } + + "assigning a zipcode to ojai document" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingAssignmentOfZipCodeToOJAIDocument(sc, tableName)) + } + + "accessing fields of ojai document without passing parametric types" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingAccessOfFieldsOfOJAIDocumentWithParametricTypes(sc, tableName)) + } + + "accessing fields by passing parametric type" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingAccessOfFieldsOfOJAIDocumentWithParametricTypes2(sc, tableName)) + } + + "accessing only projected fields" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingAccessOfProjectedFields(sc, tableName)) + } + + "accessing only projected field paths" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingAccessOfProjectedFieldPaths(sc, tableName)) + } + + "saving the processed OJAIDocuments to a maprdb table" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingOfSavingTheProcessedOJAIDocuments(sc, tableName)) + } + + "map as a value" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingMapAsaValue(sc, tableName)) + } + + "map as a key" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingMapAsaKey(sc, tableName)) + } + + "array as value" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingArrayAsaValue(sc, tableName)) + } + + "array as key" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingArrayAsaKey(sc, tableName)) + } + + "ojai document parsing functionality" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingOJAIDocumentParsingFunctionality(sc, tableName)) + } + + "multiple data types in ojai document parsing functionality" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingMultipleDataTypesInOJAIDocument(sc, tableName)) + } + +// "multiple data types in ojai document parsing functionality and type casting" should "be tested" in { +// assert(com.mapr.db.testCases.MapRDBSparkTests.testingMultipleDataTypesInOJAIDocumentAndTypeCasting(sc, tableName)) +// } + + "single data types (double) in ojai document parsing functionality and type casting" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingSingleDataTypeDoubleInOJAIAndTypeCasting(sc, tableName)) + } + + "tuple output of an RDD" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingTupleOutputOfAnRDD(sc, tableName)) + } + + "adding country into the address field of ojaidocument" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingAddingCountryInAddressField(sc, tableName)) + } + + "flat with map object" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testCaseWithFlatMap(sc, tableName)) + } + + "testing binary type in ojaidocument" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingBinaryDataTypeInOJAIDocument(sc, tableName)) + } + + "testing map type in ojaidocument" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingMapTypeInOJAIDocument(sc, tableName)) + } + + "testing date type in ojaidocument" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingDateTypeInOJAIDocument(sc, tableName)) + } + + "testing save with any object" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingSaveWithAnyObject(sc, tableName)) + } + + "testing filter funciton on Map object" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingFilterFunctionOnMapObject(sc, tableName)) + } + + "testing filter funciton on array object" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingFilterFunctionOnArrayObject(sc, tableName)) + } + + "testing filter function on array object functional way" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingFilterFunctionOnArrayObjectFunctionalway(sc, tableName)) + } + + "test where class on loadMaPRDBTable" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingWhereClauseOnloadFromMapRDB(sc, tableName)) + } + + "partiton on MapRDBTable" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingPartitionOnloadFromMapRDB(sc, tableName)) + } + + "assignment of document as a value" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingAssignmentOfDocument(sc, tableName)) + } + + "testingJoinWithRDD" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingJoinWithRDD(sc, tableName)) + } + + "testingJoinWithRDDBean" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingJoinWithRDDBean(sc, tableName)) + } + + "testingBulkJoinWithRDD" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingBulkJoinWithRDD(sc, tableName)) + } + + "testingJoinWithOjaiRDDBean" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingJoinWithOjaiRDDBean(sc, tableName)) + } + + "testingUpdateMapRDBTable" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingUpdateMapRDBTable(sc, tableName)) + } + + "testingUpdateMapRDBTablePairedRDD" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingUpdateMapRDBTablePairedRDD(sc, tableName)) + } + + "testingCheckAndUpdateMapRDBTable" should "be tested" in { + assert(MapRDBSparkTests.testingCheckAndUpdateMapRDBTable(sc, tableName)) + } + + "getter functionality for int" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForInt(sc)) + } + + "getter functionality for byte" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForByte(sc)) + } + + "getter functionality for string" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForString(sc)) + } + + "getter functionality for short" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForShort(sc)) + } + + "getter functionality for long" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForLong(sc)) + } + + "getter functionality for float" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForFloat(sc)) + } + + "getter functionality for double" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForDouble(sc)) + } + + "getter functionality for time" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForTime(sc)) + } + + "getter functionality for date" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForDate(sc)) + } + + "getter functionality for timestamp" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForTimeStamp(sc)) + } + + "getter functionality for binary" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForBinary(sc)) + } + + "getter functionality for list" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForList(sc)) + } + + "getter functionality for map" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForMap(sc)) + } + + "getter functionality for int explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForIntExpl(sc)) + } + + "getter functionality for byte explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForByteExpl(sc)) + } + + "getter functionality for string explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForStringExpl(sc)) + } + + "getter functionality for short explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForShortExpl(sc)) + } + + "getter functionality for long explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForLongExpl(sc)) + } + + "getter functionality for float explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForFloatExpl(sc)) + } + + "getter functionality for double explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForDoubleExpl(sc)) + } + + "getter functionality for time explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForTimeExpl(sc)) + } + + "getter functionality for date explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForDateExpl(sc)) + } + + "getter functionality for timestamp explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForTimeStampExpl(sc)) + } + + "getter functionality for binary explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForBinaryExpl(sc)) + } + + "getter functionality for array explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForArrayExpl(sc)) + } + + "getter functionality for map explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForMapExpl(sc)) + } + + "setter functionality for int" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForInt(sc)) + } + + "setter functionality for byte" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForByte(sc)) + } + + "setter functionality for string" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForString(sc)) + } + + "setter functionality for short" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForShort(sc)) + } + + "setter functionality for long" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForLong(sc)) + } + + "setter functionality for float" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForFloat(sc)) + } + + "setter functionality for double" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForDouble(sc)) + } + + "setter functionality for time" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForTime(sc)) + } + + "setter functionality for date" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForDate(sc)) + } + + "setter functionality for timestamp" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForTimeStamp(sc)) + } + + "setter functionality for binary" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForBinary(sc)) + } + + "setter functionality for binary with byteArray" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForBinaryWithByteArr(sc)) + } + + "setter functionality for list " should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForList(sc)) + } + + "setter functionality for map" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForMap(sc)) + } + + "getter no data" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterNoDataCase(sc)) + } + + "setter functionality for null data" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterNullToDoc(sc)) + } + + "setter functionality for map string int" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForMapStringInt(sc)) + } + + "Non dynamic setter functionality for int" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testNonDynamicSetterFuncForInt(sc)) + } + + "loading with just table name" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingSimpleLoadTable(sc,tableName)) + } + + "loading with table name and specific columns" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingLoadTableWithSpecificColumns(sc, tableName)) + } + + "loading with table name and where condition" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingLoadTableWithWhereEQCondition(sc, tableName)) + } + + "loading with table name, where condition and select clause" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingLoadTableWithWhereEQAndSelectClause(sc, tableName)) + } + + "testingLoadTableWithWhereEQConditionAndSave" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingLoadTableWithWhereEQConditionAndSave(sc, tableName)) + } + + "saving a table" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingSimpleSaveTable(sc, tableName, tableName2+"new")) + } + + "saving a table with different ID" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingIDwithSaveToMapRDB(sc, tableName, tableName2+"new")) + } + + "saving a table using bulk save mode" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingBulkSaveMode(sc, tableName, tableName2+"new1")) + } + + "saving a table using bulk save mode and bulkmode set to false for table" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingBulkSaveWithoutBulkModeSetInTable(sc, tableName, tableName2+"new1")) + } + + "saving a table using bulk save mode with bean class" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingBulkSaveModeBeanClass(sc, tableName, tableName2+"new2")) + } + + "split partitioner with string types" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingSplitPartitioner(sc, tableName, tableName2+"new2")) + } + + "split partitioner with binary types" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingSplitPartitionerWithBinaryData(sc, tableName, tableName2+"new")) + } + +// "split partitioner with bytebuffer types" should "be tested" in { +// assert(com.mapr.db.testCases.LoadAndSaveTests.testingSplitPartitionerWithByteBufferData(sc, tableName, tableName2 + "new")) +// } + + //Following are the condition testing + "simple _id only condition" should "be tested" in { + assert(PredicateTests.testingIdOnlyCondition(sc, PredicateTests.tableName)); + } + + "testingSimpleGTCondition" should "be tested" in { + assert(PredicateTests.testingSimpleGTCondition(sc, PredicateTests.tableName)) + } + "testingNotExistsCondition" should "be tested" in { + assert(PredicateTests.testingNotExistsCondition(sc, PredicateTests.tableName)) + } + + "testingSimpleINCondition" should "be tested" in { + assert(PredicateTests.testingSimpleINCondition(sc, PredicateTests.tableName)) + } + + "testingOTimeINCondition" should "be tested" in { + assert(PredicateTests.testingOTimeINCondition(sc, PredicateTests.tableName)) + } + + "testingSimpleOTimeCondition" should "be tested" in { + assert(PredicateTests.testingSimpleOTime(sc, PredicateTests.tableName)) + } + + "testingTYPEOFCondition" should "be tested" in { + assert(PredicateTests.testingTYPEOFCondition(sc, PredicateTests.tableName)) + } + + "testingComplexAND_INcondition" should "be tested" in { + assert(PredicateTests.testingComplexAND_INcondition(sc, PredicateTests.tableName)) + } + + "testingCompositeCondition" should "be tested" in { + assert(PredicateTests.testingCompositeCondition(sc, PredicateTests.tableName)) + } + + "testingThreeConditions" should "be tested" in { + assert(PredicateTests.testingThreeConditions(sc, PredicateTests.tableName)) + } + + "testingORCondition" should "be tested" in { + assert(PredicateTests.testingORCondition(sc, PredicateTests.tableName)) + } + + "testingComplexConditonWithDate" should "be tested" in { + assert(PredicateTests.testingComplexConditonWithDate(sc, PredicateTests.tableName)) + } + + "testingBetweenCondition" should "be tested" in { + assert(PredicateTests.testingBetweenCondition(sc, PredicateTests.tableName)) + } + + "testingEqualityConditionOnSeq" should "be tested" in { + assert(PredicateTests.testingEqualityConditionOnSeq(sc, PredicateTests.tableName)) + } + + "testingEqualtiyOnMapOfStrings" should "be tested" in { + assert(PredicateTests.testingEqualtiyOnMapOfStrings(sc, PredicateTests.tableName)) + } + + "testingEqualityOnMapStringInteger" should "be tested" in { + assert(PredicateTests.testingEqualityOnMapStringInteger(sc, PredicateTests.tableName)) + } + + "testingLikeCondition" should "be tested" in { + assert(PredicateTests.testingLikeCondition(sc, PredicateTests.tableName)) + } + + "testingMatchesCondition" should "be tested" in { + assert(PredicateTests.testingMatchesCondition(sc, PredicateTests.tableName)) + } + + "testing not equality on id" should "be tested" in { + assert(PredicateTests.testingNotEqualityOnID(sc, PredicateTests.tableName)) + } + + "testingNotEqualityOnMapStringInteger" should "be tested" in { + assert(PredicateTests.testingNotEqualityOnMapStringInteger(sc, PredicateTests.tableName)) + } + + "testingNotEqualityConditionOnSeq" should "be tested" in { + assert(PredicateTests.testingNotEqualityConditionOnSeq(sc, PredicateTests.tableName)) + } + + "testingSizeOf" should "be tested" in { + assert(PredicateTests.testingSizeOf(sc, PredicateTests.tableName)) + } + + "testingSizeOfWithComplexCondition" should "be tested" in { + assert(PredicateTests.testingSizeOfWithComplexCondition(sc, PredicateTests.tableName)) + } + + "testingTypeOfWithNonExistantType" should "be tested" in { + assert(PredicateTests.testingTypeOfWithNonExistantType(sc, PredicateTests.tableName)) + } + + "testingWithQueryCondition" should "be tested" in { + assert(PredicateTests.testingWithQueryCondition(sc, PredicateTests.tableName)) + } + +// "testWithListINCondition" should "be tested" in { +// assert(PredicateTests.testWithListINCondition(sc, PredicateTests.tableName)) +// } + + "testingSizeOfNotEquals" should "be tested" in { + assert(PredicateTests.testingSizeOfNotEquals(sc, PredicateTests.tableName)) + } + + "testingINConditionOnSeqwithInSeq" should "be tested" in { + assert(PredicateTests.testingINConditionOnSeqwithInSeq(sc, PredicateTests.tableName)) + } + + //The following tests are for sparkSql functionality + "testingBooleanVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingByteVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingShortVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingIntVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingLongVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingFloatVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingDoubleVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingDoubleVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingDateVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingDateVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingTimeVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimestampVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingTimeStampVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingBinaryVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingBinaryVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingMapVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingMapVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingArrayVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingArrayVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsString" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsString(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsByte" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsByte(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsShort" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsShort(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsInt" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsInt(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsLong" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsLong(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsFloat" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsFloat(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsDouble" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsDouble(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsDate" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsTime" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsTimestamp" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsBinary" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsMap" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsArray" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsByte" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsByte(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsShort" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsShort(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsInt" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsInt(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsLong" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsLong(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsFloat" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsFloat(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsDouble" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsDouble(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsDate" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsTime" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsTimeStamp" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsBinary" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsMap" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsArray" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsShort" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsShort(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsInt" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsInt(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsLong" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsLong(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsFloat" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsFloat(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsDouble" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsDouble(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsDate" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsTime" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsInt" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsInt(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsLong" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsLong(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsFloat" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsFloat(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsDouble" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsDouble(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsDate" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsTime" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsLong" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsLong(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsFloat" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsFloat(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsDouble" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsDouble(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsDate" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsTime" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsFloat" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsFloat(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsDouble" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsDouble(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsDate" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsTime" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatVsDouble" should "be tested" in{ + assert(SparkSqlAccessTests.testingFloatVsDouble(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatVsDate" should "be tested" in{ + assert(SparkSqlAccessTests.testingFloatVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatVsTime" should "be tested" in{ + assert(SparkSqlAccessTests.testingFloatVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingFloatVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingFloatVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingFloatVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingFloatVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingDoubleVsDate" should "be tested" in{ + assert(SparkSqlAccessTests.testingDoubleVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingDoubleVsTime" should "be tested" in{ + assert(SparkSqlAccessTests.testingDoubleVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingDoubleVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingDoubleVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingDoubleVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingDoubleVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingDoubleVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingDoubleVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingDoubleArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingDoubleVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingDateVsTime" should "be tested" in{ + assert(SparkSqlAccessTests.testingDateVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingDateVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingDateVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingDateVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingDateVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingDateVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingDateVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingDateArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingDateVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingTimeVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingTimeVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingTimeVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingTimeVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeStampVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingTimeStampVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeStampVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingTimeStampVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeStampVsArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingTimeStampVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingBinaryVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingBinaryVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingBinaryVsArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingBinaryVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingArrayVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingArrayVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testFilterPushDownOnIdColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testFilterPushDownOnIdColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testFilterPushDownOnMapColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testFilterPushDownOnMapColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testFilterPushDownOnArrayColumn" should " be tested" in { + assert(SparkSqlPushDownTests.testFilterPushDownOnArrayColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testLTFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testLTFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testLTEFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testLTEFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testGTFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testGTFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testGTEFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testGTEFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testComplexOrFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testComplexOrFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testComplexAndFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testComplexAndFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testStartsWithPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testStartsWithPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testEndsWithPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testEndsWithPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testContainsPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testContainsPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testINPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testINPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testComplexNotOrFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testComplexNotOrFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testComplexNotAndFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testComplexNotAndFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testProjectionPushDown" should "be tested" in { + assert(SparkSqlPushDownTests.testProjectionPushDown(spark, SparkSqlPushDownTests.tableName)) + } + + "testProjectionPushDownNextFields" should "be tested" in { + assert(SparkSqlPushDownTests.testProjectionPushDownNestedFields(spark, SparkSqlPushDownTests.tableName)) + } + + "testFilterPDOnDFWithRDDFilter" should "be tested" in { + assert(SparkSqlPushDownTests.testFilterPDOnDFWithRDDFilter(spark, SparkSqlPushDownTests.tableName)) + } + + "testProjectionPDOnDFWithRDDSelection" should "be tested" in { + assert(SparkSqlPushDownTests.testProjectionPDOnDFWithRDDSelection(spark, SparkSqlPushDownTests.tableName)) + } + + "testProjectionPDOnDFWithRDDSelectionErrorCondition" should "be tested" in { + assert(SparkSqlPushDownTests.testProjectionPDOnDFWithRDDSelectionErrorCondition(spark, SparkSqlPushDownTests.tableName)) + } + + "testingLoadExplicitSchema" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadExplicitSchema(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingLoadBeanClass" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadBeanClass(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingLoadInferSchema" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadInferSchema(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingLoadFromDFReader" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadFromDFReader(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingLoadFromDFReaderLoad" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadFromDFReaderLoad(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingLoadFromDFWriterWithOperationOption" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadFromDFWriterWithOperationOption(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingLoadFromDFReaderWithOperationOption" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadFromDFReaderWithOperationOption(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "tesitngLoadFromDFReaderWithSampleOption" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.tesitngLoadFromDFReaderWithSampleOption(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingLoadFromDFReaderWithFailOnConflict" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadFromDFReaderWithFailOnConflict(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingSaveFromSparkSession" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingSaveFromSparkSession(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingSaveFromDFWSession" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingSaveFromDFWSession(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingSaveWithBulkLoad" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingSaveWithBulkLoad(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingSaveWithComplexDocument" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingSaveWithComplexDocument(spark)) + } + + "testingUpdateToMapRDB" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingUpdateToMapRDB(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingCheckAndUpdateToMapRDB" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingCheckAndUpdateToMapRDB(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingUpdateToMapRDBAddToArray" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingUpdateToMapRDBAddToArray(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testSavingDStreamToMapRDBTable" should "be tested" in { + assert(SparkStreamingTests.testSavingDStreamToMapRDBTable(ssc, SparkStreamingTests.tableName)) + } + + "testingSimpleSaveModeBeanClass" should "be tested" in { + assert(LoadAndSaveTests.testingSimpleSaveModeBeanClass(spark.sparkContext, tableName, LoadAndSaveTests.saveToTable)) + } +} diff --git a/external/maprdb/src/test/scala/com/mapr/db/tests/AllTestsWithKryo.scala b/external/maprdb/src/test/scala/com/mapr/db/tests/AllTestsWithKryo.scala new file mode 100644 index 0000000000000..32f90ef2d03c6 --- /dev/null +++ b/external/maprdb/src/test/scala/com/mapr/db/tests/AllTestsWithKryo.scala @@ -0,0 +1,1088 @@ +package com.mapr.db.tests + +import org.scalatest._ +import org.junit.runner.RunWith +import org.scalatest.junit.JUnitRunner +import com.mapr.db.testCases._ + +import scala.language.implicitConversions +import org.junit.experimental.categories.{Categories, Category} + + +@RunWith(classOf[JUnitRunner]) +class AllTestsWithKryo extends FlatSpec with InitClusterEnableKryoSerialization with GivenWhenThen with Matchers { + + "Empty set" should "be counted" in { + assert (com.mapr.db.testCases.MapRDBSparkTests.testEmptySet(sc,tableName)) + } + + "Shakespeare most famous quote" should "be counted" in { + assert( com.mapr.db.testCases.MapRDBSparkTests.testShakespearesFamousQuote(sc, tableName)) + } + + "bean class parsing" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testBeanClassParsing(sc, tableName)) + } + + "integer collection of zip codes" should "be tested " in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testIntegerZipCodes(sc, tableName)) + } + + "maprdb bean class" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingMapRDBBeanClass(sc, tableName)) + } + + "maprdb bean collect class" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testMaprDBBeanClassWithCollect(sc, tableName)) + } + + "groupby key on ODate " should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testGroupByOnODate(sc, tableName)) + } + + "loading of ojai documents into rdd" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingLoadingOfOJAIDocuments(sc, tableName)) + } + + "cogroup test on ojai document rdd" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingCoGroupWithOJAIDOCUMENT(sc, tableName)) + } + + "cogroup on array value test on ojai document rdd" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingArrayValueWithOJAIDocument(sc, tableName)) + } + + "cogroup on array key test on ojai document rdd" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingCoGroupArrayKey(sc, tableName)) + } + + "cogroup on map value test on ojai document rdd" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingCoGroupOnMapValue(sc, tableName)) + } + + "cogroup on map key test on ojai document rdd" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingCoGroupOnMapKey(sc, tableName)) + } + + "cogroup on map key test on ojai document rdd with filter" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingCoGroupWithMapKeyWithFilter(sc, tableName)) + } + + "assigning a zipcode to ojai document" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingAssignmentOfZipCodeToOJAIDocument(sc, tableName)) + } + + "accessing fields of ojai document without passing parametric types" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingAccessOfFieldsOfOJAIDocumentWithParametricTypes(sc, tableName)) + } + + "accessing fields by passing parametric type" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingAccessOfFieldsOfOJAIDocumentWithParametricTypes2(sc, tableName)) + } + + "accessing only projected fields" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingAccessOfProjectedFields(sc, tableName)) + } + + "accessing only projected field paths" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingAccessOfProjectedFieldPaths(sc, tableName)) + } + + "saving the processed OJAIDocuments to a maprdb table" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingOfSavingTheProcessedOJAIDocuments(sc, tableName)) + } + + "map as a value" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingMapAsaValue(sc, tableName)) + } + + "map as a key" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingMapAsaKey(sc, tableName)) + } + + "array as value" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingArrayAsaValue(sc, tableName)) + } + + "array as key" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingArrayAsaKey(sc, tableName)) + } + + "ojai document parsing functionality" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingOJAIDocumentParsingFunctionality(sc, tableName)) + } + + "multiple data types in ojai document parsing functionality" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingMultipleDataTypesInOJAIDocument(sc, tableName)) + } + +// "multiple data types in ojai document parsing functionality and type casting" should "be tested" in { +// assert(com.mapr.db.testCases.MapRDBSparkTests.testingMultipleDataTypesInOJAIDocumentAndTypeCasting(sc, tableName)) +// } + + "single data types (double) in ojai document parsing functionality and type casting" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingSingleDataTypeDoubleInOJAIAndTypeCasting(sc, tableName)) + } + + "tuple output of an RDD" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingTupleOutputOfAnRDD(sc, tableName)) + } + + "adding country into the address field of ojaidocument" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingAddingCountryInAddressField(sc, tableName)) + } + + "flat with map object" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testCaseWithFlatMap(sc, tableName)) + } + + "testing binary type in ojaidocument" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingBinaryDataTypeInOJAIDocument(sc, tableName)) + } + + "testing map type in ojaidocument" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingMapTypeInOJAIDocument(sc, tableName)) + } + + "testing date type in ojaidocument" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingDateTypeInOJAIDocument(sc, tableName)) + } + + "testing save with any object" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingSaveWithAnyObject(sc, tableName)) + } + + + "testing filter funciton on Map object" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingFilterFunctionOnMapObject(sc, tableName)) + } + + + "testing filter funciton on array object" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingFilterFunctionOnArrayObject(sc, tableName)) + } + + "testing filter function on array object functional way" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingFilterFunctionOnArrayObjectFunctionalway(sc, tableName)) + } + + "test where class on loadMaPRDBTable" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingWhereClauseOnloadFromMapRDB(sc, tableName)) + } + + "partiton on MapRDBTable" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingPartitionOnloadFromMapRDB(sc, tableName)) + } + + "assignment of document as a value" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingAssignmentOfDocument(sc, tableName)) + } + + "testingJoinWithRDD" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingJoinWithRDD(sc, tableName)) + } + + "testingJoinWithRDDBean" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingJoinWithRDDBean(sc, tableName)) + } + + "testingBulkJoinWithRDD" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingBulkJoinWithRDD(sc, tableName)) + } + + "testingJoinWithOjaiRDDBean" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingJoinWithOjaiRDDBean(sc, tableName)) + } + + "testingUpdateMapRDBTable" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingUpdateMapRDBTable(sc, tableName)) + } + + "testingUpdateMapRDBTablePairedRDD" should "be tested" in { + assert(com.mapr.db.testCases.MapRDBSparkTests.testingUpdateMapRDBTablePairedRDD(sc, tableName)) + } + + "testingCheckAndUpdateMapRDBTable" should "be tested" in { + assert(MapRDBSparkTests.testingCheckAndUpdateMapRDBTable(sc, tableName)) + } + + "getter functionality for int" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForInt(sc)) + } + + "getter functionality for byte" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForByte(sc)) + } + + "getter functionality for string" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForString(sc)) + } + "getter functionality for short" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForShort(sc)) + } + "getter functionality for long" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForLong(sc)) + } + "getter functionality for float" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForFloat(sc)) + } + "getter functionality for double" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForDouble(sc)) + } + "getter functionality for time" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForTime(sc)) + } + "getter functionality for date" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForDate(sc)) + } + "getter functionality for timestamp" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForTimeStamp(sc)) + } + "getter functionality for binary" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForBinary(sc)) + } + "getter functionality for list" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForList(sc)) + } + "getter functionality for map" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForMap(sc)) + } + "getter functionality for int explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForIntExpl(sc)) + } + "getter functionality for byte explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForByteExpl(sc)) + } + "getter functionality for string explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForStringExpl(sc)) + } + "getter functionality for short explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForShortExpl(sc)) + } + "getter functionality for long explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForLongExpl(sc)) + } + "getter functionality for float explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForFloatExpl(sc)) + } + "getter functionality for double explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForDoubleExpl(sc)) + } + "getter functionality for time explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForTimeExpl(sc)) + } + "getter functionality for date explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForDateExpl(sc)) + } + "getter functionality for timestamp explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForTimeStampExpl(sc)) + } + "getter functionality for binary explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForBinaryExpl(sc)) + } + "getter functionality for array explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForArrayExpl(sc)) + } + "getter functionality for map explicit" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterFuncForMapExpl(sc)) + } + "setter functionality for int" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForInt(sc)) + } + "setter functionality for byte" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForByte(sc)) + } + "setter functionality for string" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForString(sc)) + } + "setter functionality for short" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForShort(sc)) + } + "setter functionality for long" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForLong(sc)) + } + "setter functionality for float" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForFloat(sc)) + } + "setter functionality for double" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForDouble(sc)) + } + "setter functionality for time" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForTime(sc)) + } + "setter functionality for date" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForDate(sc)) + } + "setter functionality for timestamp" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForTimeStamp(sc)) + } + "setter functionality for binary" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForBinary(sc)) + } + "setter functionality for binary with byteArray" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForBinaryWithByteArr(sc)) + } + "setter functionality for list " should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForList(sc)) + } + "setter functionality for map" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForMap(sc)) + } + "getter no data" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testGetterNoDataCase(sc)) + } + "setter functionality for null data" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterNullToDoc(sc)) + } + "setter functionality for map string int" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testSetterFuncForMapStringInt(sc)) + } + + "Non dynamic setter functionality for int" should "be tested" in { + assert(com.mapr.db.testCases.OjaiDocumentAccessTesting.testNonDynamicSetterFuncForInt(sc)) + } + + "loading with just table name" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingSimpleLoadTable(sc,tableName)) + } + + "loading with table name and specific columns" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingLoadTableWithSpecificColumns(sc, tableName)) + } + + "loading with table name and where condition" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingLoadTableWithWhereEQCondition(sc, tableName)) + } + + "loading with table name, where condition and select clause" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingLoadTableWithWhereEQAndSelectClause(sc, tableName)) + } + + "testingLoadTableWithWhereEQConditionAndSave" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingLoadTableWithWhereEQConditionAndSave(sc, tableName)) + } + + "saving a table" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingSimpleSaveTable(sc, tableName, tableName2)) + } + + "saving a table with different ID" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingIDwithSaveToMapRDB(sc, tableName, tableName2)) + } + + "saving a table using bulk save mode" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingBulkSaveMode(sc, tableName, tableName2)) + } + + "saving a table using bulk save mode and bulkmode set to false for table" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingBulkSaveWithoutBulkModeSetInTable(sc, tableName, tableName2+"new1")) + } + + "saving a table using bulk save mode with bean class" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingBulkSaveModeBeanClass(sc, tableName, tableName2)) + } + + "split partitioner with string types" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingSplitPartitioner(sc, tableName, tableName2+"new2")) + } + + "split partitioner with binary types" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingSplitPartitionerWithBinaryData(sc, tableName, tableName2+"new")) + } + + "split partitioner with bytebuffer types" should "be tested" in { + assert(com.mapr.db.testCases.LoadAndSaveTests.testingSplitPartitionerWithByteBufferData(sc, tableName, tableName2 + "new")) + } + + //Following are the condition testing + "simple _id only condition" should "be tested" in { + assert(PredicateTests.testingIdOnlyCondition(sc, PredicateTests.tableName)); + } + + "testingSimpleGTCondition" should "be tested" in { + assert(PredicateTests.testingSimpleGTCondition(sc, PredicateTests.tableName)) + } + "testingNotExistsCondition" should "be tested" in { + assert(PredicateTests.testingNotExistsCondition(sc, PredicateTests.tableName)) + } + + "testingSimpleINCondition" should "be tested" in { + assert(PredicateTests.testingSimpleINCondition(sc, PredicateTests.tableName)) + } + + "testingOTimeINCondition" should "be tested" in { + assert(PredicateTests.testingOTimeINCondition(sc, PredicateTests.tableName)) + } + + "testingSimpleOTimeCondition" should "be tested" in { + assert(PredicateTests.testingSimpleOTime(sc, PredicateTests.tableName)) + } + + "testingTYPEOFCondition" should "be tested" in { + assert(PredicateTests.testingTYPEOFCondition(sc, PredicateTests.tableName)) + } + + "testingComplexAND_INcondition" should "be tested" in { + assert(PredicateTests.testingComplexAND_INcondition(sc, PredicateTests.tableName)) + } + + "testingCompositeCondition" should "be tested" in { + assert(PredicateTests.testingCompositeCondition(sc, PredicateTests.tableName)) + } + + "testingThreeConditions" should "be tested" in { + assert(PredicateTests.testingThreeConditions(sc, PredicateTests.tableName)) + } + + "testingORCondition" should "be tested" in { + assert(PredicateTests.testingORCondition(sc, PredicateTests.tableName)) + } + + "testingComplexConditonWithDate" should "be tested" in { + assert(PredicateTests.testingComplexConditonWithDate(sc, PredicateTests.tableName)) + } + + "testingBetweenCondition" should "be tested" in { + assert(PredicateTests.testingBetweenCondition(sc, PredicateTests.tableName)) + } + + "testingEqualityConditionOnSeq" should "be tested" in { + assert(PredicateTests.testingEqualityConditionOnSeq(sc, PredicateTests.tableName)) + } + + "testingEqualtiyOnMapOfStrings" should "be tested" in { + assert(PredicateTests.testingEqualtiyOnMapOfStrings(sc, PredicateTests.tableName)) + } + + "testingEqualityOnMapStringInteger" should "be tested" in { + assert(PredicateTests.testingEqualityOnMapStringInteger(sc, PredicateTests.tableName)) + } + + "testingLikeCondition" should "be tested" in { + assert(PredicateTests.testingLikeCondition(sc, PredicateTests.tableName)) + } + + "testingMatchesCondition" should "be tested" in { + assert(PredicateTests.testingMatchesCondition(sc, PredicateTests.tableName)) + } + + "testing not equality on id" should "be tested" in { + assert(PredicateTests.testingNotEqualityOnID(sc, PredicateTests.tableName)) + } + + "testingNotEqualityOnMapStringInteger" should "be tested" in { + assert(PredicateTests.testingNotEqualityOnMapStringInteger(sc, PredicateTests.tableName)) + } + + "testingNotEqualityConditionOnSeq" should "be tested" in { + assert(PredicateTests.testingNotEqualityConditionOnSeq(sc, PredicateTests.tableName)) + } + + "testingSizeOf" should "be tested" in { + assert(PredicateTests.testingSizeOf(sc, PredicateTests.tableName)) + } + + "testingSizeOfWithComplexCondition" should "be tested" in { + assert(PredicateTests.testingSizeOfWithComplexCondition(sc, PredicateTests.tableName)) + } + + "testingTypeOfWithNonExistantType" should "be tested" in { + assert(PredicateTests.testingTypeOfWithNonExistantType(sc, PredicateTests.tableName)) + } + + "testingWithQueryCondition" should "be tested" in { + assert(PredicateTests.testingWithQueryCondition(sc, PredicateTests.tableName)) + } + +// "testWithListINCondition" should "be tested" in { +// assert(PredicateTests.testWithListINCondition(sc, PredicateTests.tableName)) +// } + + "testingSizeOfNotEquals" should "be tested" in { + assert(PredicateTests.testingSizeOfNotEquals(sc, PredicateTests.tableName)) + } + + "testingINConditionOnSeqwithInSeq" should "be tested" in { + assert(PredicateTests.testingINConditionOnSeqwithInSeq(sc, PredicateTests.tableName)) + } + + //The following tests are for sparkSql functionality + "testingBooleanVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingByteVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingShortVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingIntVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingLongVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingFloatVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingDoubleVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingDoubleVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingDateVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingDateVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingTimeVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimestampVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingTimeStampVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingBinaryVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingBinaryVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingMapVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingMapVsNull(spark, SparkSqlAccessTests.tableName)) + } + + "testingArrayVsNull" should "be tested" in { + assert(SparkSqlAccessTests.testingArrayVsNull(spark, SparkSqlAccessTests.tableName)) + } + + + "testingBooleanVsString" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsString(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsByte" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsByte(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsShort" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsShort(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsInt" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsInt(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsLong" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsLong(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsFloat" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsFloat(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsDouble" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsDouble(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsDate" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsTime" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsTimestamp" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsBinary" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsMap" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingBooleanVsArray" should "be tested" in { + assert(SparkSqlAccessTests.testingBooleanVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsByte" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsByte(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsShort" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsShort(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsInt" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsInt(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsLong" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsLong(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsFloat" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsFloat(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsDouble" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsDouble(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsDate" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsTime" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsTimeStamp" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsBinary" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsMap" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingStringVsArray" should "be tested" in { + assert(SparkSqlAccessTests.testingStringVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsShort" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsShort(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsInt" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsInt(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsLong" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsLong(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsFloat" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsFloat(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsDouble" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsDouble(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsDate" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsTime" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingByteVsArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingByteVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsInt" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsInt(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsLong" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsLong(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsFloat" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsFloat(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsDouble" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsDouble(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsDate" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsTime" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingShortArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingShortVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsLong" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsLong(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsFloat" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsFloat(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsDouble" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsDouble(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsDate" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsTime" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingIntArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingIntVsArray(spark, SparkSqlAccessTests.tableName)) + } + + + "testingLongVsFloat" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsFloat(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsDouble" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsDouble(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsDate" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsTime" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingLongArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingLongVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatVsDouble" should "be tested" in{ + assert(SparkSqlAccessTests.testingFloatVsDouble(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatVsDate" should "be tested" in{ + assert(SparkSqlAccessTests.testingFloatVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatVsTime" should "be tested" in{ + assert(SparkSqlAccessTests.testingFloatVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingFloatVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingFloatVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingFloatVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingFloatArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingFloatVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingDoubleVsDate" should "be tested" in{ + assert(SparkSqlAccessTests.testingDoubleVsDate(spark, SparkSqlAccessTests.tableName)) + } + + "testingDoubleVsTime" should "be tested" in{ + assert(SparkSqlAccessTests.testingDoubleVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingDoubleVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingDoubleVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingDoubleVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingDoubleVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingDoubleVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingDoubleVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingDoubleArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingDoubleVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingDateVsTime" should "be tested" in{ + assert(SparkSqlAccessTests.testingDateVsTime(spark, SparkSqlAccessTests.tableName)) + } + + "testingDateVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingDateVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingDateVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingDateVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingDateVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingDateVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingDateArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingDateVsArray(spark, SparkSqlAccessTests.tableName)) + } + + + "testingTimeVsTimeStamp" should "be tested" in{ + assert(SparkSqlAccessTests.testingTimeVsTimeStamp(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingTimeVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingTimeVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingTimeVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeStampVsBinary" should "be tested" in{ + assert(SparkSqlAccessTests.testingTimeStampVsBinary(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeStampVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingTimeStampVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingTimeStampArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingTimeStampVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingBinaryVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingBinaryVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testingBinaryVsArray" should "be tested" in{ + assert(SparkSqlAccessTests.testingBinaryVsArray(spark, SparkSqlAccessTests.tableName)) + } + + "testingArrayVsMap" should "be tested" in{ + assert(SparkSqlAccessTests.testingArrayVsMap(spark, SparkSqlAccessTests.tableName)) + } + + "testFilterPushDownOnIdColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testFilterPushDownOnIdColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testFilterPushDownOnMapColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testFilterPushDownOnMapColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testFilterPushDownOnArrayColumn" should " be tested" in { + assert(SparkSqlPushDownTests.testFilterPushDownOnArrayColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testLTFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testLTFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testLTEFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testLTEFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testGTFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testGTFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testGTEFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testGTEFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testComplexOrFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testComplexOrFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testComplexAndFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testComplexAndFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testStartsWithPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testStartsWithPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testEndsWithPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testEndsWithPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testContainsPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testContainsPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testINPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testINPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testComplexNotOrFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testComplexNotOrFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testComplexNotAndFilterPushDownOnNonIDColumn" should "be tested" in { + assert(SparkSqlPushDownTests.testComplexNotAndFilterPushDownOnNonIDColumn(spark, SparkSqlPushDownTests.tableName)) + } + + "testProjectionPushDown" should "be tested" in { + assert(SparkSqlPushDownTests.testProjectionPushDown(spark, SparkSqlPushDownTests.tableName)) + } + + "testProjectionPushDownNextFields" should "be tested" in { + assert(SparkSqlPushDownTests.testProjectionPushDownNestedFields(spark, SparkSqlPushDownTests.tableName)) + } + + "testFilterPDOnDFWithRDDFilter" should "be tested" in { + assert(SparkSqlPushDownTests.testFilterPDOnDFWithRDDFilter(spark, SparkSqlPushDownTests.tableName)) + } + + "testProjectionPDOnDFWithRDDSelection" should "be tested" in { + assert(SparkSqlPushDownTests.testProjectionPDOnDFWithRDDSelection(spark, SparkSqlPushDownTests.tableName)) + } + + "testProjectionPDOnDFWithRDDSelectionErrorCondition" should "be tested" in { + assert(SparkSqlPushDownTests.testProjectionPDOnDFWithRDDSelectionErrorCondition(spark, SparkSqlPushDownTests.tableName)) + } + + "testingLoadExplicitSchema" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadExplicitSchema(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingLoadBeanClass" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadBeanClass(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingLoadInferSchema" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadInferSchema(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingLoadFromDFReader" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadFromDFReader(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingLoadFromDFReaderLoad" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadFromDFReaderLoad(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingLoadFromDFWriterWithOperationOption" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadFromDFWriterWithOperationOption(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingLoadFromDFReaderWithOperationOption" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadFromDFReaderWithOperationOption(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "tesitngLoadFromDFReaderWithSampleOption" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.tesitngLoadFromDFReaderWithSampleOption(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingLoadFromDFReaderWithFailOnConflict" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingLoadFromDFReaderWithFailOnConflict(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingSaveFromSparkSession" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingSaveFromSparkSession(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingSaveFromDFWSession" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingSaveFromDFWSession(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingSaveWithBulkLoad" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingSaveWithBulkLoad(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingSaveWithComplexDocument" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingSaveWithComplexDocument(spark)) + } + + "testingUpdateToMapRDB" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingUpdateToMapRDB(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingCheckAndUpdateToMapRDB" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingCheckAndUpdateToMapRDB(spark, SparkSqlLoadAndSaveTests.tableName)) + } + + "testingUpdateToMapRDBAddToArray" should "be tested" in { + assert(SparkSqlLoadAndSaveTests.testingUpdateToMapRDBAddToArray(spark, SparkSqlLoadAndSaveTests.tableName)) + } + +// "testSavingDStreamToMapRDBTable" should "be tested" in { +// assert(SparkStreamingTests.testSavingDStreamToMapRDBTable(ssc, SparkStreamingTests.tableName)) +// } + + "testingSimpleSaveModeBeanClass" should "be tested" in { + assert(LoadAndSaveTests.testingSimpleSaveModeBeanClass(spark.sparkContext, tableName, LoadAndSaveTests.saveToTable)) + } +} diff --git a/external/maprdb/src/test/scala/com/mapr/db/tests/InitCluster.scala b/external/maprdb/src/test/scala/com/mapr/db/tests/InitCluster.scala new file mode 100644 index 0000000000000..1bedb1c1ecf91 --- /dev/null +++ b/external/maprdb/src/test/scala/com/mapr/db/tests/InitCluster.scala @@ -0,0 +1,109 @@ +package com.mapr.db.tests + +import org.apache.spark.SparkContext +import org.apache.spark.SparkConf +import org.scalatest._ +import com.mapr.db.testCases._ + +import org.apache.spark.sql.SparkSession +import org.apache.spark.streaming.{Seconds, StreamingContext} + +trait InitClusterDisableKryoSerialization extends BeforeAndAfterAll { + this: Suite => + + private val master = "local[4]" + private val appName = this.getClass.getSimpleName + val tableName = "/tmp/user_profiles_read_allTests_sparkOJAIConnector" + val tableName2 = "/tmp/user_profiles_save_allTests_sparkOJAIConnector" + + private var _sc: SparkContext = _ + private var _ssc: StreamingContext = _ + + def sc = _sc + def ssc = _ssc + + val conf: SparkConf = { + val cnf = new SparkConf() + .setMaster(master) + .setAppName(appName) + cnf + } + + lazy val spark = SparkSession.builder().appName("simpletest").config(conf).getOrCreate() + + override def beforeAll(): Unit = { + super.beforeAll() + _sc = spark.sparkContext + _sc.setLogLevel("OFF") + _ssc = new StreamingContext(spark.sparkContext, Seconds(10)) + com.mapr.db.testCases.MapRDBSparkTests.tableInitialization(sc, MapRDBSparkTests.tableName) + com.mapr.db.testCases.LoadAndSaveTests.tableInitialization(sc, tableName) + com.mapr.db.testCases.PredicateTests.tableInitialization(sc, PredicateTests.tableName) + com.mapr.db.testCases.SparkSqlAccessTests.tableInitialization(SparkSqlAccessTests.tableName) + com.mapr.db.testCases.MapRDBSparkTests.tableInitialization(sc, SparkSqlPushDownTests.tableName) + MapRDBSparkTests.tableInitialization(spark.sparkContext, SparkSqlLoadAndSaveTests.tableName) + MapRDBSparkTests.tableInitialization(spark.sparkContext, SparkStreamingTests.tableName) + } + + + + override def afterAll(): Unit = { + if (_sc != null) { + _sc.stop() + _sc = null + _ssc.stop(false, true) + } + super.afterAll() + } +} + + + +trait InitClusterEnableKryoSerialization extends BeforeAndAfterAll { + this: Suite => + + val tableName = "/tmp/user_profiles_read_allTests_sparkOJAIConnector_kryo" + val tableName2 = "/tmp/user_profiles_save_allTests_sparkOJAIConnector_kryo" + private val master = "local[4]" + private val appName = this.getClass.getSimpleName + + private var _sc: SparkContext = _ + private var _ssc: StreamingContext = _ + + def sc = _sc + def ssc = _ssc + + val conf: SparkConf = { + val cnf = new SparkConf() + .setMaster(master) + .setAppName(appName) + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "com.mapr.db.spark.OJAIKryoRegistrator") + cnf + } + + lazy val spark = SparkSession.builder().appName("simpletest").config(conf).getOrCreate() + + override def beforeAll(): Unit = { + super.beforeAll() + _sc = new SparkContext(conf) + _sc.setLogLevel("OFF") + _ssc = new StreamingContext(spark.sparkContext, Seconds(10)) + com.mapr.db.testCases.MapRDBSparkTests.tableInitialization(sc, MapRDBSparkTests.tableName) + com.mapr.db.testCases.LoadAndSaveTests.tableInitialization(sc, tableName) + com.mapr.db.testCases.PredicateTests.tableInitialization(sc, PredicateTests.tableName) + com.mapr.db.testCases.SparkSqlAccessTests.tableInitialization(SparkSqlAccessTests.tableName) + com.mapr.db.testCases.MapRDBSparkTests.tableInitialization(sc, SparkSqlPushDownTests.tableName) + MapRDBSparkTests.tableInitialization(spark.sparkContext, SparkSqlLoadAndSaveTests.tableName) + MapRDBSparkTests.tableInitialization(spark.sparkContext, SparkStreamingTests.tableName) + } + + override def afterAll(): Unit = { + if (_sc != null) { + _sc.stop() + _sc = null + _ssc.stop(false, true) + } + super.afterAll() + } +} diff --git a/pom.xml b/pom.xml index b1c1630f38d99..5eff2da8b05da 100644 --- a/pom.xml +++ b/pom.xml @@ -98,6 +98,7 @@ sql/core sql/hive assembly + external/maprdb external/flume external/flume-sink external/flume-assembly