From 19edd3e5a4ac0f571b36e3deafe49c1170042a32 Mon Sep 17 00:00:00 2001 From: Michele Rastelli Date: Thu, 9 Dec 2021 10:21:40 +0100 Subject: [PATCH 1/8] options store as CaseInsensitiveMap --- .../spark/sql/arangodb/commons/ArangoOptions.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoOptions.scala b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoOptions.scala index 2874d795..e2a83898 100644 --- a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoOptions.scala +++ b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoOptions.scala @@ -20,15 +20,15 @@ package org.apache.spark.sql.arangodb.commons -import com.arangodb.{ArangoDB, entity} import com.arangodb.model.OverwriteMode -import org.apache.spark.sql.catalyst.util.{ParseMode, PermissiveMode} +import com.arangodb.{ArangoDB, entity} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, ParseMode, PermissiveMode} import java.io.ByteArrayInputStream import java.security.KeyStore import java.security.cert.CertificateFactory import java.util -import java.util.{Base64, Locale} +import java.util.Base64 import javax.net.ssl.{SSLContext, TrustManagerFactory} import scala.collection.JavaConverters.mapAsScalaMapConverter @@ -37,7 +37,7 @@ import scala.collection.JavaConverters.mapAsScalaMapConverter * @author Michele Rastelli */ class ArangoOptions(opts: Map[String, String]) extends Serializable { - private val options: Map[String, String] = opts.map(e => (e._1.toLowerCase(Locale.US), e._2)) + private val options: Map[String, String] = CaseInsensitiveMap(opts) lazy val driverOptions: ArangoDriverOptions = new ArangoDriverOptions(options) lazy val readOptions: ArangoReadOptions = new ArangoReadOptions(options) @@ -89,7 +89,7 @@ object ArangoOptions { val FILL_BLOCK_CACHE = "fill.cache" val STREAM = "stream" val PARSE_MODE = "mode" - val CORRUPT_RECORDS_COLUMN = "columnnameofcorruptrecord" + val CORRUPT_RECORDS_COLUMN = "columnNameOfCorruptRecord" // write options val NUMBER_OF_SHARDS = "table.shards" From 9e1fea89646500ac884549ced4d4825dfbeaa427 Mon Sep 17 00:00:00 2001 From: Michele Rastelli Date: Thu, 9 Dec 2021 15:37:18 +0100 Subject: [PATCH 2/8] ArangoDBConf --- .../sql/arangodb/commons/ArangoDBConf.scala | 497 ++++++++++++++++++ .../sql/arangodb/commons/ArangoOptions.scala | 50 +- .../mapping/ArangoGeneratorImpl.scala | 4 +- .../datasource/mapping/ArangoParserImpl.scala | 4 +- .../ArangoCollectionPartitionReader.scala | 4 +- .../datasource/reader/ArangoQueryReader.scala | 4 +- .../datasource/writer/ArangoDataWriter.scala | 4 +- .../spark/sql/arangodb/JacksonTest.scala | 4 +- 8 files changed, 543 insertions(+), 28 deletions(-) create mode 100644 arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala diff --git a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala new file mode 100644 index 00000000..9f0ad31f --- /dev/null +++ b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala @@ -0,0 +1,497 @@ +package org.apache.spark.sql.arangodb.commons + +import com.arangodb.model.OverwriteMode +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DropMalformedMode, FailFastMode, PermissiveMode} + +import java.util +import scala.collection.JavaConverters.{mapAsJavaMapConverter, mapAsScalaMapConverter} + +object ArangoDBConf { + + val USER = "user" + val userConf: ConfigEntry[String] = ConfigBuilder(USER) + .doc("db user") + .version("1.0.0") + .stringConf + .createWithDefault("root") + + val PASSWORD = "password" + val passwordConf: OptionalConfigEntry[String] = ConfigBuilder(PASSWORD) + .doc("db password") + .version("1.0.0") + .stringConf + .createOptional + + val ENDPOINTS = "endpoints" + val endpointsConf: OptionalConfigEntry[String] = ConfigBuilder(ENDPOINTS) + .doc("A comma-separated list of coordinators, eg. c1:8529,c2:8529") + .version("1.0.0") + .stringConf + .createOptional + + val DB = "database" + val dbConf: ConfigEntry[String] = ConfigBuilder(DB) + .doc("database name") + .version("1.0.0") + .stringConf + .createWithDefault("_system") + + val ACQUIRE_HOST_LIST = "acquireHostList" + val acquireHostListConf: ConfigEntry[Boolean] = ConfigBuilder(ACQUIRE_HOST_LIST) + .doc("acquire the list of all known hosts in the cluster") + .version("1.0.0") + .booleanConf + .createWithDefault(false) + + val PROTOCOL = "protocol" + val protocolConf: ConfigEntry[String] = ConfigBuilder(PROTOCOL) + .doc("communication protocol") + .version("1.0.0") + .stringConf + .checkValues(Set(Protocol.HTTP.name, Protocol.VST.name)) + .createWithDefault(Protocol.HTTP.name) + + val CONTENT_TYPE = "contentType" + val contentTypeConf: ConfigEntry[String] = ConfigBuilder(CONTENT_TYPE) + .doc("content type for driver communication") + .version("1.0.0") + .stringConf + .checkValues(Set(ContentType.VPACK.name, ContentType.JSON.name)) + .createWithDefault(ContentType.JSON.name) + + val SSL_ENABLED = "ssl.enabled" + val sslEnabledConf: ConfigEntry[Boolean] = ConfigBuilder(SSL_ENABLED) + .doc("SSL secured driver connection") + .version("1.0.0") + .booleanConf + .createWithDefault(false) + + val SSL_CERT_VALUE = "ssl.cert.value" + val sslCertValueConf: OptionalConfigEntry[String] = ConfigBuilder(SSL_CERT_VALUE) + .doc("base64 encoded certificate") + .version("1.0.0") + .stringConf + .createOptional + + val SSL_CERT_TYPE = "ssl.cert.type" + val sslCertTypeConf: ConfigEntry[String] = ConfigBuilder(SSL_CERT_TYPE) + .doc("certificate type") + .version("1.0.0") + .stringConf + .createWithDefault("X.509") + + val SSL_CERT_ALIAS = "ssl.cert.alias" + val sslCertAliasConf: ConfigEntry[String] = ConfigBuilder(SSL_CERT_ALIAS) + .doc("certificate alias name") + .version("1.0.0") + .stringConf + .createWithDefault("arangodb") + + val SSL_ALGORITHM = "ssl.algorithm" + val sslAlgorithmConf: ConfigEntry[String] = ConfigBuilder(SSL_ALGORITHM) + .doc("trust manager algorithm") + .version("1.0.0") + .stringConf + .createWithDefault("SunX509") + + val SSL_KEYSTORE_TYPE = "ssl.keystore.type" + val sslKeystoreTypeConf: ConfigEntry[String] = ConfigBuilder(SSL_KEYSTORE_TYPE) + .doc("keystore type") + .version("1.0.0") + .stringConf + .createWithDefault("jks") + + val SSL_PROTOCOL = "ssl.protocol" + val sslProtocolConf: ConfigEntry[String] = ConfigBuilder(SSL_PROTOCOL) + .doc("SSLContext protocol") + .version("1.0.0") + .stringConf + .createWithDefault("TLS") + + val COLLECTION = "table" + val collectionConf: OptionalConfigEntry[String] = ConfigBuilder(COLLECTION) + .doc("ArangoDB collection name") + .version("1.0.0") + .stringConf + .createOptional + + val BATCH_SIZE = "batchSize" + val batchSizeConf: ConfigEntry[Int] = ConfigBuilder(BATCH_SIZE) + .doc("batch size") + .version("1.0.0") + .intConf + .createWithDefault(10000) + + val QUERY = "query" + val queryConf: OptionalConfigEntry[String] = ConfigBuilder(QUERY) + .doc("custom AQL read query") + .version("1.0.0") + .stringConf + .createOptional + + val SAMPLE_SIZE = "sampleSize" + val sampleSizeConf: ConfigEntry[Int] = ConfigBuilder(SAMPLE_SIZE) + .doc("sample size prefetched for schema inference") + .version("1.0.0") + .intConf + .createWithDefault(1000) + + val FILL_BLOCK_CACHE = "fillBlockCache" + val fillBlockCacheConf: ConfigEntry[Boolean] = ConfigBuilder(FILL_BLOCK_CACHE) + .doc("whether the query should store the data it reads in the RocksDB block cache") + .version("1.0.0") + .booleanConf + .createWithDefault(false) + + val STREAM = "stream" + val streamConf: ConfigEntry[Boolean] = ConfigBuilder(STREAM) + .doc("whether the query should be executed lazily") + .version("1.0.0") + .booleanConf + .createWithDefault(true) + + val PARSE_MODE = "mode" + val parseModeConf: ConfigEntry[String] = ConfigBuilder(PARSE_MODE) + .doc("allows a mode for dealing with corrupt records during parsing") + .version("1.0.0") + .stringConf + .checkValues(Set(PermissiveMode.name, DropMalformedMode.name, FailFastMode.name)) + .createWithDefault(PermissiveMode.name) + + val CORRUPT_RECORD_COLUMN = "columnNameOfCorruptRecord" + val corruptRecordColumnConf: OptionalConfigEntry[String] = ConfigBuilder(CORRUPT_RECORD_COLUMN) + .doc("allows renaming the new field having malformed string created by PERMISSIVE mode") + .version("1.0.0") + .stringConf + .createOptional + + val NUMBER_OF_SHARDS = "table.shards" + val numberOfShardsConf: ConfigEntry[Int] = ConfigBuilder(NUMBER_OF_SHARDS) + .doc("number of shards of the created collection (in case of SaveMode Append or Overwrite)") + .version("1.0.0") + .intConf + .createWithDefault(1) + + val COLLECTION_TYPE = "table.type" + val collectionTypeConf: ConfigEntry[String] = ConfigBuilder(COLLECTION_TYPE) + .doc("type of the created collection (in case of SaveMode Append or Overwrite)") + .version("1.0.0") + .stringConf + .checkValues(Set(CollectionType.DOCUMENT.name, CollectionType.EDGE.name)) + .createWithDefault(CollectionType.DOCUMENT.name) + + val WAIT_FOR_SYNC = "waitForSync" + val waitForSyncConf: ConfigEntry[Boolean] = ConfigBuilder(WAIT_FOR_SYNC) + .doc("whether to wait until the documents have been synced to disk") + .version("1.0.0") + .booleanConf + .createWithDefault(false) + + val CONFIRM_TRUNCATE = "confirmTruncate" + val confirmTruncateConf: ConfigEntry[Boolean] = ConfigBuilder(CONFIRM_TRUNCATE) + .doc("confirm to truncate table when using SaveMode.Overwrite mode") + .version("1.0.0") + .booleanConf + .createWithDefault(false) + + val OVERWRITE_MODE = "overwriteMode" + val overwriteModeConf: ConfigEntry[String] = ConfigBuilder(OVERWRITE_MODE) + .doc("configures the behavior in case a document with the specified _key value exists already") + .version("1.0.0") + .stringConf + .checkValues(Set( + OverwriteMode.ignore.getValue, + OverwriteMode.replace.getValue, + OverwriteMode.update.getValue, + OverwriteMode.conflict.getValue + )) + .createWithDefault(OverwriteMode.conflict.getValue) + + val MERGE_OBJECTS = "mergeObjects" + val mergeObjectsConf: ConfigEntry[Boolean] = ConfigBuilder(MERGE_OBJECTS) + .doc("in case overwrite.mode is set to update, controls whether objects (not arrays) will be merged") + .version("1.0.0") + .booleanConf + .createWithDefault(true) + + val KEEP_NULL = "keepNull" + val keepNullConf: ConfigEntry[Boolean] = ConfigBuilder(KEEP_NULL) + .doc("whether null values are saved within the document or used to delete corresponding existing attributes") + .version("1.0.0") + .booleanConf + .createWithDefault(true) + + + private[sql] val confEntries: Map[String, ConfigEntry[_]] = CaseInsensitiveMap(Map( + // driver config + USER -> userConf, + PASSWORD -> passwordConf, + ENDPOINTS -> endpointsConf, + DB -> dbConf, + ACQUIRE_HOST_LIST -> acquireHostListConf, + PROTOCOL -> protocolConf, + CONTENT_TYPE -> contentTypeConf, + SSL_ENABLED -> sslEnabledConf, + SSL_CERT_VALUE -> sslCertValueConf, + SSL_CERT_TYPE -> sslCertTypeConf, + SSL_CERT_ALIAS -> sslCertAliasConf, + SSL_ALGORITHM -> sslAlgorithmConf, + SSL_KEYSTORE_TYPE -> sslKeystoreTypeConf, + SSL_PROTOCOL -> sslProtocolConf, + + // read/write config + COLLECTION -> collectionConf, + BATCH_SIZE -> batchSizeConf, + + // read config + QUERY -> queryConf, + SAMPLE_SIZE -> sampleSizeConf, + FILL_BLOCK_CACHE -> fillBlockCacheConf, + STREAM -> streamConf, + PARSE_MODE -> parseModeConf, + CORRUPT_RECORD_COLUMN -> corruptRecordColumnConf, + + // write config + NUMBER_OF_SHARDS -> numberOfShardsConf, + COLLECTION_TYPE -> collectionTypeConf, + WAIT_FOR_SYNC -> waitForSyncConf, + CONFIRM_TRUNCATE -> confirmTruncateConf, + OVERWRITE_MODE -> overwriteModeConf, + MERGE_OBJECTS -> mergeObjectsConf, + KEEP_NULL -> keepNullConf + )) + + /** + * Holds information about keys that have been deprecated. + * + * @param key The deprecated key. + * @param version Version of Spark ArangoDB where key was deprecated. + * @param comment Additional info regarding to the removed config. For example, + * reasons of config deprecation, what users should use instead of it. + */ + case class DeprecatedConfig(key: String, version: String, comment: String) + + /** + * Maps deprecated Spark ArangoDB config keys to information about the deprecation. + * + * The extra information is logged as a warning when the Spark ArangoDB config is present + * in the user's configuration. + */ + val deprecatedArangoDBConfigs: Map[String, DeprecatedConfig] = { + val configs: Seq[DeprecatedConfig] = Seq() + CaseInsensitiveMap(Map(configs.map { cfg => cfg.key -> cfg }: _*)) + } + + /** + * Holds information about keys that have been removed. + * + * @param key The removed config key. + * @param version Version of Spark ArangoDB where key was removed. + * @param defaultValue The default config value. It can be used to notice + * users that they set non-default value to an already removed config. + * @param comment Additional info regarding to the removed config. + */ + case class RemovedConfig(key: String, version: String, defaultValue: String, comment: String) + + /** + * The map contains info about removed Spark ArangoDB configs. Keys are Spark ArangoDB config names, + * map values contain extra information like the version in which the config was removed, + * config's default value and a comment. + */ + val removedArangoDBConfigs: Map[String, RemovedConfig] = { + val configs: Seq[RemovedConfig] = Seq() + CaseInsensitiveMap(Map(configs.map { cfg => cfg.key -> cfg }: _*)) + } + + def apply(options: Map[String, String]): ArangoDBConf = new ArangoDBConf(options) + + def apply(options: util.Map[String, String]): ArangoDBConf = ArangoDBConf(options.asScala.toMap) + +} + +class ArangoDBConf(opts: Map[String, String]) extends Serializable with Logging { + + import ArangoDBConf._ + + private val options = CaseInsensitiveMap(opts) + private val settings = options.asJava + settings.forEach(checkConf(_, _)) + + @transient protected val reader = new ConfigReader(settings) + + lazy val driverOptions: ArangoDBDriverConf = new ArangoDBDriverConf(options) + lazy val readOptions: ArangoDBReadConf = new ArangoDBReadConf(options) + lazy val writeOptions: ArangoDBWriteConf = new ArangoDBWriteConf(options) + + def updated(kv: (String, String)): ArangoDBConf = new ArangoDBConf(options + kv) + + def updated(other: ArangoDBConf): ArangoDBConf = new ArangoDBConf(options ++ other.options) + + protected def getRequiredConf[T](entry: OptionalConfigEntry[T]): T = + getConf(entry).getOrElse(throw new IllegalArgumentException(s"Required ${entry.key} configuration parameter")) + + /** Return the value of Spark ArangoDB configuration property for the given key. */ + @throws[NoSuchElementException]("if key is not set") + def getConfString(key: String): String = Option(settings.get(key)).getOrElse(throw new NoSuchElementException(key)) + + /** + * Return the value of Spark ArangoDB configuration property for the given key. If the key is not set + * yet, return `defaultValue`. This is useful when `defaultValue` in ConfigEntry is not the + * desired one. + */ + def getConf[T](entry: ConfigEntry[T], defaultValue: T): T = + Option(settings.get(entry.key)).map(entry.valueConverter).getOrElse(defaultValue) + + /** + * Return the value of Spark ArangoDB configuration property for the given key. If the key is not set + * yet, return `defaultValue` in [[ConfigEntry]]. + */ + protected def getConf[T](entry: ConfigEntry[T]): T = entry.readFrom(reader) + + /** + * Return the value of an optional Spark ArangoDB configuration property for the given key. If the key + * is not set yet, returns None. + */ + protected def getConf[T](entry: OptionalConfigEntry[T]): Option[T] = entry.readFrom(reader) + + /** + * Return the `string` value of Spark ArangoDB configuration property for the given key. If the key is + * not set, return `defaultValue`. + */ + def getConfString(key: String, defaultValue: String): String = Option(settings.get(key)).getOrElse(defaultValue) + + /** + * Return all the configuration properties that have been set (i.e. not the default). + * This creates a new copy of the config properties in the form of a Map. + */ + def getAllConfigs: Map[String, String] = settings.asScala.toMap + + /** + * Return all the configuration definitions that have been defined in [[ArangoDBConf]]. Each + * definition contains key, defaultValue and doc. + */ + def getAllDefinedConfigs: Seq[(String, String, String, String)] = + confEntries.values.filter(_.isPublic).map { entry => + val displayValue = Option(getConfString(entry.key, null)).getOrElse(entry.defaultValueString) + (entry.key, displayValue, entry.doc, entry.version) + }.toSeq + + /** + * Logs a warning message if the given config key is deprecated. + */ + private def logDeprecationWarning(key: String): Unit = { + ArangoDBConf.deprecatedArangoDBConfigs.get(key).foreach { + case DeprecatedConfig(configName, version, comment) => + logWarning( + s"The Spark ArangoDB config '$configName' has been deprecated since version v$version " + + s"and may be removed in the future. $comment") + } + } + + private def requireDefaultValueOfRemovedConf(key: String, value: String): Unit = { + ArangoDBConf.removedArangoDBConfigs.get(key).foreach { + case RemovedConfig(configName, version, defaultValue, comment) => + if (value != defaultValue) { + throw new AnalysisException( + s"The Spark ArangoDB config '$configName' was removed in the version $version. $comment") + } + } + } + + private def checkConf(key: String, value: String): Unit = { + logDeprecationWarning(key) + requireDefaultValueOfRemovedConf(key, value) + } + +} + + +class ArangoDBDriverConf(opts: Map[String, String]) extends ArangoDBConf(opts) { + + import ArangoDBConf._ + + val user: String = getConf(userConf) + + val password: Option[String] = getConf(passwordConf) + + val endpoints: Array[String] = getRequiredConf(endpointsConf).split(",") + + val db: String = getConf(dbConf) + + val acquireHostList: Boolean = getConf(acquireHostListConf) + + val protocol: Protocol = Protocol(getConf(protocolConf)) + + val contentType: ContentType = ContentType(getConf(contentTypeConf)) + + val sslEnabled: Boolean = getConf(sslEnabledConf) + + val sslCertValue: Option[String] = getConf(sslCertValueConf) + + val sslCertType: String = getConf(sslCertTypeConf) + + val sslCertAlias: String = getConf(sslCertAliasConf) + + val sslAlgorithm: String = getConf(sslAlgorithmConf) + + val sslKeystoreType: String = getConf(sslKeystoreTypeConf) + + val sslProtocol: String = getConf(sslProtocolConf) + +} + + +class ArangoDBReadConf(opts: Map[String, String]) extends ArangoDBConf(opts) { + + import ArangoDBConf._ + + val collection: Option[String] = getConf(collectionConf) + + val query: Option[String] = getConf(queryConf) + + val batchSize: Int = getConf(batchSizeConf) + + val sampleSize: Int = getConf(sampleSizeConf) + + val fillBlockCache: Boolean = getConf(fillBlockCacheConf) + + val stream: Boolean = getConf(streamConf) + + val parseMode: String = getConf(parseModeConf) + + val corruptRecordColumn: String = getConf(corruptRecordColumnConf).getOrElse("") + + val readMode: ReadMode = + if (query.isDefined) ReadMode.Query + else if (collection.isDefined) ReadMode.Collection + else throw new IllegalArgumentException("Either collection or query must be defined") + +} + + +class ArangoDBWriteConf(opts: Map[String, String]) extends ArangoDBConf(opts) { + + import ArangoDBConf._ + + val collection: String = getRequiredConf(collectionConf) + + val batchSize: Int = getConf(batchSizeConf) + + val numberOfShards: Int = getConf(numberOfShardsConf) + + val collectionType: String = getConf(collectionTypeConf) + + val waitForSync: Boolean = getConf(waitForSyncConf) + + val confirmTruncate: Boolean = getConf(confirmTruncateConf) + + val overwriteMode: String = getConf(overwriteModeConf) + + val mergeObjects: Boolean = getConf(mergeObjectsConf) + + val keepNull: Boolean = getConf(keepNullConf) + +} \ No newline at end of file diff --git a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoOptions.scala b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoOptions.scala index e2a83898..82bf8a98 100644 --- a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoOptions.scala +++ b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoOptions.scala @@ -110,10 +110,10 @@ class ArangoDriverOptions(options: Map[String, String]) extends Serializable { private val protocol = Protocol(options.getOrElse(ArangoOptions.PROTOCOL, "http")) private val contentType: ContentType = ContentType(options.getOrElse(ArangoOptions.CONTENT_TYPE, "vpack")) private val arangoProtocol = (protocol, contentType) match { - case (Protocol.VST, ContentType.VPack) => com.arangodb.Protocol.VST - case (Protocol.VST, ContentType.Json) => throw new IllegalArgumentException("Json over VST is not supported") - case (Protocol.HTTP, ContentType.VPack) => com.arangodb.Protocol.HTTP_VPACK - case (Protocol.HTTP, ContentType.Json) => com.arangodb.Protocol.HTTP_JSON + case (Protocol.VST, ContentType.VPACK) => com.arangodb.Protocol.VST + case (Protocol.VST, ContentType.JSON) => throw new IllegalArgumentException("Json over VST is not supported") + case (Protocol.HTTP, ContentType.VPACK) => com.arangodb.Protocol.HTTP_VPACK + case (Protocol.HTTP, ContentType.JSON) => com.arangodb.Protocol.HTTP_JSON } private val sslEnabled: Boolean = options.getOrElse(ArangoOptions.SSL_ENABLED, "false").toBoolean private val sslCert: Option[String] = options.get(ArangoOptions.SSL_CERT) @@ -210,50 +210,68 @@ object ReadMode { case object Query extends ReadMode } -sealed trait ContentType +sealed trait ContentType { + val name: String +} object ContentType { - case object Json extends ContentType + case object JSON extends ContentType { + override val name: String = "json" + } - case object VPack extends ContentType + case object VPACK extends ContentType { + override val name: String = "vpack" + } def apply(value: String): ContentType = value match { - case "json" => Json - case "vpack" => VPack + case JSON.name => JSON + case VPACK.name => VPACK case _ => throw new IllegalArgumentException(s"${ArangoOptions.CONTENT_TYPE}: $value") } } -sealed trait Protocol +sealed trait Protocol { + val name: String +} object Protocol { - case object VST extends Protocol + case object VST extends Protocol { + override val name: String = "vst" + } - case object HTTP extends Protocol + case object HTTP extends Protocol { + override val name: String = "http" + } def apply(value: String): Protocol = value match { - case "vst" => VST - case "http" => HTTP + case VST.name => VST + case HTTP.name => HTTP case _ => throw new IllegalArgumentException(s"${ArangoOptions.PROTOCOL}: $value") } } sealed trait CollectionType { + val name: String + def get(): entity.CollectionType } object CollectionType { case object DOCUMENT extends CollectionType { + override val name: String = "document" + override def get() = entity.CollectionType.DOCUMENT } case object EDGE extends CollectionType { + override val name: String = "edge" + override def get() = entity.CollectionType.EDGES } def apply(value: String): CollectionType = value match { - case "document" => DOCUMENT - case "edge" => EDGE + case DOCUMENT.name => DOCUMENT + case EDGE.name => EDGE case _ => throw new IllegalArgumentException(s"${ArangoOptions.COLLECTION_TYPE}: $value") } } \ No newline at end of file diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoGeneratorImpl.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoGeneratorImpl.scala index 7dfdd78e..714d917a 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoGeneratorImpl.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoGeneratorImpl.scala @@ -20,8 +20,8 @@ abstract sealed class ArangoGeneratorImpl( class ArangoGeneratorProviderImpl extends ArangoGeneratorProvider { override def of(contentType: ContentType, schema: StructType, outputStream: OutputStream): ArangoGeneratorImpl = contentType match { - case ContentType.Json => new JsonArangoGenerator(schema, outputStream) - case ContentType.VPack => new VPackArangoGenerator(schema, outputStream) + case ContentType.JSON => new JsonArangoGenerator(schema, outputStream) + case ContentType.VPACK => new VPackArangoGenerator(schema, outputStream) case _ => throw new IllegalArgumentException } } diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoParserImpl.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoParserImpl.scala index 6ed20c1b..b7988ebb 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoParserImpl.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoParserImpl.scala @@ -25,8 +25,8 @@ abstract sealed class ArangoParserImpl( class ArangoParserProviderImpl extends ArangoParserProvider { override def of(contentType: ContentType, schema: DataType): ArangoParserImpl = contentType match { - case ContentType.Json => new JsonArangoParser(schema) - case ContentType.VPack => new VPackArangoParser(schema) + case ContentType.JSON => new JsonArangoParser(schema) + case ContentType.VPACK => new VPackArangoParser(schema) case _ => throw new IllegalArgumentException } } diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala index 122c2317..123d1321 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala @@ -40,8 +40,8 @@ class ArangoCollectionPartitionReader(inputPartition: ArangoCollectionPartition, if (iterator.hasNext) { val current = iterator.next() rowIterator = safeParser.parse(options.readOptions.contentType match { - case ContentType.VPack => current.toByteArray - case ContentType.Json => current.toString.getBytes(StandardCharsets.UTF_8) + case ContentType.VPACK => current.toByteArray + case ContentType.JSON => current.toString.getBytes(StandardCharsets.UTF_8) }) if (rowIterator.hasNext) true else next diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala index a293fd4a..fb16bce1 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala @@ -36,8 +36,8 @@ class ArangoQueryReader(schema: StructType, options: ArangoOptions) extends Part if (iterator.hasNext) { val current = iterator.next() rowIterator = safeParser.parse(options.readOptions.contentType match { - case ContentType.VPack => current.toByteArray - case ContentType.Json => current.toString.getBytes(StandardCharsets.UTF_8) + case ContentType.VPACK => current.toByteArray + case ContentType.JSON => current.toString.getBytes(StandardCharsets.UTF_8) }) if (rowIterator.hasNext) true else next diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala index 9e72f884..6d0616fb 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala @@ -72,8 +72,8 @@ class ArangoDataWriter(schema: StructType, options: ArangoOptions, partitionId: vpackGenerator.close() vpackGenerator.flush() val payload = options.writeOptions.contentType match { - case ContentType.VPack => new VPackSlice(outVPack.toByteArray) - case ContentType.Json => new VPackParser.Builder().build().fromJson(new String(outVPack.toByteArray), true) + case ContentType.VPACK => new VPackSlice(outVPack.toByteArray) + case ContentType.JSON => new VPackParser.Builder().build().fromJson(new String(outVPack.toByteArray), true) } saveDocuments(payload) } diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/JacksonTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/JacksonTest.scala index a2c501ac..6b3074ef 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/JacksonTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/JacksonTest.scala @@ -57,12 +57,12 @@ class JacksonTest { @Test def jsonRoudTrip(): Unit = { - roundTrip(ContentType.Json, jsonBytes) + roundTrip(ContentType.JSON, jsonBytes) } @Test def vpackRoudTrip(): Unit = { - roundTrip(ContentType.VPack, vpackBytes) + roundTrip(ContentType.VPACK, vpackBytes) } private def roundTrip(contentType: ContentType, data: Array[Byte]): Unit = { From 1deaae2f06cc9facca376ccfc66b9efb999e8839 Mon Sep 17 00:00:00 2001 From: Michele Rastelli Date: Fri, 10 Dec 2021 09:48:14 +0100 Subject: [PATCH 3/8] replaced ArangoOptions with ArangoDBConf --- .../sql/arangodb/commons/ArangoClient.scala | 35 ++-- .../sql/arangodb/commons/ArangoDBConf.scala | 87 +++++++-- .../sql/arangodb/commons/ArangoOptions.scala | 182 +----------------- .../sql/arangodb/commons/ArangoUtils.scala | 2 +- .../arangodb/datasource/DefaultSource.scala | 10 +- .../reader/ArangoCollectionPartition.scala | 5 +- .../ArangoCollectionPartitionReader.scala | 6 +- .../reader/ArangoDataSourceReader.scala | 4 +- .../datasource/reader/ArangoQueryReader.scala | 4 +- .../writer/ArangoDataSourceWriter.scala | 6 +- .../datasource/writer/ArangoDataWriter.scala | 6 +- .../writer/ArangoDataWriterFactory.scala | 3 +- .../sql/arangodb/datasource/ArangoTable.scala | 8 +- .../arangodb/datasource/DefaultSource.scala | 8 +- .../ArangoCollectionPartitionReader.scala | 10 +- .../reader/ArangoPartitionReaderFactory.scala | 4 +- .../datasource/reader/ArangoQueryReader.scala | 8 +- .../datasource/reader/ArangoScan.scala | 4 +- .../datasource/reader/ArangoScanBuilder.scala | 4 +- .../datasource/writer/ArangoBatchWriter.scala | 4 +- .../datasource/writer/ArangoDataWriter.scala | 15 +- .../writer/ArangoDataWriterFactory.scala | 4 +- .../writer/ArangoWriterBuilder.scala | 6 +- .../datasource/AcquireHostListTest.scala | 10 +- .../arangodb/datasource/BadRecordsTest.scala | 16 +- .../arangodb/datasource/BaseSparkTest.scala | 6 +- .../datasource/DeserializationCastTest.scala | 14 +- .../sql/arangodb/datasource/ReadTest.scala | 26 +-- .../datasource/ReadWriteDataTypeTest.scala | 14 +- .../arangodb/datasource/write/AbortTest.scala | 37 ++-- .../write/CreateCollectionTest.scala | 24 +-- .../datasource/write/OverwriteModeTest.scala | 43 +++-- .../datasource/write/SaveModeTest.scala | 62 +++--- .../write/WriteResiliencyTest.scala | 15 +- 34 files changed, 281 insertions(+), 411 deletions(-) diff --git a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoClient.scala b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoClient.scala index 94e99f71..4d519986 100644 --- a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoClient.scala +++ b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoClient.scala @@ -1,10 +1,10 @@ package org.apache.spark.sql.arangodb.commons import com.arangodb.entity.ErrorEntity -import com.arangodb.internal.{ArangoRequestParam, ArangoResponseField} import com.arangodb.internal.util.ArangoSerializationFactory.Serializer +import com.arangodb.internal.{ArangoRequestParam, ArangoResponseField} import com.arangodb.mapping.ArangoJack -import com.arangodb.model.{AqlQueryOptions, CollectionCreateOptions, OverwriteMode} +import com.arangodb.model.{AqlQueryOptions, CollectionCreateOptions} import com.arangodb.velocypack.VPackSlice import com.arangodb.velocystream.{Request, RequestType} import com.arangodb.{ArangoCursor, ArangoDB, ArangoDBException} @@ -17,12 +17,13 @@ import org.apache.spark.sql.types.StructType import scala.collection.JavaConverters.mapAsJavaMapConverter -class ArangoClient(options: ArangoOptions) extends Logging { +class ArangoClient(options: ArangoDBConf) extends Logging { private def aqlOptions(): AqlQueryOptions = { - val opt = new AqlQueryOptions().stream(options.readOptions.stream) - options.readOptions.fillBlockCache.foreach(opt.fillBlockCache(_)) - options.readOptions.batchSize.foreach(opt.batchSize(_)) + val opt = new AqlQueryOptions() + .stream(options.readOptions.stream) + .fillBlockCache(options.readOptions.fillBlockCache) + .batchSize(options.readOptions.batchSize) opt } @@ -105,8 +106,8 @@ class ArangoClient(options: ArangoOptions) extends Logging { def createCollection(): Unit = { val opts = new CollectionCreateOptions() - options.writeOptions.numberOfShards.foreach(opts.numberOfShards(_)) - options.writeOptions.collectionType.foreach(ct => opts.`type`(ct.get())) + .numberOfShards(options.writeOptions.numberOfShards) + .`type`(options.writeOptions.collectionType) arangoDB .db(options.writeOptions.db) @@ -131,14 +132,10 @@ class ArangoClient(options: ArangoOptions) extends Logging { s"/_api/document/${options.writeOptions.collection}") request.putQueryParam("silent", true) - options.writeOptions.waitForSync.foreach(request.putQueryParam("waitForSync", _)) - options.writeOptions.overwriteMode.foreach(it => { - request.putQueryParam("overwriteMode", it) - if (it == OverwriteMode.update) { - request.putQueryParam("keepNull", options.writeOptions.keepNull) - options.writeOptions.mergeObjects.foreach(request.putQueryParam("mergeObjects", _)) - } - }) + request.putQueryParam("waitForSync", options.writeOptions.waitForSync) + request.putQueryParam("overwriteMode", options.writeOptions.overwriteMode.getValue) + request.putQueryParam("keepNull", options.writeOptions.keepNull) + request.putQueryParam("mergeObjects", options.writeOptions.mergeObjects) request.setBody(data) val response = arangoDB.execute(request) @@ -162,9 +159,9 @@ class ArangoClient(options: ArangoOptions) extends Logging { object ArangoClient { - def apply(options: ArangoOptions): ArangoClient = new ArangoClient(options) + def apply(options: ArangoDBConf): ArangoClient = new ArangoClient(options) - def getCollectionShardIds(options: ArangoOptions): Array[String] = { + def getCollectionShardIds(options: ArangoDBConf): Array[String] = { try { val client = ArangoClient(options).arangoDB val res = client.execute(new Request( @@ -183,7 +180,7 @@ object ArangoClient { } } - def acquireHostList(options: ArangoOptions): Iterable[String] = { + def acquireHostList(options: ArangoDBConf): Iterable[String] = { val client = ArangoClient(options).arangoDB val response = client.execute(new Request(ArangoRequestParam.SYSTEM, RequestType.GET, "/_api/cluster/endpoints")) val field = response.getBody.get("endpoints") diff --git a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala index 9f0ad31f..cb037732 100644 --- a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala +++ b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala @@ -1,12 +1,18 @@ package org.apache.spark.sql.arangodb.commons +import com.arangodb.{ArangoDB, entity} import com.arangodb.model.OverwriteMode import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DropMalformedMode, FailFastMode, PermissiveMode} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DropMalformedMode, FailFastMode, ParseMode, PermissiveMode} +import java.io.ByteArrayInputStream +import java.security.KeyStore +import java.security.cert.CertificateFactory import java.util +import java.util.Base64 +import javax.net.ssl.{SSLContext, TrustManagerFactory} import scala.collection.JavaConverters.{mapAsJavaMapConverter, mapAsScalaMapConverter} object ArangoDBConf { @@ -32,13 +38,6 @@ object ArangoDBConf { .stringConf .createOptional - val DB = "database" - val dbConf: ConfigEntry[String] = ConfigBuilder(DB) - .doc("database name") - .version("1.0.0") - .stringConf - .createWithDefault("_system") - val ACQUIRE_HOST_LIST = "acquireHostList" val acquireHostListConf: ConfigEntry[Boolean] = ConfigBuilder(ACQUIRE_HOST_LIST) .doc("acquire the list of all known hosts in the cluster") @@ -111,6 +110,13 @@ object ArangoDBConf { .stringConf .createWithDefault("TLS") + val DB = "database" + val dbConf: ConfigEntry[String] = ConfigBuilder(DB) + .doc("database name") + .version("1.0.0") + .stringConf + .createWithDefault("_system") + val COLLECTION = "table" val collectionConf: OptionalConfigEntry[String] = ConfigBuilder(COLLECTION) .doc("ArangoDB collection name") @@ -161,8 +167,8 @@ object ArangoDBConf { .checkValues(Set(PermissiveMode.name, DropMalformedMode.name, FailFastMode.name)) .createWithDefault(PermissiveMode.name) - val CORRUPT_RECORD_COLUMN = "columnNameOfCorruptRecord" - val corruptRecordColumnConf: OptionalConfigEntry[String] = ConfigBuilder(CORRUPT_RECORD_COLUMN) + val COLUMN_NAME_OF_CORRUPT_RECORD = "columnNameOfCorruptRecord" + val columnNameOfCorruptRecordConf: OptionalConfigEntry[String] = ConfigBuilder(COLUMN_NAME_OF_CORRUPT_RECORD) .doc("allows renaming the new field having malformed string created by PERMISSIVE mode") .version("1.0.0") .stringConf @@ -230,7 +236,6 @@ object ArangoDBConf { USER -> userConf, PASSWORD -> passwordConf, ENDPOINTS -> endpointsConf, - DB -> dbConf, ACQUIRE_HOST_LIST -> acquireHostListConf, PROTOCOL -> protocolConf, CONTENT_TYPE -> contentTypeConf, @@ -243,6 +248,7 @@ object ArangoDBConf { SSL_PROTOCOL -> sslProtocolConf, // read/write config + DB -> dbConf, COLLECTION -> collectionConf, BATCH_SIZE -> batchSizeConf, @@ -252,7 +258,7 @@ object ArangoDBConf { FILL_BLOCK_CACHE -> fillBlockCacheConf, STREAM -> streamConf, PARSE_MODE -> parseModeConf, - CORRUPT_RECORD_COLUMN -> corruptRecordColumnConf, + COLUMN_NAME_OF_CORRUPT_RECORD -> columnNameOfCorruptRecordConf, // write config NUMBER_OF_SHARDS -> numberOfShardsConf, @@ -419,14 +425,17 @@ class ArangoDBDriverConf(opts: Map[String, String]) extends ArangoDBConf(opts) { val endpoints: Array[String] = getRequiredConf(endpointsConf).split(",") - val db: String = getConf(dbConf) - val acquireHostList: Boolean = getConf(acquireHostListConf) - val protocol: Protocol = Protocol(getConf(protocolConf)) - val contentType: ContentType = ContentType(getConf(contentTypeConf)) + private val arangoProtocol = (Protocol(getConf(protocolConf)), contentType) match { + case (Protocol.VST, ContentType.VPACK) => com.arangodb.Protocol.VST + case (Protocol.VST, ContentType.JSON) => throw new IllegalArgumentException("Json over VST is not supported") + case (Protocol.HTTP, ContentType.VPACK) => com.arangodb.Protocol.HTTP_VPACK + case (Protocol.HTTP, ContentType.JSON) => com.arangodb.Protocol.HTTP_JSON + } + val sslEnabled: Boolean = getConf(sslEnabledConf) val sslCertValue: Option[String] = getConf(sslCertValueConf) @@ -441,6 +450,40 @@ class ArangoDBDriverConf(opts: Map[String, String]) extends ArangoDBConf(opts) { val sslProtocol: String = getConf(sslProtocolConf) + def builder(): ArangoDB.Builder = { + val builder = new ArangoDB.Builder() + .useProtocol(arangoProtocol) + .user(user) + password.foreach(builder.password) + + if (sslEnabled) { + builder + .useSsl(true) + .sslContext(getSslContext) + } + + endpoints + .map(_.split(":")) + .foreach(host => builder.host(host(0), host(1).toInt)) + builder + } + + def getSslContext: SSLContext = sslCertValue match { + case Some(b64cert) => + val is = new ByteArrayInputStream(Base64.getDecoder.decode(b64cert)) + val cert = CertificateFactory.getInstance(sslCertType).generateCertificate(is) + val ks = KeyStore.getInstance(sslKeystoreType) + ks.load(null) + ks.setCertificateEntry(sslCertAlias, cert) + val tmf = TrustManagerFactory.getInstance(sslAlgorithm) + tmf.init(ks) + val sc = SSLContext.getInstance(sslProtocol) + sc.init(null, tmf.getTrustManagers, null) + sc + case None => SSLContext.getDefault + } + + } @@ -448,6 +491,8 @@ class ArangoDBReadConf(opts: Map[String, String]) extends ArangoDBConf(opts) { import ArangoDBConf._ + val db: String = getConf(dbConf) + val collection: Option[String] = getConf(collectionConf) val query: Option[String] = getConf(queryConf) @@ -460,9 +505,9 @@ class ArangoDBReadConf(opts: Map[String, String]) extends ArangoDBConf(opts) { val stream: Boolean = getConf(streamConf) - val parseMode: String = getConf(parseModeConf) + val parseMode: ParseMode = ParseMode.fromString(getConf(parseModeConf)) - val corruptRecordColumn: String = getConf(corruptRecordColumnConf).getOrElse("") + val columnNameOfCorruptRecord: String = getConf(columnNameOfCorruptRecordConf).getOrElse("") val readMode: ReadMode = if (query.isDefined) ReadMode.Query @@ -476,19 +521,21 @@ class ArangoDBWriteConf(opts: Map[String, String]) extends ArangoDBConf(opts) { import ArangoDBConf._ + val db: String = getConf(dbConf) + val collection: String = getRequiredConf(collectionConf) val batchSize: Int = getConf(batchSizeConf) val numberOfShards: Int = getConf(numberOfShardsConf) - val collectionType: String = getConf(collectionTypeConf) + val collectionType: entity.CollectionType = CollectionType(getConf(collectionTypeConf)).get() val waitForSync: Boolean = getConf(waitForSyncConf) val confirmTruncate: Boolean = getConf(confirmTruncateConf) - val overwriteMode: String = getConf(overwriteModeConf) + val overwriteMode: OverwriteMode = OverwriteMode.valueOf(getConf(overwriteModeConf)) val mergeObjects: Boolean = getConf(mergeObjectsConf) diff --git a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoOptions.scala b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoOptions.scala index 82bf8a98..041973de 100644 --- a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoOptions.scala +++ b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoOptions.scala @@ -20,181 +20,7 @@ package org.apache.spark.sql.arangodb.commons -import com.arangodb.model.OverwriteMode -import com.arangodb.{ArangoDB, entity} -import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, ParseMode, PermissiveMode} - -import java.io.ByteArrayInputStream -import java.security.KeyStore -import java.security.cert.CertificateFactory -import java.util -import java.util.Base64 -import javax.net.ssl.{SSLContext, TrustManagerFactory} -import scala.collection.JavaConverters.mapAsScalaMapConverter - - -/** - * @author Michele Rastelli - */ -class ArangoOptions(opts: Map[String, String]) extends Serializable { - private val options: Map[String, String] = CaseInsensitiveMap(opts) - - lazy val driverOptions: ArangoDriverOptions = new ArangoDriverOptions(options) - lazy val readOptions: ArangoReadOptions = new ArangoReadOptions(options) - lazy val writeOptions: ArangoWriteOptions = new ArangoWriteOptions(options) - - def updated(kv: (String, String)): ArangoOptions = new ArangoOptions(options + kv) - - def updated(other: ArangoOptions): ArangoOptions = new ArangoOptions(options ++ other.options) -} - -object ArangoOptions { - - // driver options - val USER = "user" - val PASSWORD = "password" - val ENDPOINTS = "endpoints" - val PROTOCOL = "protocol" - - val SSL_ENABLED = "ssl.enabled" - - // Base64 encoded certificate - val SSL_CERT = "ssl.cert.value" - - // certificate type, default "X.509" - val SSL_CERT_TYPE = "ssl.cert.type" - - // certificate alias name - val SSL_CERT_ALIAS = "ssl.cert.alias" - - // trustmanager algorithm, default "SunX509" - val SSL_ALGORITHM = "ssl.algorithm" - - // keystore type, default "jks" - val SSL_KEYSTORE = "ssl.keystore.type" - - // SSLContext protocol, default "TLS" - val SSL_PROTOCOL = "ssl.protocol" - - // read/write options - val DB = "database" - val COLLECTION = "table" - val BATCH_SIZE = "batch.size" - val CONTENT_TYPE = "content-type" - val ACQUIRE_HOST_LIST = "acquire-host-list" - - // read options - val QUERY = "query" - val SAMPLE_SIZE = "sample.size" - val FILL_BLOCK_CACHE = "fill.cache" - val STREAM = "stream" - val PARSE_MODE = "mode" - val CORRUPT_RECORDS_COLUMN = "columnNameOfCorruptRecord" - - // write options - val NUMBER_OF_SHARDS = "table.shards" - val COLLECTION_TYPE = "table.type" - val WAIT_FOR_SYNC = "wait.sync" - val CONFIRM_TRUNCATE = "confirm.truncate" - val OVERWRITE_MODE = "overwrite.mode" - val KEEP_NULL = "keep.null" - val MERGE_OBJECTS = "merge.objects" - - def apply(options: Map[String, String]): ArangoOptions = new ArangoOptions(options) - - def apply(options: util.Map[String, String]): ArangoOptions = ArangoOptions(options.asScala.toMap) - -} - -class ArangoDriverOptions(options: Map[String, String]) extends Serializable { - private val protocol = Protocol(options.getOrElse(ArangoOptions.PROTOCOL, "http")) - private val contentType: ContentType = ContentType(options.getOrElse(ArangoOptions.CONTENT_TYPE, "vpack")) - private val arangoProtocol = (protocol, contentType) match { - case (Protocol.VST, ContentType.VPACK) => com.arangodb.Protocol.VST - case (Protocol.VST, ContentType.JSON) => throw new IllegalArgumentException("Json over VST is not supported") - case (Protocol.HTTP, ContentType.VPACK) => com.arangodb.Protocol.HTTP_VPACK - case (Protocol.HTTP, ContentType.JSON) => com.arangodb.Protocol.HTTP_JSON - } - private val sslEnabled: Boolean = options.getOrElse(ArangoOptions.SSL_ENABLED, "false").toBoolean - private val sslCert: Option[String] = options.get(ArangoOptions.SSL_CERT) - private val sslCertType: String = options.getOrElse(ArangoOptions.SSL_CERT_TYPE, "X.509") - private val sslCertAlias: String = options.getOrElse(ArangoOptions.SSL_CERT_ALIAS, "arangodb") - private val sslAlgorithm: String = options.getOrElse(ArangoOptions.SSL_ALGORITHM, TrustManagerFactory.getDefaultAlgorithm) - private val sslKeystore: String = options.getOrElse(ArangoOptions.SSL_KEYSTORE, KeyStore.getDefaultType) - private val sslProtocol: String = options.getOrElse(ArangoOptions.SSL_PROTOCOL, "TLS") - - val endpoints: Seq[String] = options(ArangoOptions.ENDPOINTS).split(",") - val acquireHostList: Boolean = options.getOrElse(ArangoOptions.ACQUIRE_HOST_LIST, "false").toBoolean - - def builder(): ArangoDB.Builder = { - val builder = new ArangoDB.Builder() - .useProtocol(arangoProtocol) - - if (sslEnabled) { - builder - .useSsl(true) - .sslContext(getSslContext) - } - - options.get(ArangoOptions.USER).foreach(builder.user) - options.get(ArangoOptions.PASSWORD).foreach(builder.password) - endpoints - .map(_.split(":")) - .foreach(host => builder.host(host(0), host(1).toInt)) - builder - } - - def getSslContext: SSLContext = sslCert match { - case Some(b64cert) => - val is = new ByteArrayInputStream(Base64.getDecoder.decode(b64cert)) - val cert = CertificateFactory.getInstance(sslCertType).generateCertificate(is) - val ks = KeyStore.getInstance(sslKeystore) - ks.load(null) - ks.setCertificateEntry(sslCertAlias, cert) - val tmf = TrustManagerFactory.getInstance(sslAlgorithm) - tmf.init(ks) - val sc = SSLContext.getInstance(sslProtocol) - sc.init(null, tmf.getTrustManagers, null) - sc - case None => SSLContext.getDefault - } - -} - -abstract class CommonOptions(options: Map[String, String]) extends Serializable { - val db: String = options.getOrElse(ArangoOptions.DB, "_system") - val contentType: ContentType = ContentType(options.getOrElse(ArangoOptions.CONTENT_TYPE, "vpack")) - - protected def getRequired(key: String): String = options - .getOrElse(key, throw new IllegalArgumentException(s"Required $key configuration parameter not found")) -} - -class ArangoReadOptions(options: Map[String, String]) extends CommonOptions(options) { - val batchSize: Option[Int] = options.get(ArangoOptions.BATCH_SIZE).map(_.toInt) - val sampleSize: Int = options.get(ArangoOptions.SAMPLE_SIZE).map(_.toInt).getOrElse(1000) - val collection: Option[String] = options.get(ArangoOptions.COLLECTION) - val query: Option[String] = options.get(ArangoOptions.QUERY) - val readMode: ReadMode = - if (query.isDefined) ReadMode.Query - else if (collection.isDefined) ReadMode.Collection - else throw new IllegalArgumentException("Either collection or query must be defined") - val fillBlockCache: Option[Boolean] = options.get(ArangoOptions.FILL_BLOCK_CACHE).map(_.toBoolean) - val stream: Boolean = options.getOrElse(ArangoOptions.STREAM, "true").toBoolean - val parseMode: ParseMode = options.get(ArangoOptions.PARSE_MODE).map(ParseMode.fromString).getOrElse(PermissiveMode) - val columnNameOfCorruptRecord: String = options.getOrElse(ArangoOptions.CORRUPT_RECORDS_COLUMN, "") -} - -class ArangoWriteOptions(options: Map[String, String]) extends CommonOptions(options) { - val batchSize: Int = options.get(ArangoOptions.BATCH_SIZE).map(_.toInt).getOrElse(1000) - val collection: String = getRequired(ArangoOptions.COLLECTION) - val numberOfShards: Option[Int] = options.get(ArangoOptions.NUMBER_OF_SHARDS).map(_.toInt) - val collectionType: Option[CollectionType] = options.get(ArangoOptions.COLLECTION_TYPE).map(CollectionType(_)) - val waitForSync: Option[Boolean] = options.get(ArangoOptions.WAIT_FOR_SYNC).map(_.toBoolean) - val confirmTruncate: Boolean = options.getOrElse(ArangoOptions.CONFIRM_TRUNCATE, "false").toBoolean - val overwriteMode: Option[OverwriteMode] = options.get(ArangoOptions.OVERWRITE_MODE).map(OverwriteMode.valueOf) - val keepNull: Boolean = options.getOrElse(ArangoOptions.KEEP_NULL, "true").toBoolean - val mergeObjects: Option[Boolean] = options.get(ArangoOptions.MERGE_OBJECTS).map(_.toBoolean) -} +import com.arangodb.entity sealed trait ReadMode @@ -226,7 +52,7 @@ object ContentType { def apply(value: String): ContentType = value match { case JSON.name => JSON case VPACK.name => VPACK - case _ => throw new IllegalArgumentException(s"${ArangoOptions.CONTENT_TYPE}: $value") + case _ => throw new IllegalArgumentException(s"${ArangoDBConf.CONTENT_TYPE}: $value") } } @@ -246,7 +72,7 @@ object Protocol { def apply(value: String): Protocol = value match { case VST.name => VST case HTTP.name => HTTP - case _ => throw new IllegalArgumentException(s"${ArangoOptions.PROTOCOL}: $value") + case _ => throw new IllegalArgumentException(s"${ArangoDBConf.PROTOCOL}: $value") } } @@ -272,6 +98,6 @@ object CollectionType { def apply(value: String): CollectionType = value match { case DOCUMENT.name => DOCUMENT case EDGE.name => EDGE - case _ => throw new IllegalArgumentException(s"${ArangoOptions.COLLECTION_TYPE}: $value") + case _ => throw new IllegalArgumentException(s"${ArangoDBConf.COLLECTION_TYPE}: $value") } } \ No newline at end of file diff --git a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoUtils.scala b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoUtils.scala index b486b9e4..073520f5 100644 --- a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoUtils.scala +++ b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoUtils.scala @@ -8,7 +8,7 @@ import org.apache.spark.sql.{Encoders, SparkSession} */ object ArangoUtils { - def inferSchema(options: ArangoOptions): StructType = { + def inferSchema(options: ArangoDBConf): StructType = { val client = ArangoClient(options) val sampleEntries = options.readOptions.readMode match { case ReadMode.Query => client.readQuerySample() diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/DefaultSource.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/DefaultSource.scala index 3ef7be50..f3a28c47 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/DefaultSource.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/DefaultSource.scala @@ -1,7 +1,7 @@ package org.apache.spark.sql.arangodb.datasource import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoOptions, ArangoUtils} +import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoUtils} import org.apache.spark.sql.arangodb.datasource.reader.ArangoDataSourceReader import org.apache.spark.sql.arangodb.datasource.writer.ArangoDataSourceWriter import org.apache.spark.sql.sources.DataSourceRegister @@ -18,18 +18,18 @@ class DefaultSource extends DataSourceV2 with DataSourceRegister private var inferredSchema: StructType = _ - private def inferSchema(options: ArangoOptions): StructType = { + private def inferSchema(options: ArangoDBConf): StructType = { if (inferredSchema == null) { inferredSchema = ArangoUtils.inferSchema(options) } inferredSchema } - private def extractOptions(options: DataSourceOptions): ArangoOptions = { - val opts: ArangoOptions = ArangoOptions(options.asMap()) + private def extractOptions(options: DataSourceOptions): ArangoDBConf = { + val opts: ArangoDBConf = ArangoDBConf(options.asMap()) if (opts.driverOptions.acquireHostList) { val hosts = ArangoClient.acquireHostList(opts) - opts.updated(ArangoOptions.ENDPOINTS, hosts.mkString(",")) + opts.updated(ArangoDBConf.ENDPOINTS, hosts.mkString(",")) } else { opts } diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartition.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartition.scala index eff00b73..5a6173ad 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartition.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartition.scala @@ -1,6 +1,5 @@ package org.apache.spark.sql.arangodb.datasource.reader -import org.apache.spark.sql.arangodb.commons.ArangoOptions import org.apache.spark.sql.arangodb.commons.utils.PushDownCtx import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader} @@ -17,7 +16,7 @@ class ArangoCollectionPartition( val shardId: String, val endpoint: String, val ctx: PushDownCtx, - val options: ArangoOptions + val options: ArangoDBConf ) extends InputPartition[InternalRow] { override def createPartitionReader(): InputPartitionReader[InternalRow] = new ArangoCollectionPartitionReader(this, ctx, options) @@ -28,7 +27,7 @@ class ArangoCollectionPartition( */ class SingletonPartition( val schema: StructType, - val options: ArangoOptions + val options: ArangoDBConf ) extends InputPartition[InternalRow] { override def createPartitionReader(): InputPartitionReader[InternalRow] = new ArangoQueryReader(schema, options) diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala index f8a4f993..df156b43 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala @@ -4,7 +4,7 @@ import com.arangodb.entity.CursorEntity.Warning import org.apache.spark.internal.Logging import org.apache.spark.sql.arangodb.commons.mapping.ArangoParserProvider import org.apache.spark.sql.arangodb.commons.utils.PushDownCtx -import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoOptions, ContentType} +import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoDBConf, ContentType} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.FailureSafeParser import org.apache.spark.sql.sources.v2.reader.InputPartitionReader @@ -18,11 +18,11 @@ import scala.collection.JavaConverters.iterableAsScalaIterableConverter class ArangoCollectionPartitionReader( inputPartition: ArangoCollectionPartition, ctx: PushDownCtx, - opts: ArangoOptions) + opts: ArangoDBConf) extends InputPartitionReader[InternalRow] with Logging { // override endpoints with partition endpoint - private val options = opts.updated(ArangoOptions.ENDPOINTS, inputPartition.endpoint) + private val options = opts.updated(ArangoDBConf.ENDPOINTS, inputPartition.endpoint) private val actualSchema = StructType(ctx.requiredSchema.filterNot(_.name == options.readOptions.columnNameOfCorruptRecord)) private val parser = ArangoParserProvider().of(options.readOptions.contentType, actualSchema) private val safeParser = new FailureSafeParser[Array[Byte]]( diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoDataSourceReader.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoDataSourceReader.scala index fbd3ab9f..d91643cd 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoDataSourceReader.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoDataSourceReader.scala @@ -4,7 +4,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.arangodb.commons.filter.{FilterSupport, PushableFilter} import org.apache.spark.sql.arangodb.commons.utils.PushDownCtx -import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoOptions, ReadMode} +import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoDBConf, ReadMode} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, InputPartition, SupportsPushDownFilters, SupportsPushDownRequiredColumns} @@ -13,7 +13,7 @@ import org.apache.spark.sql.types.{StringType, StructType} import java.util import scala.collection.JavaConverters.seqAsJavaListConverter -class ArangoDataSourceReader(tableSchema: StructType, options: ArangoOptions) extends DataSourceReader +class ArangoDataSourceReader(tableSchema: StructType, options: ArangoDBConf) extends DataSourceReader with SupportsPushDownFilters with SupportsPushDownRequiredColumns with Logging { diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala index d403786f..f0af7f14 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala @@ -3,7 +3,7 @@ package org.apache.spark.sql.arangodb.datasource.reader import com.arangodb.entity.CursorEntity.Warning import org.apache.spark.internal.Logging import org.apache.spark.sql.arangodb.commons.mapping.ArangoParserProvider -import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoOptions, ContentType} +import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoDBConf, ContentType} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.FailureSafeParser import org.apache.spark.sql.sources.v2.reader.InputPartitionReader @@ -14,7 +14,7 @@ import scala.annotation.tailrec import scala.collection.JavaConverters.iterableAsScalaIterableConverter -class ArangoQueryReader(schema: StructType, options: ArangoOptions) extends InputPartitionReader[InternalRow] +class ArangoQueryReader(schema: StructType, options: ArangoDBConf) extends InputPartitionReader[InternalRow] with Logging { private val actualSchema = StructType(schema.filterNot(_.name == options.readOptions.columnNameOfCorruptRecord)) diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataSourceWriter.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataSourceWriter.scala index e5f74961..e249b47f 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataSourceWriter.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataSourceWriter.scala @@ -2,12 +2,12 @@ package org.apache.spark.sql.arangodb.datasource.writer import org.apache.spark.sql.arangodb.commons.exceptions.DataWriteAbortException import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoOptions} +import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoDBConf} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.sources.v2.writer.{DataSourceWriter, DataWriterFactory, WriterCommitMessage} import org.apache.spark.sql.types.StructType -class ArangoDataSourceWriter(writeUUID: String, schema: StructType, mode: SaveMode, options: ArangoOptions) extends DataSourceWriter { +class ArangoDataSourceWriter(writeUUID: String, schema: StructType, mode: SaveMode, options: ArangoDBConf) extends DataSourceWriter { private val client = ArangoClient(options) private var createdCollection = false @@ -16,7 +16,7 @@ class ArangoDataSourceWriter(writeUUID: String, schema: StructType, mode: SaveMo throw new AnalysisException( "You are attempting to use overwrite mode which will truncate this collection prior to inserting data. If " + "you just want to change data already in the collection set save mode 'append' and " + - "'overwrite.mode=(replace|update)'. To actually truncate set 'confirm.truncate=true'.") + s"'overwrite.mode=(replace|update)'. To actually truncate set '${ArangoDBConf.CONFIRM_TRUNCATE}=true'.") } if (client.collectionExists()) { diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala index 19c73e5b..cfe5a04f 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala @@ -4,7 +4,7 @@ import com.arangodb.model.OverwriteMode import com.arangodb.velocypack.{VPackParser, VPackSlice} import org.apache.spark.sql.arangodb.commons.exceptions.DataWriteAbortException import org.apache.spark.sql.arangodb.commons.mapping.{ArangoGenerator, ArangoGeneratorProvider} -import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoOptions, ContentType} +import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoDBConf, ContentType} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage} import org.apache.spark.sql.types.StructType @@ -12,7 +12,7 @@ import org.apache.spark.sql.types.StructType import java.io.ByteArrayOutputStream import scala.annotation.tailrec -class ArangoDataWriter(schema: StructType, options: ArangoOptions, partitionId: Int) extends DataWriter[InternalRow] { +class ArangoDataWriter(schema: StructType, options: ArangoDBConf, partitionId: Int) extends DataWriter[InternalRow] { private var failures = 0 private var endpointIdx = partitionId private val endpoints = Stream.continually(options.driverOptions.endpoints).flatten @@ -46,7 +46,7 @@ class ArangoDataWriter(schema: StructType, options: ArangoOptions, partitionId: "Task cannot be retried. To make batch writes idempotent, so that they can be retried, consider using " + "'keep.null=true' (default) and 'overwrite.mode=(ignore|replace|update)'.") - private def createClient() = ArangoClient(options.updated(ArangoOptions.ENDPOINTS, endpoints(endpointIdx))) + private def createClient() = ArangoClient(options.updated(ArangoDBConf.ENDPOINTS, endpoints(endpointIdx))) private def canRetry: Boolean = if (options.writeOptions.overwriteMode.isEmpty) false diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriterFactory.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriterFactory.scala index f9c82bab..e173f926 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriterFactory.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriterFactory.scala @@ -1,11 +1,10 @@ package org.apache.spark.sql.arangodb.datasource.writer -import org.apache.spark.sql.arangodb.commons.ArangoOptions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory} import org.apache.spark.sql.types.StructType -class ArangoDataWriterFactory(schema: StructType, options: ArangoOptions) extends DataWriterFactory[InternalRow] { +class ArangoDataWriterFactory(schema: StructType, options: ArangoDBConf) extends DataWriterFactory[InternalRow] { override def createDataWriter(partitionId: Int, taskId: Long, epochId: Long): DataWriter[InternalRow] = { new ArangoDataWriter(schema, options, partitionId) } diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/ArangoTable.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/ArangoTable.scala index fb21cd60..dc99a046 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/ArangoTable.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/ArangoTable.scala @@ -1,6 +1,6 @@ package org.apache.spark.sql.arangodb.datasource -import org.apache.spark.sql.arangodb.commons.{ArangoOptions, ArangoUtils} +import org.apache.spark.sql.arangodb.commons.{ArangoDBConf, ArangoUtils} import org.apache.spark.sql.arangodb.datasource.reader.ArangoScanBuilder import org.apache.spark.sql.arangodb.datasource.writer.ArangoWriterBuilder import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} @@ -12,7 +12,7 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap import java.util import scala.collection.JavaConverters.setAsJavaSetConverter -class ArangoTable(private var tableSchema: StructType, options: ArangoOptions) extends Table with SupportsRead with SupportsWrite { +class ArangoTable(private var tableSchema: StructType, options: ArangoDBConf) extends Table with SupportsRead with SupportsWrite { override def name(): String = this.getClass.toString @@ -34,8 +34,8 @@ class ArangoTable(private var tableSchema: StructType, options: ArangoOptions) e ).asJava override def newScanBuilder(scanOptions: CaseInsensitiveStringMap): ScanBuilder = - new ArangoScanBuilder(options.updated(ArangoOptions(scanOptions)), schema()) + new ArangoScanBuilder(options.updated(ArangoDBConf(scanOptions)), schema()) override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = - new ArangoWriterBuilder(info.schema(), options.updated(ArangoOptions(info.options()))) + new ArangoWriterBuilder(info.schema(), options.updated(ArangoDBConf(info.options()))) } diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/DefaultSource.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/DefaultSource.scala index c2c4c17d..ff4becf3 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/DefaultSource.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/DefaultSource.scala @@ -1,6 +1,6 @@ package org.apache.spark.sql.arangodb.datasource -import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoOptions} +import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoDBConf} import org.apache.spark.sql.connector.catalog.{Table, TableProvider} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.sources.DataSourceRegister @@ -13,11 +13,11 @@ class DefaultSource extends TableProvider with DataSourceRegister { private var table: ArangoTable = _ - private def extractOptions(options: util.Map[String, String]): ArangoOptions = { - val opts: ArangoOptions = ArangoOptions(options) + private def extractOptions(options: util.Map[String, String]): ArangoDBConf = { + val opts: ArangoDBConf = ArangoDBConf(options) if (opts.driverOptions.acquireHostList) { val hosts = ArangoClient.acquireHostList(opts) - opts.updated(ArangoOptions.ENDPOINTS, hosts.mkString(",")) + opts.updated(ArangoDBConf.ENDPOINTS, hosts.mkString(",")) } else { opts } diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala index 123d1321..2d1c7a03 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala @@ -4,7 +4,7 @@ import com.arangodb.entity.CursorEntity.Warning import org.apache.spark.internal.Logging import org.apache.spark.sql.arangodb.commons.mapping.ArangoParserProvider import org.apache.spark.sql.arangodb.commons.utils.PushDownCtx -import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoOptions, ContentType} +import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoDBConf, ContentType} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.FailureSafeParser import org.apache.spark.sql.connector.read.PartitionReader @@ -15,13 +15,13 @@ import scala.annotation.tailrec import scala.collection.JavaConverters.iterableAsScalaIterableConverter -class ArangoCollectionPartitionReader(inputPartition: ArangoCollectionPartition, ctx: PushDownCtx, opts: ArangoOptions) +class ArangoCollectionPartitionReader(inputPartition: ArangoCollectionPartition, ctx: PushDownCtx, opts: ArangoDBConf) extends PartitionReader[InternalRow] with Logging { // override endpoints with partition endpoint - private val options = opts.updated(ArangoOptions.ENDPOINTS, inputPartition.endpoint) + private val options = opts.updated(ArangoDBConf.ENDPOINTS, inputPartition.endpoint) private val actualSchema = StructType(ctx.requiredSchema.filterNot(_.name == options.readOptions.columnNameOfCorruptRecord)) - private val parser = ArangoParserProvider().of(options.readOptions.contentType, actualSchema) + private val parser = ArangoParserProvider().of(options.driverOptions.contentType, actualSchema) private val safeParser = new FailureSafeParser[Array[Byte]]( parser.parse, options.readOptions.parseMode, @@ -39,7 +39,7 @@ class ArangoCollectionPartitionReader(inputPartition: ArangoCollectionPartition, final override def next: Boolean = if (iterator.hasNext) { val current = iterator.next() - rowIterator = safeParser.parse(options.readOptions.contentType match { + rowIterator = safeParser.parse(options.driverOptions.contentType match { case ContentType.VPACK => current.toByteArray case ContentType.JSON => current.toString.getBytes(StandardCharsets.UTF_8) }) diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoPartitionReaderFactory.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoPartitionReaderFactory.scala index 08229903..10f70b35 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoPartitionReaderFactory.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoPartitionReaderFactory.scala @@ -1,11 +1,11 @@ package org.apache.spark.sql.arangodb.datasource.reader -import org.apache.spark.sql.arangodb.commons.ArangoOptions +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.arangodb.commons.utils.PushDownCtx import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, PartitionReaderFactory} -class ArangoPartitionReaderFactory(ctx: PushDownCtx, options: ArangoOptions) extends PartitionReaderFactory { +class ArangoPartitionReaderFactory(ctx: PushDownCtx, options: ArangoDBConf) extends PartitionReaderFactory { override def createReader(partition: InputPartition): PartitionReader[InternalRow] = partition match { case p: ArangoCollectionPartition => new ArangoCollectionPartitionReader(p, ctx, options) case SingletonPartition => new ArangoQueryReader(ctx.requiredSchema, options) diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala index fb16bce1..1d62b1d3 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala @@ -3,7 +3,7 @@ package org.apache.spark.sql.arangodb.datasource.reader import com.arangodb.entity.CursorEntity.Warning import org.apache.spark.internal.Logging import org.apache.spark.sql.arangodb.commons.mapping.ArangoParserProvider -import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoOptions, ContentType} +import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoDBConf, ContentType} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.FailureSafeParser import org.apache.spark.sql.connector.read.PartitionReader @@ -14,10 +14,10 @@ import scala.annotation.tailrec import scala.collection.JavaConverters.iterableAsScalaIterableConverter -class ArangoQueryReader(schema: StructType, options: ArangoOptions) extends PartitionReader[InternalRow] with Logging { +class ArangoQueryReader(schema: StructType, options: ArangoDBConf) extends PartitionReader[InternalRow] with Logging { private val actualSchema = StructType(schema.filterNot(_.name == options.readOptions.columnNameOfCorruptRecord)) - private val parser = ArangoParserProvider().of(options.readOptions.contentType, actualSchema) + private val parser = ArangoParserProvider().of(options.driverOptions.contentType, actualSchema) private val safeParser = new FailureSafeParser[Array[Byte]]( parser.parse, options.readOptions.parseMode, @@ -35,7 +35,7 @@ class ArangoQueryReader(schema: StructType, options: ArangoOptions) extends Part final override def next: Boolean = if (iterator.hasNext) { val current = iterator.next() - rowIterator = safeParser.parse(options.readOptions.contentType match { + rowIterator = safeParser.parse(options.driverOptions.contentType match { case ContentType.VPACK => current.toByteArray case ContentType.JSON => current.toString.getBytes(StandardCharsets.UTF_8) }) diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoScan.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoScan.scala index 211c89b5..3fe1e7fd 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoScan.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoScan.scala @@ -1,12 +1,12 @@ package org.apache.spark.sql.arangodb.datasource.reader -import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoOptions, ReadMode} +import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoDBConf, ReadMode} import org.apache.spark.sql.arangodb.commons.utils.PushDownCtx import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.connector.read.{Batch, InputPartition, PartitionReaderFactory, Scan} import org.apache.spark.sql.types.StructType -class ArangoScan(ctx: PushDownCtx, options: ArangoOptions) extends Scan with Batch { +class ArangoScan(ctx: PushDownCtx, options: ArangoDBConf) extends Scan with Batch { ExprUtils.verifyColumnNameOfCorruptRecord(ctx.requiredSchema, options.readOptions.columnNameOfCorruptRecord) override def readSchema(): StructType = ctx.requiredSchema diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoScanBuilder.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoScanBuilder.scala index 521e430f..fb4ea22b 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoScanBuilder.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoScanBuilder.scala @@ -1,14 +1,14 @@ package org.apache.spark.sql.arangodb.datasource.reader import org.apache.spark.internal.Logging -import org.apache.spark.sql.arangodb.commons.ArangoOptions +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.arangodb.commons.filter.{FilterSupport, PushableFilter} import org.apache.spark.sql.arangodb.commons.utils.PushDownCtx import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType -class ArangoScanBuilder(options: ArangoOptions, tableSchema: StructType) extends ScanBuilder +class ArangoScanBuilder(options: ArangoDBConf, tableSchema: StructType) extends ScanBuilder with SupportsPushDownFilters with SupportsPushDownRequiredColumns with Logging { diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoBatchWriter.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoBatchWriter.scala index ce1a2ea0..e4697516 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoBatchWriter.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoBatchWriter.scala @@ -1,12 +1,12 @@ package org.apache.spark.sql.arangodb.datasource.writer import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoOptions} +import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoDBConf} import org.apache.spark.sql.arangodb.commons.exceptions.DataWriteAbortException import org.apache.spark.sql.connector.write.{BatchWrite, DataWriterFactory, PhysicalWriteInfo, WriterCommitMessage} import org.apache.spark.sql.types.StructType -class ArangoBatchWriter(schema: StructType, options: ArangoOptions, mode: SaveMode) extends BatchWrite { +class ArangoBatchWriter(schema: StructType, options: ArangoDBConf, mode: SaveMode) extends BatchWrite { private val client = ArangoClient(options) override def createBatchWriterFactory(info: PhysicalWriteInfo): DataWriterFactory = diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala index 6d0616fb..8bf51ac6 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala @@ -4,7 +4,7 @@ import com.arangodb.model.OverwriteMode import com.arangodb.velocypack.{VPackParser, VPackSlice} import org.apache.spark.sql.arangodb.commons.exceptions.DataWriteAbortException import org.apache.spark.sql.arangodb.commons.mapping.{ArangoGenerator, ArangoGeneratorProvider} -import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoOptions, ContentType} +import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoDBConf, ContentType} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.write.{DataWriter, WriterCommitMessage} import org.apache.spark.sql.types.StructType @@ -12,7 +12,7 @@ import org.apache.spark.sql.types.StructType import java.io.ByteArrayOutputStream import scala.annotation.tailrec -class ArangoDataWriter(schema: StructType, options: ArangoOptions, partitionId: Int) extends DataWriter[InternalRow] { +class ArangoDataWriter(schema: StructType, options: ArangoDBConf, partitionId: Int) extends DataWriter[InternalRow] { private var failures = 0 private var endpointIdx = partitionId private val endpoints = Stream.continually(options.driverOptions.endpoints).flatten @@ -49,11 +49,10 @@ class ArangoDataWriter(schema: StructType, options: ArangoOptions, partitionId: client.shutdown() } - private def createClient() = ArangoClient(options.updated(ArangoOptions.ENDPOINTS, endpoints(endpointIdx))) + private def createClient() = ArangoClient(options.updated(ArangoDBConf.ENDPOINTS, endpoints(endpointIdx))) private def canRetry: Boolean = - if (options.writeOptions.overwriteMode.isEmpty) false - else options.writeOptions.overwriteMode.get match { + options.writeOptions.overwriteMode match { case OverwriteMode.ignore => true case OverwriteMode.replace => true case OverwriteMode.update => options.writeOptions.keepNull @@ -63,7 +62,7 @@ class ArangoDataWriter(schema: StructType, options: ArangoOptions, partitionId: private def initBatch(): Unit = { batchCount = 0 outVPack = new ByteArrayOutputStream() - vpackGenerator = ArangoGeneratorProvider().of(options.writeOptions.contentType, schema, outVPack) + vpackGenerator = ArangoGeneratorProvider().of(options.driverOptions.contentType, schema, outVPack) vpackGenerator.writeStartArray() } @@ -71,7 +70,7 @@ class ArangoDataWriter(schema: StructType, options: ArangoOptions, partitionId: vpackGenerator.writeEndArray() vpackGenerator.close() vpackGenerator.flush() - val payload = options.writeOptions.contentType match { + val payload = options.driverOptions.contentType match { case ContentType.VPACK => new VPackSlice(outVPack.toByteArray) case ContentType.JSON => new VPackParser.Builder().build().fromJson(new String(outVPack.toByteArray), true) } @@ -88,7 +87,7 @@ class ArangoDataWriter(schema: StructType, options: ArangoOptions, partitionId: client.shutdown() failures += 1 endpointIdx += 1 - if (canRetry && failures < options.driverOptions.endpoints.size) { + if (canRetry && failures < options.driverOptions.endpoints.length) { client = createClient() saveDocuments(payload) } else throw e diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriterFactory.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriterFactory.scala index 84ce302e..d4513fd7 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriterFactory.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriterFactory.scala @@ -1,11 +1,11 @@ package org.apache.spark.sql.arangodb.datasource.writer -import org.apache.spark.sql.arangodb.commons.ArangoOptions +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.connector.write.{DataWriter, DataWriterFactory} import org.apache.spark.sql.types.StructType -class ArangoDataWriterFactory(schema: StructType, options: ArangoOptions) extends DataWriterFactory { +class ArangoDataWriterFactory(schema: StructType, options: ArangoDBConf) extends DataWriterFactory { override def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = { new ArangoDataWriter(schema, options, partitionId) } diff --git a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoWriterBuilder.scala b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoWriterBuilder.scala index 56f87f58..e8de5a1e 100644 --- a/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoWriterBuilder.scala +++ b/arangodb-spark-datasource-3.1/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoWriterBuilder.scala @@ -1,11 +1,11 @@ package org.apache.spark.sql.arangodb.datasource.writer import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoOptions} +import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoDBConf} import org.apache.spark.sql.connector.write.{BatchWrite, SupportsTruncate, WriteBuilder} import org.apache.spark.sql.types.StructType -class ArangoWriterBuilder(schema: StructType, options: ArangoOptions) extends WriteBuilder with SupportsTruncate { +class ArangoWriterBuilder(schema: StructType, options: ArangoDBConf) extends WriteBuilder with SupportsTruncate { private var mode: SaveMode = SaveMode.Append @@ -31,7 +31,7 @@ class ArangoWriterBuilder(schema: StructType, options: ArangoOptions) extends Wr throw new AnalysisException( "You are attempting to use overwrite mode which will truncate this collection prior to inserting data. If " + "you just want to change data already in the collection set save mode 'append' and " + - "'overwrite.mode=(replace|update)'. To actually truncate set 'confirm.truncate=true'.") + s"'overwrite.mode=(replace|update)'. To actually truncate set '${ArangoDBConf.CONFIRM_TRUNCATE}=true'.") } } diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/AcquireHostListTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/AcquireHostListTest.scala index 88cbab69..3faef509 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/AcquireHostListTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/AcquireHostListTest.scala @@ -1,7 +1,7 @@ package org.apache.spark.sql.arangodb.datasource import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.arangodb.commons.ArangoOptions +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.junit.jupiter.api.{Disabled, Test} @Disabled("manual test only") @@ -18,10 +18,10 @@ class AcquireHostListTest { spark.read .format("org.apache.spark.sql.arangodb.datasource") .options(Map( - ArangoOptions.COLLECTION -> "_fishbowl", - ArangoOptions.ENDPOINTS -> "172.17.0.1:8529", - ArangoOptions.ACQUIRE_HOST_LIST -> "true", - ArangoOptions.PASSWORD -> "test" + ArangoDBConf.COLLECTION -> "_fishbowl", + ArangoDBConf.ENDPOINTS -> "172.17.0.1:8529", + ArangoDBConf.ACQUIRE_HOST_LIST -> "true", + ArangoDBConf.PASSWORD -> "test" )) .load() .show() diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/BadRecordsTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/BadRecordsTest.scala index c1e59cd0..be9d656a 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/BadRecordsTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/BadRecordsTest.scala @@ -2,7 +2,7 @@ package org.apache.spark.sql.arangodb.datasource import org.apache.spark.{SPARK_VERSION, SparkException} import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.arangodb.commons.ArangoOptions +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.catalyst.util.{BadRecordException, DropMalformedMode, FailFastMode, ParseMode} import org.apache.spark.sql.types._ import org.assertj.core.api.Assertions.{assertThat, catchThrowable} @@ -74,7 +74,7 @@ class BadRecordsTest extends BaseSparkTest { contentType: String ) = { // PERMISSIVE - doTestBadRecord(schema, data, jsonData, Map(ArangoOptions.CONTENT_TYPE -> contentType)) + doTestBadRecord(schema, data, jsonData, Map(ArangoDBConf.CONTENT_TYPE -> contentType)) // PERMISSIVE with columnNameOfCorruptRecord doTestBadRecord( @@ -82,23 +82,23 @@ class BadRecordsTest extends BaseSparkTest { data, jsonData, Map( - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.CORRUPT_RECORDS_COLUMN -> "corruptRecord" + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.COLUMN_NAME_OF_CORRUPT_RECORD -> "corruptRecord" ) ) // DROPMALFORMED doTestBadRecord(schema, data, jsonData, Map( - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.PARSE_MODE -> DropMalformedMode.name + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.PARSE_MODE -> DropMalformedMode.name ) ) // FAILFAST val df = BaseSparkTest.createDF(collectionName, data, schema, Map( - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.PARSE_MODE -> FailFastMode.name + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.PARSE_MODE -> FailFastMode.name )) val thrown = catchThrowable(new ThrowingCallable() { override def call(): Unit = df.collect() diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/BaseSparkTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/BaseSparkTest.scala index 70bc2d92..6a0962d2 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/BaseSparkTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/BaseSparkTest.scala @@ -8,7 +8,7 @@ import com.fasterxml.jackson.core.JsonGenerator import com.fasterxml.jackson.databind.module.SimpleModule import com.fasterxml.jackson.databind.{JsonSerializer, ObjectMapper, SerializerProvider} import com.fasterxml.jackson.module.scala.DefaultScalaModule -import org.apache.spark.sql.arangodb.commons.ArangoOptions +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.types._ import org.apache.spark.sql.{DataFrame, SparkSession} import org.junit.jupiter.api.{AfterEach, BeforeAll} @@ -166,7 +166,7 @@ object BaseSparkTest { val df = spark.read .format(arangoDatasource) - .options(options ++ additionalOptions + (ArangoOptions.COLLECTION -> name)) + .options(options ++ additionalOptions + (ArangoDBConf.COLLECTION -> name)) .schema(schema) .load() df.createOrReplaceTempView(name) @@ -176,7 +176,7 @@ object BaseSparkTest { def createQueryDF(query: String, schema: StructType, additionalOptions: Map[String, String] = Map.empty): DataFrame = spark.read .format(arangoDatasource) - .options(options ++ additionalOptions + (ArangoOptions.QUERY -> query)) + .options(options ++ additionalOptions + (ArangoDBConf.QUERY -> query)) .schema(schema) .load() diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/DeserializationCastTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/DeserializationCastTest.scala index 65a7e2fb..926dc178 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/DeserializationCastTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/DeserializationCastTest.scala @@ -1,7 +1,7 @@ package org.apache.spark.sql.arangodb.datasource import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.arangodb.commons.ArangoOptions +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.types.{BooleanType, DoubleType, IntegerType, StringType, StructField, StructType} import org.assertj.core.api.Assertions.assertThat import org.junit.jupiter.api.Disabled @@ -88,15 +88,15 @@ class DeserializationCastTest extends BaseSparkTest { ) private def doTestImplicitCast( - schema: StructType, - data: Iterable[Map[String, Any]], - jsonData: Seq[String], - contentType: String - ) = { + schema: StructType, + data: Iterable[Map[String, Any]], + jsonData: Seq[String], + contentType: String + ) = { import spark.implicits._ val dfFromJson: DataFrame = spark.read.schema(schema).json(jsonData.toDS) dfFromJson.show() - val df = BaseSparkTest.createDF(collectionName, data, schema, Map(ArangoOptions.CONTENT_TYPE -> contentType)) + val df = BaseSparkTest.createDF(collectionName, data, schema, Map(ArangoDBConf.CONTENT_TYPE -> contentType)) assertThat(df.collect()).isEqualTo(dfFromJson.collect()) } } diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/ReadTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/ReadTest.scala index 40021fe0..0b107391 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/ReadTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/ReadTest.scala @@ -1,7 +1,7 @@ package org.apache.spark.sql.arangodb.datasource import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.arangodb.commons.ArangoOptions +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.functions.col import org.apache.spark.sql.types.{NumericType, StringType, StructField, StructType} import org.assertj.core.api.Assertions.assertThat @@ -18,9 +18,9 @@ class ReadTest extends BaseSparkTest { val df = spark.read .format(BaseSparkTest.arangoDatasource) .options(options + ( - ArangoOptions.COLLECTION -> "users", - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType + ArangoDBConf.COLLECTION -> "users", + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType )) .schema(BaseSparkTest.usersSchema) .load() @@ -60,9 +60,9 @@ class ReadTest extends BaseSparkTest { val usersDF = spark.read .format(BaseSparkTest.arangoDatasource) .options(options + ( - ArangoOptions.COLLECTION -> "users", - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType + ArangoDBConf.COLLECTION -> "users", + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType )) .load() @@ -91,9 +91,9 @@ class ReadTest extends BaseSparkTest { assumeTrue(isSingle) val additionalOptions = Map( - ArangoOptions.CORRUPT_RECORDS_COLUMN -> "badRecord", - ArangoOptions.SAMPLE_SIZE -> "2", - ArangoOptions.CONTENT_TYPE -> contentType + ArangoDBConf.COLUMN_NAME_OF_CORRUPT_RECORD -> "badRecord", + ArangoDBConf.SAMPLE_SIZE -> "2", + ArangoDBConf.CONTENT_TYPE -> contentType ) doInferCollectionSchemaWithCorruptRecordColumn( @@ -153,9 +153,9 @@ class ReadTest extends BaseSparkTest { val df = spark.read .format(BaseSparkTest.arangoDatasource) .options(options + ( - ArangoOptions.QUERY -> query, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType + ArangoDBConf.QUERY -> query, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType )) .load() diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/ReadWriteDataTypeTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/ReadWriteDataTypeTest.scala index 60a82eb8..c86b1baf 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/ReadWriteDataTypeTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/ReadWriteDataTypeTest.scala @@ -1,11 +1,11 @@ package org.apache.spark.sql.arangodb.datasource -import org.apache.spark.sql.arangodb.commons.ArangoOptions +import com.arangodb.model.OverwriteMode +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.arangodb.datasource.BaseSparkTest.arangoDatasource import org.apache.spark.sql.types._ import org.apache.spark.sql.{DataFrame, SaveMode} import org.assertj.core.api.Assertions.assertThat -import org.junit.jupiter.api.Assumptions.assumeTrue import org.junit.jupiter.api.{AfterAll, BeforeAll} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.MethodSource @@ -25,11 +25,11 @@ class ReadWriteDataTypeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Overwrite) .options(options + ( - ArangoOptions.COLLECTION -> (ReadWriteDataTypeTest.collectionName + "_2"), - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.OVERWRITE_MODE -> "replace", - ArangoOptions.CONFIRM_TRUNCATE -> "true" + ArangoDBConf.COLLECTION -> (ReadWriteDataTypeTest.collectionName + "_2"), + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.OVERWRITE_MODE -> OverwriteMode.replace.getValue, + ArangoDBConf.CONFIRM_TRUNCATE -> "true" )) .save() diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/AbortTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/AbortTest.scala index 0f298968..dd91f8b5 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/AbortTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/AbortTest.scala @@ -1,8 +1,9 @@ package org.apache.spark.sql.arangodb.datasource.write import com.arangodb.ArangoCollection +import com.arangodb.model.OverwriteMode import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.arangodb.commons.ArangoOptions +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.arangodb.commons.exceptions.{ArangoDBMultiException, DataWriteAbortException} import org.apache.spark.sql.arangodb.datasource.BaseSparkTest import org.apache.spark.{SPARK_VERSION, SparkException} @@ -56,10 +57,10 @@ class AbortTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Append) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.OVERWRITE_MODE -> "replace" + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.OVERWRITE_MODE -> OverwriteMode.replace.getValue )) .save() }) @@ -83,11 +84,11 @@ class AbortTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Overwrite) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.CONFIRM_TRUNCATE -> "true", - ArangoOptions.OVERWRITE_MODE -> "replace" + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.CONFIRM_TRUNCATE -> "true", + ArangoDBConf.OVERWRITE_MODE -> OverwriteMode.replace.getValue )) .save() }) @@ -113,10 +114,10 @@ class AbortTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.ErrorIfExists) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.OVERWRITE_MODE -> "replace" + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.OVERWRITE_MODE -> OverwriteMode.replace.getValue )) .save() }) @@ -142,10 +143,10 @@ class AbortTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Ignore) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.OVERWRITE_MODE -> "replace" + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.OVERWRITE_MODE -> OverwriteMode.replace.getValue )) .save() }) diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/CreateCollectionTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/CreateCollectionTest.scala index 48f56a21..b9172adf 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/CreateCollectionTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/CreateCollectionTest.scala @@ -3,7 +3,7 @@ package org.apache.spark.sql.arangodb.datasource.write import com.arangodb.ArangoCollection import com.arangodb.entity.CollectionType import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.arangodb.commons.ArangoOptions +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.arangodb.datasource.BaseSparkTest import org.assertj.core.api.Assertions.assertThat import org.junit.jupiter.api.BeforeEach @@ -42,11 +42,11 @@ class CreateCollectionTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Append) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.NUMBER_OF_SHARDS -> "5", - ArangoOptions.COLLECTION_TYPE -> "edge" + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.NUMBER_OF_SHARDS -> "5", + ArangoDBConf.COLLECTION_TYPE -> org.apache.spark.sql.arangodb.commons.CollectionType.EDGE.name )) .save() @@ -63,12 +63,12 @@ class CreateCollectionTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Overwrite) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.CONFIRM_TRUNCATE -> "true", - ArangoOptions.NUMBER_OF_SHARDS -> "5", - ArangoOptions.COLLECTION_TYPE -> "edge" + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.CONFIRM_TRUNCATE -> "true", + ArangoDBConf.NUMBER_OF_SHARDS -> "5", + ArangoDBConf.COLLECTION_TYPE -> org.apache.spark.sql.arangodb.commons.CollectionType.EDGE.name )) .save() diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/OverwriteModeTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/OverwriteModeTest.scala index a393c3ca..c8945df8 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/OverwriteModeTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/OverwriteModeTest.scala @@ -2,9 +2,10 @@ package org.apache.spark.sql.arangodb.datasource.write import com.arangodb.ArangoCollection import com.arangodb.entity.BaseDocument +import com.arangodb.model.OverwriteMode import org.apache.spark.SparkException import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.arangodb.commons.ArangoOptions +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.arangodb.commons.exceptions.ArangoDBMultiException import org.apache.spark.sql.arangodb.datasource.BaseSparkTest import org.assertj.core.api.Assertions.{assertThat, catchThrowable} @@ -54,10 +55,10 @@ class OverwriteModeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Append) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.OVERWRITE_MODE -> "conflict" + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.OVERWRITE_MODE -> OverwriteMode.conflict.getValue )) .save() }) @@ -80,10 +81,10 @@ class OverwriteModeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Append) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.OVERWRITE_MODE -> "ignore" + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.OVERWRITE_MODE -> OverwriteMode.ignore.getValue )) .save() @@ -104,10 +105,10 @@ class OverwriteModeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Append) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.OVERWRITE_MODE -> "replace" + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.OVERWRITE_MODE -> OverwriteMode.replace.getValue )) .save() @@ -128,10 +129,10 @@ class OverwriteModeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Append) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.OVERWRITE_MODE -> "update" + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.OVERWRITE_MODE -> OverwriteMode.update.getValue )) .save() @@ -154,10 +155,10 @@ class OverwriteModeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Append) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.OVERWRITE_MODE -> "update" + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.OVERWRITE_MODE -> OverwriteMode.update.getValue )) .save() diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/SaveModeTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/SaveModeTest.scala index b936bbcd..b0ecee19 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/SaveModeTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/SaveModeTest.scala @@ -2,7 +2,7 @@ package org.apache.spark.sql.arangodb.datasource.write import com.arangodb.ArangoCollection import org.apache.spark.SPARK_VERSION -import org.apache.spark.sql.arangodb.commons.ArangoOptions +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.arangodb.datasource.BaseSparkTest import org.apache.spark.sql.{AnalysisException, SaveMode} import org.assertj.core.api.Assertions.{assertThat, catchThrowable} @@ -48,9 +48,9 @@ class SaveModeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Append) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType )) .save() @@ -66,9 +66,9 @@ class SaveModeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Append) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType )) .save() @@ -83,15 +83,15 @@ class SaveModeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Overwrite) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType )) .save() }) assertThat(thrown).isInstanceOf(classOf[AnalysisException]) - assertThat(thrown.getMessage).contains(ArangoOptions.CONFIRM_TRUNCATE) + assertThat(thrown.getMessage).contains(ArangoDBConf.CONFIRM_TRUNCATE) } @ParameterizedTest @@ -101,10 +101,10 @@ class SaveModeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Overwrite) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.CONFIRM_TRUNCATE -> "true" + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.CONFIRM_TRUNCATE -> "true" )) .save() @@ -121,10 +121,10 @@ class SaveModeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Overwrite) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.CONFIRM_TRUNCATE -> "true" + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.CONFIRM_TRUNCATE -> "true" )) .save() @@ -142,9 +142,9 @@ class SaveModeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.ErrorIfExists) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType )) .save() }) @@ -162,9 +162,9 @@ class SaveModeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.ErrorIfExists) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType )) .save() @@ -180,9 +180,9 @@ class SaveModeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Ignore) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType )) .save() @@ -200,9 +200,9 @@ class SaveModeTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Ignore) .options(options + ( - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType )) .save() diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/WriteResiliencyTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/WriteResiliencyTest.scala index 77ff43cd..ca7243bc 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/WriteResiliencyTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/WriteResiliencyTest.scala @@ -1,8 +1,9 @@ package org.apache.spark.sql.arangodb.datasource.write import com.arangodb.ArangoCollection +import com.arangodb.model.OverwriteMode import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.arangodb.commons.ArangoOptions +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.arangodb.datasource.BaseSparkTest import org.assertj.core.api.Assertions.assertThat import org.junit.jupiter.api.BeforeEach @@ -45,12 +46,12 @@ class WriteResiliencyTest extends BaseSparkTest { .format(BaseSparkTest.arangoDatasource) .mode(SaveMode.Overwrite) .options(options + ( - ArangoOptions.ENDPOINTS -> (BaseSparkTest.endpoints + ",wrongHost:8529"), - ArangoOptions.COLLECTION -> collectionName, - ArangoOptions.PROTOCOL -> protocol, - ArangoOptions.CONTENT_TYPE -> contentType, - ArangoOptions.CONFIRM_TRUNCATE -> "true", - ArangoOptions.OVERWRITE_MODE -> "replace" + ArangoDBConf.ENDPOINTS -> (BaseSparkTest.endpoints + ",wrongHost:8529"), + ArangoDBConf.COLLECTION -> collectionName, + ArangoDBConf.PROTOCOL -> protocol, + ArangoDBConf.CONTENT_TYPE -> contentType, + ArangoDBConf.CONFIRM_TRUNCATE -> "true", + ArangoDBConf.OVERWRITE_MODE -> OverwriteMode.replace.getValue )) .save() From fb79fc5005b6a891caddc14fb070ceb889835d33 Mon Sep 17 00:00:00 2001 From: Michele Rastelli Date: Fri, 10 Dec 2021 10:57:11 +0100 Subject: [PATCH 4/8] enabled DeserializationCastTest json tests --- .../sql/arangodb/datasource/BaseSparkTest.scala | 7 ------- .../datasource/DeserializationCastTest.scala | 12 +++++++----- .../spark/sql/arangodb/datasource/SslTest.scala | 9 +-------- 3 files changed, 8 insertions(+), 20 deletions(-) diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/BaseSparkTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/BaseSparkTest.scala index 6a0962d2..7a7a48ac 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/BaseSparkTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/BaseSparkTest.scala @@ -87,13 +87,6 @@ object BaseSparkTest { } else { endpoints } - }, - "topology" -> { - if (isSingle) { - "single" - } else { - "cluster" - } } ) diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/DeserializationCastTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/DeserializationCastTest.scala index 926dc178..24fc145e 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/DeserializationCastTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/DeserializationCastTest.scala @@ -4,14 +4,10 @@ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.types.{BooleanType, DoubleType, IntegerType, StringType, StructField, StructType} import org.assertj.core.api.Assertions.assertThat -import org.junit.jupiter.api.Disabled +import org.junit.jupiter.api.Assumptions.assumeTrue import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource -/** - * FIXME: many vpack tests fail - */ -@Disabled class DeserializationCastTest extends BaseSparkTest { private val collectionName = "deserializationCast" @@ -93,6 +89,12 @@ class DeserializationCastTest extends BaseSparkTest { jsonData: Seq[String], contentType: String ) = { + + /** + * FIXME: many vpack tests fail + */ + assumeTrue(contentType != "vpack") + import spark.implicits._ val dfFromJson: DataFrame = spark.read.schema(schema).json(jsonData.toDS) dfFromJson.show() diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/SslTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/SslTest.scala index 845a7e68..704f54af 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/SslTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/SslTest.scala @@ -125,14 +125,7 @@ object SslTest { "database" -> database, "user" -> user, "password" -> password, - "endpoints" -> endpoints, - "topology" -> { - if (isSingle) { - "single" - } else { - "cluster" - } - } + "endpoints" -> endpoints ) private val spark: SparkSession = SparkSession.builder() From 4a6d747a97b46f3f3d4bf5c3c2b3f90fb7623ae2 Mon Sep 17 00:00:00 2001 From: Michele Rastelli Date: Fri, 10 Dec 2021 11:26:55 +0100 Subject: [PATCH 5/8] Spark 2.4: ArangoDBConf refactoring --- .../sql/arangodb/commons/ArangoDBConf.scala | 33 ++----------------- .../arangodb/datasource/DefaultSource.scala | 2 +- .../mapping/ArangoGeneratorImpl.scala | 10 +++--- .../datasource/mapping/ArangoParserImpl.scala | 10 +++--- .../reader/ArangoCollectionPartition.scala | 1 + .../ArangoCollectionPartitionReader.scala | 8 ++--- .../datasource/reader/ArangoQueryReader.scala | 8 ++--- .../datasource/writer/ArangoDataWriter.scala | 13 ++++---- .../writer/ArangoDataWriterFactory.scala | 1 + 9 files changed, 29 insertions(+), 57 deletions(-) diff --git a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala index cb037732..554c31cb 100644 --- a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala +++ b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala @@ -20,35 +20,30 @@ object ArangoDBConf { val USER = "user" val userConf: ConfigEntry[String] = ConfigBuilder(USER) .doc("db user") - .version("1.0.0") .stringConf .createWithDefault("root") val PASSWORD = "password" val passwordConf: OptionalConfigEntry[String] = ConfigBuilder(PASSWORD) .doc("db password") - .version("1.0.0") .stringConf .createOptional val ENDPOINTS = "endpoints" val endpointsConf: OptionalConfigEntry[String] = ConfigBuilder(ENDPOINTS) .doc("A comma-separated list of coordinators, eg. c1:8529,c2:8529") - .version("1.0.0") .stringConf .createOptional val ACQUIRE_HOST_LIST = "acquireHostList" val acquireHostListConf: ConfigEntry[Boolean] = ConfigBuilder(ACQUIRE_HOST_LIST) .doc("acquire the list of all known hosts in the cluster") - .version("1.0.0") .booleanConf .createWithDefault(false) val PROTOCOL = "protocol" val protocolConf: ConfigEntry[String] = ConfigBuilder(PROTOCOL) .doc("communication protocol") - .version("1.0.0") .stringConf .checkValues(Set(Protocol.HTTP.name, Protocol.VST.name)) .createWithDefault(Protocol.HTTP.name) @@ -56,7 +51,6 @@ object ArangoDBConf { val CONTENT_TYPE = "contentType" val contentTypeConf: ConfigEntry[String] = ConfigBuilder(CONTENT_TYPE) .doc("content type for driver communication") - .version("1.0.0") .stringConf .checkValues(Set(ContentType.VPACK.name, ContentType.JSON.name)) .createWithDefault(ContentType.JSON.name) @@ -64,105 +58,90 @@ object ArangoDBConf { val SSL_ENABLED = "ssl.enabled" val sslEnabledConf: ConfigEntry[Boolean] = ConfigBuilder(SSL_ENABLED) .doc("SSL secured driver connection") - .version("1.0.0") .booleanConf .createWithDefault(false) val SSL_CERT_VALUE = "ssl.cert.value" val sslCertValueConf: OptionalConfigEntry[String] = ConfigBuilder(SSL_CERT_VALUE) .doc("base64 encoded certificate") - .version("1.0.0") .stringConf .createOptional val SSL_CERT_TYPE = "ssl.cert.type" val sslCertTypeConf: ConfigEntry[String] = ConfigBuilder(SSL_CERT_TYPE) .doc("certificate type") - .version("1.0.0") .stringConf .createWithDefault("X.509") val SSL_CERT_ALIAS = "ssl.cert.alias" val sslCertAliasConf: ConfigEntry[String] = ConfigBuilder(SSL_CERT_ALIAS) .doc("certificate alias name") - .version("1.0.0") .stringConf .createWithDefault("arangodb") val SSL_ALGORITHM = "ssl.algorithm" val sslAlgorithmConf: ConfigEntry[String] = ConfigBuilder(SSL_ALGORITHM) .doc("trust manager algorithm") - .version("1.0.0") .stringConf .createWithDefault("SunX509") val SSL_KEYSTORE_TYPE = "ssl.keystore.type" val sslKeystoreTypeConf: ConfigEntry[String] = ConfigBuilder(SSL_KEYSTORE_TYPE) .doc("keystore type") - .version("1.0.0") .stringConf .createWithDefault("jks") val SSL_PROTOCOL = "ssl.protocol" val sslProtocolConf: ConfigEntry[String] = ConfigBuilder(SSL_PROTOCOL) .doc("SSLContext protocol") - .version("1.0.0") .stringConf .createWithDefault("TLS") val DB = "database" val dbConf: ConfigEntry[String] = ConfigBuilder(DB) .doc("database name") - .version("1.0.0") .stringConf .createWithDefault("_system") val COLLECTION = "table" val collectionConf: OptionalConfigEntry[String] = ConfigBuilder(COLLECTION) .doc("ArangoDB collection name") - .version("1.0.0") .stringConf .createOptional val BATCH_SIZE = "batchSize" val batchSizeConf: ConfigEntry[Int] = ConfigBuilder(BATCH_SIZE) .doc("batch size") - .version("1.0.0") .intConf .createWithDefault(10000) val QUERY = "query" val queryConf: OptionalConfigEntry[String] = ConfigBuilder(QUERY) .doc("custom AQL read query") - .version("1.0.0") .stringConf .createOptional val SAMPLE_SIZE = "sampleSize" val sampleSizeConf: ConfigEntry[Int] = ConfigBuilder(SAMPLE_SIZE) .doc("sample size prefetched for schema inference") - .version("1.0.0") .intConf .createWithDefault(1000) val FILL_BLOCK_CACHE = "fillBlockCache" val fillBlockCacheConf: ConfigEntry[Boolean] = ConfigBuilder(FILL_BLOCK_CACHE) .doc("whether the query should store the data it reads in the RocksDB block cache") - .version("1.0.0") .booleanConf .createWithDefault(false) val STREAM = "stream" val streamConf: ConfigEntry[Boolean] = ConfigBuilder(STREAM) .doc("whether the query should be executed lazily") - .version("1.0.0") .booleanConf .createWithDefault(true) val PARSE_MODE = "mode" val parseModeConf: ConfigEntry[String] = ConfigBuilder(PARSE_MODE) .doc("allows a mode for dealing with corrupt records during parsing") - .version("1.0.0") .stringConf .checkValues(Set(PermissiveMode.name, DropMalformedMode.name, FailFastMode.name)) .createWithDefault(PermissiveMode.name) @@ -170,21 +149,18 @@ object ArangoDBConf { val COLUMN_NAME_OF_CORRUPT_RECORD = "columnNameOfCorruptRecord" val columnNameOfCorruptRecordConf: OptionalConfigEntry[String] = ConfigBuilder(COLUMN_NAME_OF_CORRUPT_RECORD) .doc("allows renaming the new field having malformed string created by PERMISSIVE mode") - .version("1.0.0") .stringConf .createOptional val NUMBER_OF_SHARDS = "table.shards" val numberOfShardsConf: ConfigEntry[Int] = ConfigBuilder(NUMBER_OF_SHARDS) .doc("number of shards of the created collection (in case of SaveMode Append or Overwrite)") - .version("1.0.0") .intConf .createWithDefault(1) val COLLECTION_TYPE = "table.type" val collectionTypeConf: ConfigEntry[String] = ConfigBuilder(COLLECTION_TYPE) .doc("type of the created collection (in case of SaveMode Append or Overwrite)") - .version("1.0.0") .stringConf .checkValues(Set(CollectionType.DOCUMENT.name, CollectionType.EDGE.name)) .createWithDefault(CollectionType.DOCUMENT.name) @@ -192,21 +168,18 @@ object ArangoDBConf { val WAIT_FOR_SYNC = "waitForSync" val waitForSyncConf: ConfigEntry[Boolean] = ConfigBuilder(WAIT_FOR_SYNC) .doc("whether to wait until the documents have been synced to disk") - .version("1.0.0") .booleanConf .createWithDefault(false) val CONFIRM_TRUNCATE = "confirmTruncate" val confirmTruncateConf: ConfigEntry[Boolean] = ConfigBuilder(CONFIRM_TRUNCATE) .doc("confirm to truncate table when using SaveMode.Overwrite mode") - .version("1.0.0") .booleanConf .createWithDefault(false) val OVERWRITE_MODE = "overwriteMode" val overwriteModeConf: ConfigEntry[String] = ConfigBuilder(OVERWRITE_MODE) .doc("configures the behavior in case a document with the specified _key value exists already") - .version("1.0.0") .stringConf .checkValues(Set( OverwriteMode.ignore.getValue, @@ -219,14 +192,12 @@ object ArangoDBConf { val MERGE_OBJECTS = "mergeObjects" val mergeObjectsConf: ConfigEntry[Boolean] = ConfigBuilder(MERGE_OBJECTS) .doc("in case overwrite.mode is set to update, controls whether objects (not arrays) will be merged") - .version("1.0.0") .booleanConf .createWithDefault(true) val KEEP_NULL = "keepNull" val keepNullConf: ConfigEntry[Boolean] = ConfigBuilder(KEEP_NULL) .doc("whether null values are saved within the document or used to delete corresponding existing attributes") - .version("1.0.0") .booleanConf .createWithDefault(true) @@ -379,10 +350,10 @@ class ArangoDBConf(opts: Map[String, String]) extends Serializable with Logging * Return all the configuration definitions that have been defined in [[ArangoDBConf]]. Each * definition contains key, defaultValue and doc. */ - def getAllDefinedConfigs: Seq[(String, String, String, String)] = + def getAllDefinedConfigs: Seq[(String, String, String)] = confEntries.values.filter(_.isPublic).map { entry => val displayValue = Option(getConfString(entry.key, null)).getOrElse(entry.defaultValueString) - (entry.key, displayValue, entry.doc, entry.version) + (entry.key, displayValue, entry.doc) }.toSeq /** diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/DefaultSource.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/DefaultSource.scala index f3a28c47..46891bd2 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/DefaultSource.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/DefaultSource.scala @@ -1,7 +1,7 @@ package org.apache.spark.sql.arangodb.datasource import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoUtils} +import org.apache.spark.sql.arangodb.commons.{ArangoClient, ArangoDBConf, ArangoUtils} import org.apache.spark.sql.arangodb.datasource.reader.ArangoDataSourceReader import org.apache.spark.sql.arangodb.datasource.writer.ArangoDataSourceWriter import org.apache.spark.sql.sources.DataSourceRegister diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoGeneratorImpl.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoGeneratorImpl.scala index 468c8ef2..6ac8f914 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoGeneratorImpl.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoGeneratorImpl.scala @@ -10,9 +10,9 @@ import org.apache.spark.sql.arangodb.commons.mapping.{ArangoGenerator, ArangoGen import java.io.OutputStream abstract sealed class ArangoGeneratorImpl( - schema: DataType, - writer: OutputStream, - options: JSONOptions) + schema: DataType, + writer: OutputStream, + options: JSONOptions) extends JacksonGenerator( schema, options.buildJsonFactory().createGenerator(writer), @@ -20,8 +20,8 @@ abstract sealed class ArangoGeneratorImpl( class ArangoGeneratorProviderImpl extends ArangoGeneratorProvider { override def of(contentType: ContentType, schema: StructType, outputStream: OutputStream): ArangoGeneratorImpl = contentType match { - case ContentType.Json => new JsonArangoGenerator(schema, outputStream) - case ContentType.VPack => new VPackArangoGenerator(schema, outputStream) + case ContentType.JSON => new JsonArangoGenerator(schema, outputStream) + case ContentType.VPACK => new VPackArangoGenerator(schema, outputStream) case _ => throw new IllegalArgumentException } } diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoParserImpl.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoParserImpl.scala index d2fe44d0..68e91655 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoParserImpl.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/mapping/ArangoParserImpl.scala @@ -11,9 +11,9 @@ import org.apache.spark.sql.types.DataType import org.apache.spark.unsafe.types.UTF8String abstract sealed class ArangoParserImpl( - schema: DataType, - options: JSONOptions, - recordLiteral: Array[Byte] => UTF8String) + schema: DataType, + options: JSONOptions, + recordLiteral: Array[Byte] => UTF8String) extends JacksonParser(schema, options) with ArangoParser { def parse(data: Array[Byte]): Seq[InternalRow] = super.parse( data, @@ -24,8 +24,8 @@ abstract sealed class ArangoParserImpl( class ArangoParserProviderImpl extends ArangoParserProvider { override def of(contentType: ContentType, schema: DataType): ArangoParserImpl = contentType match { - case ContentType.Json => new JsonArangoParser(schema) - case ContentType.VPack => new VPackArangoParser(schema) + case ContentType.JSON => new JsonArangoParser(schema) + case ContentType.VPACK => new VPackArangoParser(schema) case _ => throw new IllegalArgumentException } } diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartition.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartition.scala index 5a6173ad..b5489032 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartition.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartition.scala @@ -1,5 +1,6 @@ package org.apache.spark.sql.arangodb.datasource.reader +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.arangodb.commons.utils.PushDownCtx import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader} diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala index df156b43..4482a97f 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoCollectionPartitionReader.scala @@ -24,7 +24,7 @@ class ArangoCollectionPartitionReader( // override endpoints with partition endpoint private val options = opts.updated(ArangoDBConf.ENDPOINTS, inputPartition.endpoint) private val actualSchema = StructType(ctx.requiredSchema.filterNot(_.name == options.readOptions.columnNameOfCorruptRecord)) - private val parser = ArangoParserProvider().of(options.readOptions.contentType, actualSchema) + private val parser = ArangoParserProvider().of(options.driverOptions.contentType, actualSchema) private val safeParser = new FailureSafeParser[Array[Byte]]( parser.parse(_).toSeq, options.readOptions.parseMode, @@ -42,9 +42,9 @@ class ArangoCollectionPartitionReader( final override def next: Boolean = if (iterator.hasNext) { val current = iterator.next() - rowIterator = safeParser.parse(options.readOptions.contentType match { - case ContentType.VPack => current.toByteArray - case ContentType.Json => current.toString.getBytes(StandardCharsets.UTF_8) + rowIterator = safeParser.parse(options.driverOptions.contentType match { + case ContentType.VPACK => current.toByteArray + case ContentType.JSON => current.toString.getBytes(StandardCharsets.UTF_8) }) if (rowIterator.hasNext) true else next diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala index f0af7f14..7f3a1948 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/reader/ArangoQueryReader.scala @@ -18,7 +18,7 @@ class ArangoQueryReader(schema: StructType, options: ArangoDBConf) extends Input with Logging { private val actualSchema = StructType(schema.filterNot(_.name == options.readOptions.columnNameOfCorruptRecord)) - private val parser = ArangoParserProvider().of(options.readOptions.contentType, actualSchema) + private val parser = ArangoParserProvider().of(options.driverOptions.contentType, actualSchema) private val safeParser = new FailureSafeParser[Array[Byte]]( parser.parse(_).toSeq, options.readOptions.parseMode, @@ -36,9 +36,9 @@ class ArangoQueryReader(schema: StructType, options: ArangoDBConf) extends Input final override def next: Boolean = if (iterator.hasNext) { val current = iterator.next() - rowIterator = safeParser.parse(options.readOptions.contentType match { - case ContentType.VPack => current.toByteArray - case ContentType.Json => current.toString.getBytes(StandardCharsets.UTF_8) + rowIterator = safeParser.parse(options.driverOptions.contentType match { + case ContentType.VPACK => current.toByteArray + case ContentType.JSON => current.toString.getBytes(StandardCharsets.UTF_8) }) if (rowIterator.hasNext) true else next diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala index cfe5a04f..f0925783 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriter.scala @@ -49,8 +49,7 @@ class ArangoDataWriter(schema: StructType, options: ArangoDBConf, partitionId: I private def createClient() = ArangoClient(options.updated(ArangoDBConf.ENDPOINTS, endpoints(endpointIdx))) private def canRetry: Boolean = - if (options.writeOptions.overwriteMode.isEmpty) false - else options.writeOptions.overwriteMode.get match { + options.writeOptions.overwriteMode match { case OverwriteMode.ignore => true case OverwriteMode.replace => true case OverwriteMode.update => options.writeOptions.keepNull @@ -60,7 +59,7 @@ class ArangoDataWriter(schema: StructType, options: ArangoDBConf, partitionId: I private def initBatch(): Unit = { batchCount = 0 outVPack = new ByteArrayOutputStream() - vpackGenerator = ArangoGeneratorProvider().of(options.writeOptions.contentType, schema, outVPack) + vpackGenerator = ArangoGeneratorProvider().of(options.driverOptions.contentType, schema, outVPack) vpackGenerator.writeStartArray() } @@ -68,9 +67,9 @@ class ArangoDataWriter(schema: StructType, options: ArangoDBConf, partitionId: I vpackGenerator.writeEndArray() vpackGenerator.close() vpackGenerator.flush() - val payload = options.writeOptions.contentType match { - case ContentType.VPack => new VPackSlice(outVPack.toByteArray) - case ContentType.Json => new VPackParser.Builder().build().fromJson(new String(outVPack.toByteArray), true) + val payload = options.driverOptions.contentType match { + case ContentType.VPACK => new VPackSlice(outVPack.toByteArray) + case ContentType.JSON => new VPackParser.Builder().build().fromJson(new String(outVPack.toByteArray), true) } saveDocuments(payload) } @@ -85,7 +84,7 @@ class ArangoDataWriter(schema: StructType, options: ArangoDBConf, partitionId: I client.shutdown() failures += 1 endpointIdx += 1 - if (canRetry && failures < options.driverOptions.endpoints.size) { + if (canRetry && failures < options.driverOptions.endpoints.length) { client = createClient() saveDocuments(payload) } else throw e diff --git a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriterFactory.scala b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriterFactory.scala index e173f926..57c0070b 100644 --- a/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriterFactory.scala +++ b/arangodb-spark-datasource-2.4/src/main/scala/org/apache/spark/sql/arangodb/datasource/writer/ArangoDataWriterFactory.scala @@ -1,5 +1,6 @@ package org.apache.spark.sql.arangodb.datasource.writer +import org.apache.spark.sql.arangodb.commons.ArangoDBConf import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory} import org.apache.spark.sql.types.StructType From 1bae89e39e15740d698c47a647220dcd559dbb30 Mon Sep 17 00:00:00 2001 From: Michele Rastelli Date: Fri, 10 Dec 2021 11:52:04 +0100 Subject: [PATCH 6/8] doc upd --- README.md | 60 +++++++++---------- .../{ArangoOptions.scala => package.scala} | 0 .../write/CreateCollectionTest.scala | 11 ++-- 3 files changed, 35 insertions(+), 36 deletions(-) rename arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/{ArangoOptions.scala => package.scala} (100%) diff --git a/README.md b/README.md index 17a88b98..20265515 100644 --- a/README.md +++ b/README.md @@ -47,9 +47,9 @@ To use in external Spark cluster, submit your application with the following par - `user`: db user, default `root` - `password`: db password - `endpoints`: list of coordinators, eg. `c1:8529,c2:8529` (required) -- `acquire-host-list`: acquire the list of all known hosts in the cluster (`true`|`false`), default `false` +- `acquireHostList`: acquire the list of all known hosts in the cluster (`true`|`false`), default `false` - `protocol`: communication protocol (`vst`|`http`), default `http` -- `content-type`: content type for driver communication (`json`|`vpack`), default `vpack` +- `contentType`: content type for driver communication (`json`|`vpack`), default `json` - `ssl.enabled`: ssl secured driver connection (`true`|`false`), default `false` - `ssl.cert.value`: base64 encoded certificate - `ssl.cert.type`: certificate type, default `X.509` @@ -57,7 +57,6 @@ To use in external Spark cluster, submit your application with the following par - `ssl.algorithm`: trust manager algorithm, default `SunX509` - `ssl.keystore.type`: keystore type, default `jks` - `ssl.protocol`: SSLContext protocol, default `TLS` -- `database`: database name, default `_system` ### SSL @@ -134,11 +133,12 @@ usersDF.filter(col("birthday") === "1982-12-15").show() ### Read Configuration +- `database`: database name, default `_system` - `table`: datasource ArangoDB collection name, ignored if `query` is specified. Either `table` or `query` is required. - `query`: custom AQL read query. If set, `table` will be ignored. Either `table` or `query` is required. -- `sample.size`: sample size prefetched for schema inference, only used if read schema is not provided, default `1000` -- `batch.size`: reading batch size, default `1000` -- `fill.cache`: whether the query should store the data it reads in the RocksDB block cache (`true`|`false`) +- `batchSize`: reading batch size, default `10000` +- `sampleSize`: sample size prefetched for schema inference, only used if read schema is not provided, default `1000` +- `fillBlockCache`: whether the query should store the data it reads in the RocksDB block cache (`true`|`false`), default `false` - `stream`: whether the query should be executed lazily, default `true` - `mode`: allows a mode for dealing with corrupt records during parsing: - `PERMISSIVE` : when it meets a corrupted record, puts the malformed string into a field configured by @@ -211,24 +211,24 @@ according to the related target collection definition and is different from the ### Write Configuration - `table`: target ArangoDB collection name (required) +- `batchSize`: writing batch size, default `10000` - `table.shards`: number of shards of the created collection (in case of SaveMode `Append` or `Overwrite`) -- `table.type`: type (`document`|`edge`) of the created collection (in case of SaveMode `Append` or `Overwrite`) -- `batch.size`: writing batch size, default `1000` -- `wait.sync`: whether to wait until the documents have been synced to disk (`true`|`false`) -- `confirm.truncate`: confirm to truncate table when using `SaveMode.Overwrite` mode, default `false` -- `overwrite.mode`: configures the behavior in case a document with the specified `_key` value exists already +- `table.type`: type (`document`|`edge`) of the created collection (in case of SaveMode `Append` or `Overwrite`), default `document` +- `waitForSync`: whether to wait until the documents have been synced to disk (`true`|`false`), default `false` +- `confirmTruncate`: confirm to truncate table when using save mode `Overwrite` mode, default `false` +- `overwriteMode`: configures the behavior in case a document with the specified `_key` value exists already - `ignore`: it will not be written - `replace`: it will be overwritten with the specified document value - `update`: it will be patched (partially updated) with the specified document value. The overwrite mode can be - further controlled via the `keep.null` and `merge.objects` parameter. `keep.null` will also be automatically set to + further controlled via the `keepNull` and `mergeObjects` parameter. `keepNull` will also be automatically set to `true`, so that null values are kept in the saved documents and not used to remove existing document fields (as for default ArangoDB upsert behavior). - - `conflict`: return a unique constraint violation error so that the insert operation fails -- `merge.objects`: in case `overwrite.mode` is set to `update`, controls whether objects (not arrays) will be merged. - - `true`: objects will be merged + - `conflict` (default): return a unique constraint violation error so that the insert operation fails +- `mergeObjects`: in case `overwriteMode` is set to `update`, controls whether objects (not arrays) will be merged. + - `true` (default): objects will be merged - `false`: existing document fields will be overwritten -- `keep.null`: in case `overwrite.mode` is set to `update` - - `true`: `null` values are saved within the document (default) +- `keepNull`: in case `overwriteMode` is set to `update` + - `true` (default): `null` values are saved within the document (default) - `false`: `null` values are used to delete corresponding existing attributes ### SaveMode @@ -239,18 +239,18 @@ already exists. Spark 2.4 implementation supports all save modes with the following semantics: - `Append`: the target collection is created if it does not exist - `Overwrite`: the target collection is created if it does not exist, it is truncated otherwise. Use in combination with - `confirm.truncate` write configuration parameter. + `confirmTruncate` write configuration parameter. - `ErrorIfExists`: the target collection is created if it does not exist, an `AnalysisException` is thrown otherwise - `Ignore`: the target collection is created if it does not exist, no write is performed otherwise Spark 3.1 implementation supports: - `Append`: the target collection is created if it does not exist - `Overwrite`: the target collection is created if it does not exist, it is truncated otherwise. Use in combination with - `confirm.truncate` write configuration parameter. + `confirmTruncate` write configuration parameter. -`SaveMode.ErrorIfExists` and `SaveMode.Ignore` behave the same as `SaveMode.Append`. +In Spark 3.1, save modes `ErrorIfExists` and `Ignore` behave the same as `Append`. -Use `overwrite.mode` write configuration parameter to specify the documents overwrite behavior (in case a document with +Use `overwriteMode` write configuration parameter to specify the documents overwrite behavior (in case a document with the same `_key` already exists). @@ -260,24 +260,24 @@ The data of each partition is saved in batches using ArangoDB API for inserting ([create multiple documents](https://www.arangodb.com/docs/stable/http/document-working-with-documents.html#create-multiple-documents)). This operation is not atomic, therefore some documents could be successfully written to the database, while others could fail. To makes the job more resilient to temporary errors (i.e. connectivity problems), in case of failure the request -will be retried (with another coordinator) if the configured `overwrite.mode` allows for idempotent requests, namely: +will be retried (with another coordinator) if the configured `overwriteMode` allows for idempotent requests, namely: - `replace` - `ignore` - `update` with `keep.null=true` -These configurations of `overwrite.mode` would also be compatible with speculative execution of tasks. +These configurations of `overwriteMode` would also be compatible with speculative execution of tasks. A failing batch-saving request is retried at most once for every coordinator. After that, if still failing, the write task for the related partition is aborted. According to the Spark configuration, the task could be retried and -rescheduled on a different executor, if the `overwrite.mode` allows for idempotent requests (as above). +rescheduled on a different executor, if the `overwriteMode` allows for idempotent requests (as above). If a task ultimately fails and is aborted, the entire write job will be aborted as well. Depending on the `SaveMode` configuration, the following cleanup operations will be performed: -- `SaveMode.Append`: no cleanup is performed and the underlying data source may require manual cleanup. +- `Append`: no cleanup is performed and the underlying data source may require manual cleanup. `DataWriteAbortException` is thrown. -- `SaveMode.Overwrite`: the target collection will be truncated -- `SaveMode.ErrorIfExists`: the target collection will be dropped -- `SaveMode.Ignore`: if the collection did not exist before it will be dropped, nothing otherwise +- `Overwrite`: the target collection will be truncated +- `ErrorIfExists`: the target collection will be dropped +- `Ignore`: if the collection did not exist before it will be dropped, nothing otherwise ### Write Limitations @@ -285,9 +285,9 @@ configuration, the following cleanup operations will be performed: - Batch writes are not performed atomically, so in some cases (i.e. in case of `overwrite.mode: conflict`) some documents in the batch may be written and some others may return an exception (i.e. due to conflicting key). - Writing records with `_key` attribute is only allowed on collections sharded by `_key`. -- In case of `SaveMode.Append`, failed jobs cannot be rolled back and the underlying data source may require manual +- In case of save mode `Append`, failed jobs cannot be rolled back and the underlying data source may require manual cleanup. -- Speculative execution of tasks would only work for idempotent `overwrite.mode` configurations +- Speculative execution of tasks would only work for idempotent `overwriteMode` configurations (see [Write Resiliency](#write-resiliency)). diff --git a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoOptions.scala b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/package.scala similarity index 100% rename from arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoOptions.scala rename to arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/package.scala diff --git a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/CreateCollectionTest.scala b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/CreateCollectionTest.scala index b9172adf..3253710c 100644 --- a/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/CreateCollectionTest.scala +++ b/integration-tests/src/test/scala/org/apache/spark/sql/arangodb/datasource/write/CreateCollectionTest.scala @@ -1,9 +1,8 @@ package org.apache.spark.sql.arangodb.datasource.write import com.arangodb.ArangoCollection -import com.arangodb.entity.CollectionType import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.arangodb.commons.ArangoDBConf +import org.apache.spark.sql.arangodb.commons.{ArangoDBConf, CollectionType} import org.apache.spark.sql.arangodb.datasource.BaseSparkTest import org.assertj.core.api.Assertions.assertThat import org.junit.jupiter.api.BeforeEach @@ -46,14 +45,14 @@ class CreateCollectionTest extends BaseSparkTest { ArangoDBConf.PROTOCOL -> protocol, ArangoDBConf.CONTENT_TYPE -> contentType, ArangoDBConf.NUMBER_OF_SHARDS -> "5", - ArangoDBConf.COLLECTION_TYPE -> org.apache.spark.sql.arangodb.commons.CollectionType.EDGE.name + ArangoDBConf.COLLECTION_TYPE -> CollectionType.EDGE.name )) .save() if (isCluster) { assertThat(collection.getProperties.getNumberOfShards).isEqualTo(5) } - assertThat(collection.getProperties.getType.getType).isEqualTo(CollectionType.EDGES.getType) + assertThat(collection.getProperties.getType.getType).isEqualTo(com.arangodb.entity.CollectionType.EDGES.getType) } @ParameterizedTest @@ -68,14 +67,14 @@ class CreateCollectionTest extends BaseSparkTest { ArangoDBConf.CONTENT_TYPE -> contentType, ArangoDBConf.CONFIRM_TRUNCATE -> "true", ArangoDBConf.NUMBER_OF_SHARDS -> "5", - ArangoDBConf.COLLECTION_TYPE -> org.apache.spark.sql.arangodb.commons.CollectionType.EDGE.name + ArangoDBConf.COLLECTION_TYPE -> CollectionType.EDGE.name )) .save() if (isCluster) { assertThat(collection.getProperties.getNumberOfShards).isEqualTo(5) } - assertThat(collection.getProperties.getType.getType).isEqualTo(CollectionType.EDGES.getType) + assertThat(collection.getProperties.getType.getType).isEqualTo(com.arangodb.entity.CollectionType.EDGES.getType) } } From 035e382f2bb2c9baf911293064f73b91bd68dbc2 Mon Sep 17 00:00:00 2001 From: Michele Rastelli Date: Fri, 10 Dec 2021 12:11:58 +0100 Subject: [PATCH 7/8] fixed Scala 2.11 compilation --- .../org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala index 554c31cb..67fd0d1e 100644 --- a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala +++ b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala @@ -294,8 +294,8 @@ class ArangoDBConf(opts: Map[String, String]) extends Serializable with Logging import ArangoDBConf._ private val options = CaseInsensitiveMap(opts) + options.foreach(i => checkConf(i._1, i._2)) private val settings = options.asJava - settings.forEach(checkConf(_, _)) @transient protected val reader = new ConfigReader(settings) From 36701c76b335d91d232ca453311fb5960fe5c624 Mon Sep 17 00:00:00 2001 From: Michele Rastelli Date: Fri, 10 Dec 2021 12:29:40 +0100 Subject: [PATCH 8/8] fixed Scala 2.11 serialization --- .../sql/arangodb/commons/ArangoDBConf.scala | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala index 67fd0d1e..5c6c4557 100644 --- a/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala +++ b/arangodb-spark-commons/src/main/scala/org/apache/spark/sql/arangodb/commons/ArangoDBConf.scala @@ -293,26 +293,25 @@ class ArangoDBConf(opts: Map[String, String]) extends Serializable with Logging import ArangoDBConf._ - private val options = CaseInsensitiveMap(opts) - options.foreach(i => checkConf(i._1, i._2)) - private val settings = options.asJava + private val settings = CaseInsensitiveMap(opts) + settings.foreach(i => checkConf(i._1, i._2)) - @transient protected val reader = new ConfigReader(settings) + @transient protected val reader = new ConfigReader(settings.asJava) - lazy val driverOptions: ArangoDBDriverConf = new ArangoDBDriverConf(options) - lazy val readOptions: ArangoDBReadConf = new ArangoDBReadConf(options) - lazy val writeOptions: ArangoDBWriteConf = new ArangoDBWriteConf(options) + lazy val driverOptions: ArangoDBDriverConf = new ArangoDBDriverConf(settings) + lazy val readOptions: ArangoDBReadConf = new ArangoDBReadConf(settings) + lazy val writeOptions: ArangoDBWriteConf = new ArangoDBWriteConf(settings) - def updated(kv: (String, String)): ArangoDBConf = new ArangoDBConf(options + kv) + def updated(kv: (String, String)): ArangoDBConf = new ArangoDBConf(settings + kv) - def updated(other: ArangoDBConf): ArangoDBConf = new ArangoDBConf(options ++ other.options) + def updated(other: ArangoDBConf): ArangoDBConf = new ArangoDBConf(settings ++ other.settings) protected def getRequiredConf[T](entry: OptionalConfigEntry[T]): T = getConf(entry).getOrElse(throw new IllegalArgumentException(s"Required ${entry.key} configuration parameter")) /** Return the value of Spark ArangoDB configuration property for the given key. */ @throws[NoSuchElementException]("if key is not set") - def getConfString(key: String): String = Option(settings.get(key)).getOrElse(throw new NoSuchElementException(key)) + def getConfString(key: String): String = settings.get(key).getOrElse(throw new NoSuchElementException(key)) /** * Return the value of Spark ArangoDB configuration property for the given key. If the key is not set @@ -320,7 +319,7 @@ class ArangoDBConf(opts: Map[String, String]) extends Serializable with Logging * desired one. */ def getConf[T](entry: ConfigEntry[T], defaultValue: T): T = - Option(settings.get(entry.key)).map(entry.valueConverter).getOrElse(defaultValue) + settings.get(entry.key).map(entry.valueConverter).getOrElse(defaultValue) /** * Return the value of Spark ArangoDB configuration property for the given key. If the key is not set @@ -338,13 +337,13 @@ class ArangoDBConf(opts: Map[String, String]) extends Serializable with Logging * Return the `string` value of Spark ArangoDB configuration property for the given key. If the key is * not set, return `defaultValue`. */ - def getConfString(key: String, defaultValue: String): String = Option(settings.get(key)).getOrElse(defaultValue) + def getConfString(key: String, defaultValue: String): String = settings.get(key).getOrElse(defaultValue) /** * Return all the configuration properties that have been set (i.e. not the default). * This creates a new copy of the config properties in the form of a Map. */ - def getAllConfigs: Map[String, String] = settings.asScala.toMap + def getAllConfigs: Map[String, String] = settings.toMap /** * Return all the configuration definitions that have been defined in [[ArangoDBConf]]. Each