├── spark-mongodb
└── src
│ ├── test
│ ├── resources
│ │ └── scala.version
│ └── scala
│ │ ├── com
│ │ └── stratio
│ │ │ └── datasource
│ │ │ ├── MongodbTestConstants.scala
│ │ │ └── mongodb
│ │ │ ├── config
│ │ │ └── ConfigTest.scala
│ │ │ ├── MongoEmbedDatabase.scala
│ │ │ ├── MongoClusterEmbedDatabase.scala
│ │ │ ├── schema
│ │ │ ├── MongodbSchemaIT.scala
│ │ │ └── MongodbRowConverterIT.scala
│ │ │ ├── partitioner
│ │ │ └── MongodbPartitionerIT.scala
│ │ │ ├── MongodbRelationIT.scala
│ │ │ ├── client
│ │ │ └── MongodbClientFactoryTest.scala
│ │ │ ├── TestBsonData.scala
│ │ │ ├── writer
│ │ │ └── MongodbWriterIT.scala
│ │ │ └── reader
│ │ │ └── MongodbReaderIT.scala
│ │ └── org
│ │ └── apache
│ │ └── spark
│ │ └── sql
│ │ └── mongodb
│ │ └── TestSQLContext.scala
│ └── main
│ ├── include
│ └── NOTICE
│ └── scala
│ └── com
│ └── stratio
│ └── datasource
│ ├── mongodb
│ ├── package.scala
│ ├── config
│ │ ├── MongodbCredentials.scala
│ │ ├── MongodbSSLOptions.scala
│ │ ├── MongodbConfigBuilder.scala
│ │ ├── MongodbConfigReader.scala
│ │ └── MongodbConfig.scala
│ ├── client
│ │ ├── ConnectionStatus.scala
│ │ ├── MongodbConnection.scala
│ │ └── MongodbClientFactory.scala
│ ├── util
│ │ └── usingMongoClient.scala
│ ├── partitioner
│ │ ├── MongodbPartition.scala
│ │ └── MongodbPartitioner.scala
│ ├── MongodbFunctions.scala
│ ├── writer
│ │ ├── MongodbSimpleWriter.scala
│ │ ├── MongodbBatchWriter.scala
│ │ └── MongodbWriter.scala
│ ├── sources
│ │ └── customFilters.scala
│ ├── MongodbContext.scala
│ ├── MongodbDataFrame.scala
│ ├── rdd
│ │ ├── MongodbRDD.scala
│ │ └── MongodbRDDIterator.scala
│ ├── DefaultSource.scala
│ ├── reader
│ │ └── MongodbReader.scala
│ ├── schema
│ │ ├── MongodbSchema.scala
│ │ ├── MongodbRowConverter.scala
│ │ └── JsonSupport.scala
│ ├── query
│ │ └── FilterSection.scala
│ └── MongodbRelation.scala
│ ├── partitioner
│ ├── PartitionRange.scala
│ └── Partitioner.scala
│ ├── schema
│ ├── SchemaProvider.scala
│ └── RowConverter.scala
│ └── util
│ ├── using.scala
│ └── Config.scala
├── doc
├── src
│ └── site
│ │ └── sphinx
│ │ ├── index.rst
│ │ ├── faqs.rst
│ │ ├── PoweredBy.rst
│ │ └── about.rst
└── pom.xml
├── .gitignore
├── CHANGELOG.md
├── spark-mongodb-examples
├── src
│ └── main
│ │ └── scala
│ │ └── com
│ │ └── stratio
│ │ └── datasource
│ │ └── mongodb
│ │ └── examples
│ │ ├── SQLExample.scala
│ │ ├── NestedFieldsExample.scala
│ │ ├── DataFrameAPIExample.scala
│ │ └── ExampleUtils.scala
└── pom.xml
├── README.md
├── pom.xml
├── spark-mongodb_2.11
└── pom.xml
└── spark-mongodb_2.10
└── pom.xml
/spark-mongodb/src/test/resources/scala.version:
--------------------------------------------------------------------------------
1 | ${scala.binary.version}
--------------------------------------------------------------------------------
/spark-mongodb/src/main/include/NOTICE:
--------------------------------------------------------------------------------
1 | Stratio Spark-MongoDB
2 | Copyright 2014 Stratio
3 |
4 | This product includes software developed at
5 | Stratio (http://www.openstratio.org/).
--------------------------------------------------------------------------------
/doc/src/site/sphinx/index.rst:
--------------------------------------------------------------------------------
1 | Contents:
2 | ********
3 | .. toctree::
4 | :maxdepth: 4
5 | :numbered:
6 |
7 | about.rst
8 | First_Steps.rst
9 | faqs.rst
10 | PoweredBy.rst
11 |
--------------------------------------------------------------------------------
/.gitignore:
--------------------------------------------------------------------------------
1 | hs_err_pid*.log
2 | nohup.out
3 | .idea
4 | *.iml
5 | **/.idea
6 | */.classpath
7 | */.project
8 | */.settings
9 | */.cache
10 | */test-output/
11 | *.log
12 | */*.versionsBackup
13 | target/
14 | *GitIgnored*
15 | *.asc
16 | *.gpg
17 |
--------------------------------------------------------------------------------
/doc/src/site/sphinx/faqs.rst:
--------------------------------------------------------------------------------
1 | ============
2 | Faqs
3 | ============
4 |
5 |
6 | - I can't connect to a replicaSet.
7 |
8 | When you access to a replicaSet you have to indicate IP or hostname on the same way that is set on the replicaSet config. You can see the configuration running this command "rs.conf()". We recommend use the same rule for each node.
--------------------------------------------------------------------------------
/doc/src/site/sphinx/PoweredBy.rst:
--------------------------------------------------------------------------------
1 | ============
2 | Powered by
3 | ============
4 |
5 | If you are using this Data Source, feel free to briefly share your experience by Pull Request.
6 |
7 |
8 | ***************
9 | Companies
10 | ***************
11 |
12 | - `Stratio platform `_:
13 | - `Crossdata `_: Crossdata uses spark-mongodb datasource as a piece of the MongoDB connector.
14 |
15 |
16 |
17 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/package.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource
17 |
18 |
19 | /**
20 | * Whole MongoDB helpers.
21 | */
22 | package object mongodb extends MongodbFunctions
23 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/config/MongodbCredentials.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.config
17 |
18 | case class MongodbCredentials(
19 | user: String,
20 | database: String,
21 | password: Array[Char])
22 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/client/ConnectionStatus.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.client
17 |
18 | /**
19 | * Different statuses for one mongoDbClient connection
20 | */
21 | object ConnectionStatus extends Enumeration {
22 |
23 | type ConnectionStatus = Value
24 |
25 | val Busy, Free = Value
26 | }
27 |
--------------------------------------------------------------------------------
/CHANGELOG.md:
--------------------------------------------------------------------------------
1 | # Changelog
2 |
3 | ## 0.13.0 (upcoming)
4 |
5 | * Pending changelog
6 |
7 | ## 0.12.0 (upcoming)
8 |
9 | * Upgrade Spark to 2.0.0
10 |
11 | ## 0.11.1 (March 2016)
12 |
13 | * Max and Min splitVector bounds for not sharded collections (see doc)
14 | * Config parameter renamed idasobjectid -> idAsObjectId
15 |
16 | ## 0.11.0 (February 2016)
17 |
18 | * Compatibility with spark 1.6
19 | * Refactor mongo clients to improve performance
20 |
21 | ## 0.10.1 (November 2015)
22 |
23 | * Allow filter by _id like an ObjectId or another type
24 | * Correct out datatypes
25 | * Support for more types
26 |
27 | ## 0.10.0 (October 2015)
28 |
29 | * Adapted to Spark 1.5
30 |
31 | ## 0.9.2 (November 2015)
32 |
33 | * Partial NOT filter working
34 |
35 | ## 0.9.1 (November 2015)
36 |
37 | * Refactor update _id on writer(_idField property not necessary)
38 |
39 | ## 0.9.0 (October 2015)
40 |
41 | * Mapping config for client
42 | * Refactor documentation
43 | * R API added to documentation
44 | * Bugs fixed
45 |
46 | ## 0.8.7 (August 2015)
47 |
48 | * Bugs fixed
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/partitioner/PartitionRange.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.partitioner
17 |
18 | /**
19 | * Determines a range of keys that compounds some partition
20 | * @param minKey Starting partition key
21 | * @param maxKey Final partition key
22 | * @tparam T Partition key type.
23 | */
24 | case class PartitionRange[T](minKey: Option[T], maxKey: Option[T])
25 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/schema/SchemaProvider.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.schema
17 |
18 | import org.apache.spark.sql.types.StructType
19 |
20 | /**
21 | * Knows the way to provide some Data Source schema
22 | */
23 | trait SchemaProvider {
24 |
25 | /**
26 | * Provides the schema for current implementation of Data Source
27 | * @return schema
28 | */
29 | def schema(): StructType
30 |
31 | }
32 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/partitioner/Partitioner.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.partitioner
17 |
18 | import org.apache.spark.Partition
19 |
20 | /**
21 | * Provides the way to compute and get spark partitions over
22 | * some Data Source.
23 | * @tparam T
24 | */
25 | trait Partitioner[T <: Partition] extends Serializable {
26 |
27 | /**
28 | * Retrieves some Data Source partitions
29 | * @return An array with computed partitions
30 | */
31 | def computePartitions(): Array[T]
32 |
33 | }
34 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/util/usingMongoClient.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.util
17 |
18 | import com.mongodb.casbah.MongoClient
19 | import com.stratio.datasource.mongodb.client.MongodbClientFactory
20 |
21 | import scala.util.Try
22 |
23 | object usingMongoClient {
24 |
25 | def apply[A](mongoClient: MongoClient)(code: MongoClient => A): A =
26 | try {
27 | code(mongoClient)
28 | } finally {
29 | Try(MongodbClientFactory.closeByClient(mongoClient))
30 | }
31 | }
32 |
33 |
--------------------------------------------------------------------------------
/spark-mongodb/src/test/scala/com/stratio/datasource/MongodbTestConstants.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource
17 |
18 | import scala.io.Source
19 |
20 | trait MongodbTestConstants {
21 |
22 | val scalaBinaryVersionFromFile = Source.fromInputStream(getClass.getResourceAsStream("/scala.version")).mkString
23 | val mongoPort: Int = if(scalaBinaryVersionFromFile == "2.10") 21027 else 21127
24 | val db: String = if(scalaBinaryVersionFromFile == "2.10") "testDB210" else "testDB211"
25 | val scalaBinaryVersion: String = s" [Scala $scalaBinaryVersionFromFile]"
26 | }
27 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/partitioner/MongodbPartition.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.partitioner
17 |
18 | import com.mongodb.casbah.Imports._
19 | import com.stratio.datasource.partitioner.PartitionRange
20 | import org.apache.spark.Partition
21 |
22 | /**
23 | * @param index Partition index
24 | * @param hosts Hosts that hold partition data
25 | * @param partitionRange Partition range
26 | */
27 | case class MongodbPartition(
28 | index: Int,
29 | hosts: Seq[String],
30 | partitionRange: PartitionRange[DBObject]) extends Partition
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/MongodbFunctions.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb
17 |
18 |
19 | import org.apache.spark.sql.{SQLContext, DataFrame}
20 |
21 | import scala.language.implicitConversions
22 |
23 |
24 | /**
25 | * Helpers for getting / storing MongoDB data.
26 | */
27 | trait MongodbFunctions {
28 |
29 | implicit def toMongodbContext(sqlContext: SQLContext): MongodbContext =
30 | new MongodbContext(sqlContext)
31 |
32 | implicit def toMongodbSchemaRDD(dataFrame: DataFrame): MongodbDataFrame =
33 | new MongodbDataFrame(dataFrame)
34 |
35 | }
36 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/writer/MongodbSimpleWriter.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.writer
17 |
18 | import com.mongodb.casbah.Imports._
19 | import com.stratio.datasource.util.Config
20 |
21 | /**
22 | * A simple mongodb writer.
23 | *
24 | * @param config Configuration parameters (host,database,collection,...)
25 | */
26 | private[mongodb] class MongodbSimpleWriter(config: Config) extends MongodbWriter(config) {
27 |
28 | override def save(it: Iterator[DBObject], mongoClient: MongoClient): Unit =
29 | it.foreach(dbo => dbCollection(mongoClient).save(dbo, writeConcern))
30 |
31 | }
32 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/client/MongodbConnection.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.client
17 |
18 | import com.stratio.datasource.mongodb.client.ConnectionStatus._
19 | import com.stratio.datasource.mongodb.client.MongodbClientFactory.Client
20 |
21 | /**
22 | * Connection object that represents one client connection with the status and the expire session time
23 | */
24 | case class MongodbConnection(client: Client,
25 | timeOut: Long = 0L,
26 | status: ConnectionStatus = ConnectionStatus.Busy,
27 | index: Int = 0)
28 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/config/MongodbSSLOptions.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.config
17 |
18 | import scala.language.implicitConversions
19 |
20 | /**
21 | * Case class with the SSL options.
22 | */
23 | case class MongodbSSLOptions(keyStore: Option[String] = None,
24 | keyStorePassword: Option[String] = None,
25 | trustStore: String,
26 | trustStorePassword: Option[String] = None)
27 |
28 | object MongodbSSLOptions {
29 |
30 | implicit def stringToOption(parameter: String): Option[String] = Some(parameter)
31 | }
32 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/schema/RowConverter.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.schema
17 |
18 | import org.apache.spark.rdd.RDD
19 | import org.apache.spark.sql.Row
20 | import org.apache.spark.sql.types.StructType
21 |
22 | /**
23 | * Knows how to map from some Data Source native RDD to an {{{RDD[Row]}}}
24 | * @tparam T Original RDD type
25 | */
26 | trait RowConverter[T] {
27 |
28 | /**
29 | * Given a known schema,
30 | * it maps an RDD of some specified type to an {{{RDD[Row}}}
31 | * @param schema RDD native schema
32 | * @param rdd Current native RDD
33 | * @return A brand new RDD of Spark SQL Row type.
34 | */
35 | def asRow(schema: StructType, rdd: RDD[T]): RDD[Row]
36 |
37 | }
38 |
39 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/sources/customFilters.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.sources
17 |
18 | import org.apache.spark.sql.sources.Filter
19 |
20 | trait GeoFilter extends Filter {
21 | val attribute: String
22 | val maxDistance: Option[Double]
23 | }
24 |
25 | case class Near(
26 | attribute: String,
27 | x: Double, y: Double,
28 | maxDistance: Option[Double] = None
29 | ) extends GeoFilter
30 |
31 | case class NearSphere(
32 | attribute: String,
33 | longitude: Double, latitude: Double,
34 | maxDistance: Option[Double] = None
35 | ) extends GeoFilter
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/util/using.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.util
17 |
18 | import scala.language.reflectiveCalls
19 | import scala.util.Try
20 |
21 | /**
22 | * DSL helper for enclosing some functionality into a closeable type.
23 | * Helper will be responsible of closing the object.
24 | * i.e.:{{{
25 | * import java.io._
26 | * val writer = new PrintWriter(new File("test.txt" ))
27 | * using(writer){ w =>
28 | * w.append("hi!")
29 | * }
30 | * }}}
31 | */
32 | object using {
33 |
34 | type AutoClosable = { def close(): Unit }
35 |
36 | def apply[A <: AutoClosable, B](resource: A)(code: A => B): B =
37 | try {
38 | code(resource)
39 | }
40 | finally {
41 | Try(resource.close())
42 | }
43 |
44 | }
45 |
--------------------------------------------------------------------------------
/doc/src/site/sphinx/about.rst:
--------------------------------------------------------------------------------
1 | About
2 | *****
3 |
4 | Spark-Mongodb is a library that allows the user to read/write data with `Spark SQL `__
5 | from/into MongoDB collections.
6 |
7 | `MongoDB `__ provides a documental data model
8 | richer than typical key/value systems. `Spark `__ is a
9 | fast and general-purpose cluster computing system that can run applications up to 100 times faster than Hadoop.
10 |
11 | Integrating MongoDB and Spark gives us a system that combines the best of both
12 | worlds opening to MongoDB the possibility of solving a wide range of new use cases.
13 |
14 |
15 | Latest compatible versions
16 | ==========================
17 | +-----------------+----------------+----------+
18 | | Spark-MongoDB | Apache Spark | MongoDB |
19 | +=================+================+==========+
20 | | 0.10.x | 1.5.x | 3.0.x |
21 | +-----------------+----------------+----------+
22 | | 0.8.2 - 0.9.2 | 1.4.0 | 3.0.x |
23 | +-----------------+----------------+----------+
24 | | 0.8.1 | 1.3.0 | 3.0.x |
25 | +-----------------+----------------+----------+
26 | | 0.8.0 | 1.2.1 | 3.0.x |
27 | +-----------------+----------------+----------+
28 |
29 |
30 | Requirements
31 | ============
32 | This library requires Apache Spark 1.5.X, Scala 2.10 or Scala 2.11, Casbah 2.8.X
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/config/MongodbConfigBuilder.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.config
17 |
18 | import com.stratio.datasource.util.{Config, ConfigBuilder}
19 | import Config._
20 |
21 | /**
22 | * A specialized Mongo configuration builder.
23 | * It focuses on mongodb config parameters
24 | * such as host,database,collection, samplingRatio (for schema infer)
25 | * @param props Initial properties map
26 | */
27 |
28 | case class MongodbConfigBuilder(props: Map[Property, Any] = Map()) extends {
29 |
30 | override val properties = Map() ++ props
31 |
32 | } with ConfigBuilder[MongodbConfigBuilder](properties) {
33 |
34 | val requiredProperties: List[Property] = MongodbConfig.required
35 |
36 | def apply(props: Map[Property, Any]) = MongodbConfigBuilder(props)
37 | }
38 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/MongodbContext.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb
17 |
18 | import com.stratio.datasource.util.Config
19 | import org.apache.spark.sql.{DataFrame, SQLContext}
20 | import org.apache.spark.sql.types.StructType
21 |
22 | import scala.language.implicitConversions
23 |
24 | /**
25 | * @param sqlContext Spark SQLContext
26 | */
27 | class MongodbContext(sqlContext: SQLContext) {
28 |
29 | /**
30 | * It retrieves a bunch of MongoDB objects
31 | * given a MongDB configuration object.
32 | * @param config MongoDB configuration object
33 | * @return A dataFrame
34 | */
35 | def fromMongoDB(config: Config,schema:Option[StructType]=None): DataFrame =
36 | sqlContext.baseRelationToDataFrame(
37 | new MongodbRelation(config, schema)(sqlContext))
38 |
39 | }
40 |
--------------------------------------------------------------------------------
/spark-mongodb-examples/src/main/scala/com/stratio/datasource/mongodb/examples/SQLExample.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.examples
17 |
18 | import com.stratio.datasource.mongodb.examples.MongoExampleFunctions._
19 |
20 | object SQLExample extends App with MongoDefaultConstants {
21 |
22 | val mongoClient = prepareEnvironment()
23 |
24 | withSQLContext { sqlContext =>
25 |
26 | sqlContext.sql(
27 | s"""|CREATE TEMPORARY TABLE $Collection
28 | |(id STRING, age INT, description STRING, enrolled BOOLEAN, name STRING, optionalField BOOLEAN)
29 | |USING $MongoProvider
30 | |OPTIONS (
31 | |host '$MongoHost:$MongoPort',
32 | |database '$Database',
33 | |collection '$Collection'
34 | |)
35 | """.stripMargin.replaceAll("\n", " "))
36 |
37 | sqlContext.sql(s"SELECT id, name FROM $Collection WHERE age > 16").show(5)
38 |
39 | }
40 |
41 | cleanEnvironment(mongoClient)
42 | }
--------------------------------------------------------------------------------
/spark-mongodb-examples/src/main/scala/com/stratio/datasource/mongodb/examples/NestedFieldsExample.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.examples
17 |
18 | import com.stratio.datasource.mongodb.examples.MongoExampleFunctions._
19 |
20 | object NestedFieldsExample extends App with MongoDefaultConstants {
21 |
22 | val mongoClient = prepareEnvironment()
23 |
24 | withSQLContext { sqlContext =>
25 | sqlContext.sql(
26 | s"""|CREATE TEMPORARY TABLE $Collection
27 | |(id STRING, age INT, description STRING, enrolled BOOLEAN, name STRING, optionalField BOOLEAN, fieldWithSubDoc struct> )
28 | |USING $MongoProvider
29 | |OPTIONS (
30 | |host '$MongoHost:$MongoPort',
31 | |database '$Database',
32 | |collection '$Collection'
33 | |)
34 | """.stripMargin.replaceAll("\n", " "))
35 |
36 | sqlContext.sql(s"SELECT fieldWithSubDoc.subDoc[0] FROM $Collection WHERE age = 14").show(5)
37 | }
38 |
39 | cleanEnvironment(mongoClient)
40 | }
--------------------------------------------------------------------------------
/spark-mongodb-examples/src/main/scala/com/stratio/datasource/mongodb/examples/DataFrameAPIExample.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.examples
17 |
18 | import com.stratio.datasource.mongodb.examples.MongoExampleFunctions._
19 |
20 | object DataFrameAPIExample extends App with MongoDefaultConstants {
21 |
22 | val mongoClient = prepareEnvironment()
23 |
24 | withSQLContext { sqlContext =>
25 |
26 | sqlContext.sql(
27 | s"""|CREATE TEMPORARY TABLE $Collection
28 | |(id STRING, age INT, description STRING, enrolled BOOLEAN, name STRING, optionalField BOOLEAN)
29 | |USING $MongoProvider
30 | |OPTIONS (
31 | |host '$MongoHost:$MongoPort',
32 | |database '$Database',
33 | |collection '$Collection'
34 | |)
35 | """.stripMargin.replaceAll("\n", " "))
36 |
37 | import org.apache.spark.sql.functions._
38 |
39 | val studentsDF = sqlContext.read.format(MongoProvider).table(Collection)
40 | studentsDF.where(studentsDF("age") > 15).groupBy(studentsDF("enrolled")).agg(avg("age"), max("age")).show(5)
41 |
42 | }
43 |
44 | cleanEnvironment(mongoClient)
45 | }
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/MongodbDataFrame.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb
17 |
18 | import com.stratio.datasource.mongodb.schema.MongodbRowConverter
19 | import com.stratio.datasource.mongodb.writer.{MongodbBatchWriter, MongodbSimpleWriter}
20 | import com.stratio.datasource.util.Config
21 | import org.apache.spark.sql.DataFrame
22 |
23 | import scala.language.implicitConversions
24 |
25 | /**
26 | * @param dataFrame Spark SchemaRDD
27 | */
28 | class MongodbDataFrame(dataFrame: DataFrame) extends Serializable {
29 |
30 | /**
31 | * It allows storing data in Mongodb from some existing SchemaRDD
32 | * @param config MongoDB configuration object
33 | * @param batch It indicates whether it has to be saved in batch mode or not.
34 | */
35 | def saveToMongodb(config: Config, batch: Boolean = true): Unit = {
36 | val schema = dataFrame.schema
37 | dataFrame.foreachPartition(it => {
38 | val writer =
39 | if (batch) new MongodbBatchWriter(config)
40 | else new MongodbSimpleWriter(config)
41 |
42 | writer.saveWithPk(
43 | it.map(row => MongodbRowConverter.rowAsDBObject(row, schema)))
44 |
45 | })
46 | }
47 |
48 |
49 |
50 | }
51 |
--------------------------------------------------------------------------------
/spark-mongodb/src/test/scala/org/apache/spark/sql/mongodb/TestSQLContext.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package org.apache.spark.sql.mongodb
17 |
18 | import org.apache.spark.sql.SQLContext
19 | import org.apache.spark.sql.internal.SQLConf
20 | import org.apache.spark.{SparkConf, SparkContext}
21 |
22 |
23 | /**
24 | * A special [[SQLContext]] prepared for testing.
25 | */
26 | class TestSQLContext(sc: SparkContext) extends SQLContext(sc) { self =>
27 |
28 | def this() {
29 | this(new SparkContext("local[2]", "test-sql-context",
30 | new SparkConf().set("spark.sql.testkey", "true")))
31 | }
32 |
33 | protected[sql] override lazy val conf: SQLConf = new SQLConf {
34 |
35 | clear()
36 |
37 | override def clear(): Unit = {
38 | super.clear()
39 |
40 | // Make sure we start with the default test configs even after clear
41 | TestSQLContext.overrideConfs.map {
42 | case (key, value) => setConfString(key, value)
43 | }
44 | }
45 | }
46 |
47 | }
48 |
49 | object TestSQLContext {
50 |
51 | /**
52 | * A map used to store all confs that need to be overridden in sql/core unit tests.
53 | */
54 | val overrideConfs: Map[String, String] =
55 | Map(
56 | // Fewer shuffle partitions to speed up testing.
57 | SQLConf.SHUFFLE_PARTITIONS.key -> "5")
58 | }
59 |
60 | object TemporaryTestSQLContext extends TestSQLContext
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/config/MongodbConfigReader.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.config
17 |
18 | import com.mongodb.casbah.Imports._
19 | import com.mongodb.{MongoCredential, ServerAddress}
20 | import com.stratio.datasource.mongodb.config.MongodbConfig._
21 | import com.stratio.datasource.util.Config
22 |
23 | object MongodbConfigReader {
24 |
25 | implicit class MongodbConfigFunctions(config: Config) {
26 | @transient protected[mongodb] val hosts : List[ServerAddress] =
27 | config[List[String]](MongodbConfig.Host)
28 | .map(add => new ServerAddress(add))
29 |
30 | @transient protected[mongodb] val credentials: List[MongoCredential] =
31 | config.getOrElse[List[MongodbCredentials]](MongodbConfig.Credentials, MongodbConfig.DefaultCredentials).map{
32 | case MongodbCredentials(user,database,password) =>
33 | MongoCredential.createCredential(user,database,password)
34 | }
35 |
36 | @transient protected[mongodb] val sslOptions: Option[MongodbSSLOptions] =
37 | config.get[MongodbSSLOptions](MongodbConfig.SSLOptions)
38 |
39 | @transient protected[mongodb] val writeConcern: WriteConcern = config.get[String](MongodbConfig.WriteConcern) match {
40 | case Some(wConcern) => parseWriteConcern(wConcern)
41 | case None => DefaultWriteConcern
42 | }
43 |
44 | protected[mongodb] val clientOptions = config.properties.filterKeys(_.contains(MongodbConfig.ListMongoClientOptions))
45 | }
46 |
47 | }
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/rdd/MongodbRDD.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.rdd
17 |
18 | import com.mongodb.casbah.Imports._
19 | import com.stratio.datasource.mongodb.partitioner.{MongodbPartition, MongodbPartitioner}
20 | import com.stratio.datasource.util.Config
21 | import org.apache.spark.rdd.RDD
22 | import org.apache.spark.sql.SQLContext
23 | import org.apache.spark.{Partition, TaskContext}
24 |
25 | import com.stratio.datasource.mongodb.query.{NoFilters, FilterSection}
26 |
27 | /**
28 | * @param sc Spark SQLContext
29 | * @param config Config parameters
30 | * @param requiredColumns Fields to project
31 | * @param filters Query filters
32 | */
33 | class MongodbRDD(
34 | sc: SQLContext,
35 | config: Config,
36 | partitioner: MongodbPartitioner,
37 | requiredColumns: Array[String] = Array(),
38 | filters: FilterSection = NoFilters)
39 | extends RDD[DBObject](sc.sparkContext, deps = Nil) {
40 |
41 | override def getPartitions: Array[Partition] =
42 | partitioner.computePartitions().asInstanceOf[Array[Partition]]
43 |
44 | override def getPreferredLocations(split: Partition): Seq[String] =
45 | split.asInstanceOf[MongodbPartition].hosts.map(new ServerAddress(_).getHost)
46 |
47 | override def compute(
48 | split: Partition,
49 | context: TaskContext): MongodbRDDIterator =
50 | new MongodbRDDIterator(
51 | context,
52 | split.asInstanceOf[MongodbPartition],
53 | config,
54 | requiredColumns,
55 | filters)
56 |
57 | }
58 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/writer/MongodbBatchWriter.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.writer
17 |
18 | import com.mongodb.casbah.Imports._
19 | import com.stratio.datasource.mongodb.config.MongodbConfig
20 | import com.stratio.datasource.util.Config
21 |
22 | /**
23 | * A batch writer implementation for mongodb writer.
24 | * The used semantics for storing objects is 'replace'.
25 | *
26 | * @param config Configuration parameters (host,database,collection,...)
27 | */
28 | private[mongodb] class MongodbBatchWriter(config: Config) extends MongodbWriter(config) {
29 |
30 | private val IdKey = "_id"
31 |
32 | private val bulkBatchSize = config.getOrElse[Int](MongodbConfig.BulkBatchSize, MongodbConfig.DefaultBulkBatchSize)
33 |
34 | private val pkConfig: Option[Array[String]] = config.get[Array[String]](MongodbConfig.UpdateFields)
35 |
36 | override def save(it: Iterator[DBObject], mongoClient: MongoClient): Unit = {
37 | it.grouped(bulkBatchSize).foreach { group =>
38 | val bulkOperation = dbCollection(mongoClient).initializeUnorderedBulkOperation
39 | group.foreach { element =>
40 | val query = getUpdateQuery(element)
41 | if (query.isEmpty) bulkOperation.insert(element)
42 | else bulkOperation.find(query).upsert().replaceOne(element)
43 | }
44 |
45 | bulkOperation.execute(writeConcern)
46 | }
47 | }
48 |
49 | private def getUpdateQuery(element: DBObject): Map[String, AnyRef] = {
50 | if(element.contains(IdKey)) Map(IdKey -> element.get(IdKey))
51 | else {
52 | val pkValues : Map[String, AnyRef] =
53 | if (pkConfig.isDefined)
54 | pkConfig.get.flatMap(field => if (element.contains(field)) Some(field -> element.get(field)) else None).toMap
55 | else Map.empty[String, AnyRef]
56 | pkValues
57 | }
58 | }
59 | }
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/rdd/MongodbRDDIterator.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.rdd
17 |
18 | import com.mongodb.casbah.Imports._
19 | import com.stratio.datasource.mongodb.query.FilterSection
20 | import com.stratio.datasource.mongodb.reader.MongodbReader
21 | import com.stratio.datasource.util.Config
22 | import org.apache.spark._
23 | import org.apache.spark.sql.sources.Filter
24 |
25 | /**
26 | * MongoRDD values iterator.
27 | *
28 | * @param taskContext Spark task context.
29 | * @param partition Spark partition.
30 | * @param config Configuration object.
31 | * @param requiredColumns Pruning fields
32 | * @param filters Added query filters
33 | */
34 | class MongodbRDDIterator(
35 | taskContext: TaskContext,
36 | partition: Partition,
37 | config: Config,
38 | requiredColumns: Array[String],
39 | filters: FilterSection)
40 | extends Iterator[DBObject] {
41 |
42 | private var closed = false
43 | private var initialized = false
44 |
45 | lazy val reader = {
46 | initialized = true
47 | initReader()
48 | }
49 |
50 | // Register an on-task-completion callback to close the input stream.
51 | taskContext.addTaskCompletionListener((context: TaskContext) => closeIfNeeded())
52 |
53 | override def hasNext: Boolean = {
54 | !closed && reader.hasNext
55 | }
56 |
57 | override def next(): DBObject = {
58 | if (!hasNext) {
59 | throw new NoSuchElementException("End of stream")
60 | }
61 | reader.next()
62 | }
63 |
64 | def closeIfNeeded(): Unit = {
65 | if (!closed) {
66 | closed = true
67 | close()
68 | }
69 | }
70 |
71 | protected def close(): Unit = {
72 | if (initialized) {
73 | reader.close()
74 | initialized = false
75 | }
76 | }
77 |
78 | def initReader() = {
79 | val reader = new MongodbReader(config, requiredColumns, filters)
80 | reader.init(partition)
81 | reader
82 | }
83 | }
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/writer/MongodbWriter.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.writer
17 |
18 | import com.mongodb.casbah.Imports._
19 | import com.stratio.datasource.mongodb.client.MongodbClientFactory
20 | import com.stratio.datasource.mongodb.config.{MongodbConfigReader, MongodbConfig}
21 | import com.stratio.datasource.mongodb.util.usingMongoClient
22 | import com.stratio.datasource.util.Config
23 |
24 | /**
25 | * Abstract Mongodb writer.
26 | * Used for saving a bunch of mongodb objects
27 | * into specified database and collection
28 | *
29 | * @param config Configuration parameters (host,database,collection,...)
30 | */
31 | protected[mongodb] abstract class MongodbWriter(config: Config) extends Serializable {
32 |
33 | import MongodbConfigReader._
34 |
35 | private val languageConfig = config.get[String](MongodbConfig.Language)
36 |
37 | private val connectionsTime = config.get[String](MongodbConfig.ConnectionsTime).map(_.toLong)
38 |
39 | protected val writeConcern = config.writeConcern
40 |
41 | /**
42 | * A MongoDB collection created from the specified database and collection.
43 | */
44 | protected def dbCollection(mongoClient: MongoClient): MongoCollection =
45 | mongoClient(config(MongodbConfig.Database))(config(MongodbConfig.Collection))
46 |
47 | /**
48 | * Abstract method that checks if a primary key exists in provided configuration
49 | * and the language parameter.
50 | * Then calls the 'save' method.
51 | *
52 | * @param it DBObject iterator.
53 | */
54 | def saveWithPk(it: Iterator[DBObject]): Unit = {
55 | val itModified = if (languageConfig.isDefined) {
56 | it.map {
57 | case obj: BasicDBObject =>
58 | if (languageConfig.isDefined) obj.append("language", languageConfig.get)
59 | obj
60 | }
61 | } else it
62 |
63 | usingMongoClient(MongodbClientFactory.getClient(config.hosts, config.credentials, config.sslOptions, config.clientOptions).clientConnection) { mongoClient =>
64 | save(itModified, mongoClient: MongoClient)
65 | }
66 | }
67 |
68 | /**
69 | * Abstract method for storing a bunch of MongoDB objects.
70 | *
71 | * @param it Iterator of mongodb objects.
72 | */
73 | def save(it: Iterator[DBObject], mongoClient: MongoClient): Unit
74 |
75 | }
--------------------------------------------------------------------------------
/spark-mongodb/src/test/scala/com/stratio/datasource/mongodb/config/ConfigTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.config
17 |
18 | import com.stratio.datasource.util.{Config, ConfigBuilder}
19 | import com.stratio.datasource.MongodbTestConstants
20 | import Config.Property
21 | import org.junit.runner.RunWith
22 | import org.scalatest.junit.JUnitRunner
23 | import org.scalatest.{FlatSpec, Matchers}
24 |
25 | @RunWith(classOf[JUnitRunner])
26 | class ConfigTest extends FlatSpec
27 | with Matchers
28 | with ConfigHelpers
29 | with MongodbTestConstants{
30 |
31 | behavior of "ConfigBuilder"
32 |
33 | it should "config a builder with any kind of property types" + scalaBinaryVersion in {
34 |
35 | val b = (Builder() /: desiredProps.toList){
36 | case (builder,(property,propValue)) => builder.set(property,propValue)
37 | }
38 |
39 | b.properties.toList.diff(desiredProps.toList) should equal(Nil)
40 |
41 | }
42 |
43 | it should "build a deep config with configured properties" + scalaBinaryVersion in {
44 |
45 | val b = (Builder() /: desiredProps.toList){
46 | case (builder,(property,propValue)) => builder.set(property,propValue)
47 | }
48 |
49 | b.build().properties.toList.diff(desiredProps.toList) should equal(Nil)
50 |
51 | }
52 |
53 | it should "fail at getting any property with the wrong expected type" + scalaBinaryVersion in {
54 |
55 | val config = Builder().set("prop1",1).set("prop2",new { val x = 1}).build()
56 |
57 | a [ClassCastException] should be thrownBy {
58 | config[Int]("prop2")
59 | }
60 |
61 | }
62 |
63 | it should "fail at building time if any required property is not defined" + scalaBinaryVersion in {
64 |
65 | a [IllegalArgumentException] should be thrownBy {
66 | Builder()
67 | .set("prop1",1)
68 | .build()
69 | }
70 |
71 | }
72 |
73 | }
74 | trait ConfigHelpers {
75 |
76 | case class Builder(
77 | override val properties: Map[Property,Any]=Map()) extends ConfigBuilder[Builder]{
78 | val requiredProperties: List[Property] = List("prop1","prop2")
79 | def apply(props: Map[Property, Any]): Builder =
80 | new Builder(props)
81 | }
82 |
83 | // sample values
84 |
85 | val desiredProps = Map(
86 | "prop1" ->1,
87 | "prop2" -> "hi",
88 | "prop3" -> 1.0d,
89 | "prop4" -> new { val x = 5 })
90 |
91 | }
--------------------------------------------------------------------------------
/spark-mongodb-examples/src/main/scala/com/stratio/datasource/mongodb/examples/ExampleUtils.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.examples
17 |
18 | import com.mongodb.QueryBuilder
19 | import com.mongodb.casbah.MongoClient
20 | import com.mongodb.casbah.commons.{MongoDBList, MongoDBObject}
21 | import com.stratio.datasource.mongodb.examples.DataFrameAPIExample._
22 | import org.apache.spark.sql.SQLContext
23 | import org.apache.spark.{SparkConf, SparkContext}
24 |
25 | trait MongoDefaultConstants {
26 | val Database = "highschool"
27 | val Collection = "students"
28 | val MongoHost = "127.0.0.1"
29 | val MongoPort = 27017
30 | val MongoProvider = "com.stratio.datasource.mongodb"
31 | }
32 |
33 | object MongoExampleFunctions {
34 |
35 | def withSQLContext(block: SQLContext => Unit) = {
36 |
37 | val sparkConf = new SparkConf().
38 | setAppName("MongoDFExample").
39 | setMaster("local[4]")
40 |
41 | val sc = new SparkContext(sparkConf)
42 | try {
43 | val sqlContext = new SQLContext(sc)
44 | block(sqlContext)
45 | } finally {
46 | sc.stop()
47 | }
48 |
49 | }
50 |
51 | def prepareEnvironment(): MongoClient = {
52 | val mongoClient = MongoClient(MongoHost, MongoPort)
53 | populateTable(mongoClient)
54 | mongoClient
55 | }
56 |
57 | def cleanEnvironment(mongoClient: MongoClient) = {
58 | cleanData(mongoClient)
59 | mongoClient.close()
60 | }
61 |
62 | private def populateTable(client: MongoClient): Unit = {
63 |
64 | val collection = client(Database)(Collection)
65 | for (a <- 1 to 10) {
66 | collection.insert {
67 | MongoDBObject("id" -> a.toString,
68 | "age" -> (10 + a),
69 | "description" -> s"description $a",
70 | "enrolled" -> (a % 2 == 0),
71 | "name" -> s"Name $a"
72 | )
73 | }
74 | }
75 |
76 | collection.update(QueryBuilder.start("age").greaterThan(14).get, MongoDBObject(("$set", MongoDBObject(("optionalField", true)))), multi = true)
77 | collection.update(QueryBuilder.start("age").is(14).get, MongoDBObject(("$set", MongoDBObject(("fieldWithSubDoc", MongoDBObject(("subDoc", MongoDBList("foo", "bar"))))))))
78 | }
79 |
80 | private def cleanData(client: MongoClient): Unit = {
81 | val collection = client(Database)(Collection)
82 | collection.dropCollection()
83 | }
84 | }
85 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # Spark-Mongodb
2 |
3 | [](https://gitter.im/Stratio/Spark-MongoDB?utm_source=badge&utm_medium=badge&utm_campaign=pr-badge&utm_content=badge)
4 |
5 | Spark-Mongodb is a library that allows the user to read/write data with [Spark SQL](http://spark.apache.org/docs/latest/sql-programming-guide.html)
6 | from/into MongoDB collections.
7 |
8 | If you are using this Data Source, feel free to briefly share your experience by Pull Request this [file](https://github.com/Stratio/spark-mongodb/blob/master/doc/src/site/sphinx/PoweredBy.rst).
9 |
10 | ## Requirements##
11 |
12 | This library requires Apache Spark, Scala 2.10 or Scala 2.11, Casbah 2.8.X
13 |
14 | #### Latest compatible versions####
15 |
16 | | spark-MongoDB | Apache Spark | MongoDB |
17 | | ----------------- | ------------- | -------- |
18 | | 0.12.x | 2.0.0 | 3.0.x |
19 | | 0.10.x - 0.11.x | 1.5.x | 3.0.x |
20 | | 0.8.2 - 0.9.2 | 1.4.0 | 3.0.x |
21 | | 0.8.1 | 1.3.0 | 3.0.x |
22 | | 0.8.0 | 1.2.1 | 3.0.x |
23 |
24 |
25 | ## How to use Spark-MongoDB##
26 |
27 | There also exists a [First Steps] () document where we show some simple examples.
28 |
29 | - [Using the library](https://github.com/Stratio/spark-mongodb/blob/master/doc/src/site/sphinx/First_Steps.rst#using-the-library)
30 | - [Configuration parameters](https://github.com/Stratio/spark-mongodb/blob/master/doc/src/site/sphinx/First_Steps.rst#configuration-parameters)
31 | - [Examples](https://github.com/Stratio/spark-mongodb/blob/master/doc/src/site/sphinx/First_Steps.rst#examples)
32 | - [Scala API](https://github.com/Stratio/spark-mongodb/blob/master/doc/src/site/sphinx/First_Steps.rst#scala-api)
33 | - [Python API](https://github.com/Stratio/spark-mongodb/blob/master/doc/src/site/sphinx/First_Steps.rst#python-api)
34 | - [Java API](https://github.com/Stratio/spark-mongodb/blob/master/doc/src/site/sphinx/First_Steps.rst#java-api)
35 | - [R API](https://github.com/Stratio/spark-mongodb/blob/master/doc/src/site/sphinx/First_Steps.rst#r-api)
36 | - [Faqs](https://github.com/Stratio/spark-mongodb/blob/master/doc/src/site/sphinx/faqs.rst)
37 |
38 |
39 |
40 | # License #
41 |
42 | Licensed to STRATIO (C) under one or more contributor license agreements.
43 | See the NOTICE file distributed with this work for additional information
44 | regarding copyright ownership. The STRATIO (C) licenses this file
45 | to you under the Apache License, Version 2.0 (the
46 | "License"); you may not use this file except in compliance
47 | with the License. You may obtain a copy of the License at
48 |
49 | http://www.apache.org/licenses/LICENSE-2.0
50 |
51 | Unless required by applicable law or agreed to in writing,
52 | software distributed under the License is distributed on an
53 | "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
54 | KIND, either express or implied. See the License for the
55 | specific language governing permissions and limitations
56 | under the License.
57 |
58 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/DefaultSource.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb
17 |
18 | import com.stratio.datasource.mongodb.config.MongodbConfigBuilder
19 | import com.stratio.datasource.mongodb.config.MongodbConfig._
20 | import org.apache.spark.sql.SaveMode._
21 | import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, RelationProvider, SchemaRelationProvider}
22 | import org.apache.spark.sql.types.StructType
23 | import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
24 |
25 | /**
26 | * Allows creation of MongoDB based tables using
27 | * the syntax CREATE TEMPORARY TABLE ... USING com.stratio.deep.mongodb.
28 | * Required options are detailed in [[com.stratio.datasource.mongodb.config.MongodbConfig]]
29 | */
30 | class DefaultSource extends RelationProvider with SchemaRelationProvider with CreatableRelationProvider{
31 |
32 | override def createRelation(
33 | sqlContext: SQLContext,
34 | parameters: Map[String, String]): BaseRelation = {
35 |
36 | new MongodbRelation(MongodbConfigBuilder(parseParameters(parameters)).build())(sqlContext)
37 |
38 | }
39 |
40 | override def createRelation(
41 | sqlContext: SQLContext,
42 | parameters: Map[String, String],
43 | schema: StructType): BaseRelation = {
44 |
45 | new MongodbRelation(MongodbConfigBuilder(parseParameters(parameters)).build(), Some(schema))(sqlContext)
46 |
47 | }
48 |
49 | override def createRelation(
50 | sqlContext: SQLContext,
51 | mode: SaveMode,
52 | parameters: Map[String, String],
53 | data: DataFrame): BaseRelation = {
54 |
55 | val mongodbRelation = new MongodbRelation(
56 | MongodbConfigBuilder(parseParameters(parameters)).build(), Some(data.schema))(sqlContext)
57 |
58 | mode match{
59 | case Append => mongodbRelation.insert(data, overwrite = false)
60 | case Overwrite => mongodbRelation.insert(data, overwrite = true)
61 | case ErrorIfExists => if(mongodbRelation.isEmptyCollection) mongodbRelation.insert(data, overwrite = false)
62 | else throw new UnsupportedOperationException("Writing in a non-empty collection.")
63 | case Ignore => if(mongodbRelation.isEmptyCollection) mongodbRelation.insert(data, overwrite = false)
64 | }
65 |
66 | mongodbRelation
67 | }
68 |
69 | }
70 |
--------------------------------------------------------------------------------
/spark-mongodb/src/test/scala/com/stratio/datasource/mongodb/MongoEmbedDatabase.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb
17 |
18 | import com.mongodb.{DBCollection, DBObject, MongoClient}
19 | import com.stratio.datasource.MongodbTestConstants
20 | import de.flapdoodle.embed.mongo.config.{MongodConfigBuilder, Net, RuntimeConfigBuilder}
21 | import de.flapdoodle.embed.mongo.distribution.{IFeatureAwareVersion, Version}
22 | import de.flapdoodle.embed.mongo.{Command, MongodExecutable, MongodProcess, MongodStarter}
23 | import de.flapdoodle.embed.process.config.IRuntimeConfig
24 | import de.flapdoodle.embed.process.config.io.ProcessOutput
25 | import de.flapdoodle.embed.process.runtime.Network
26 |
27 | trait MongoEmbedDatabase extends MongodbTestConstants {
28 | private val runtimeConfig = new RuntimeConfigBuilder()
29 | .defaults(Command.MongoD)
30 | .processOutput(ProcessOutput.getDefaultInstanceSilent)
31 | .build()
32 |
33 | protected def mongoStart(port: Int = mongoPort,
34 | version: IFeatureAwareVersion = Version.Main.PRODUCTION,
35 | runtimeConfig: IRuntimeConfig = runtimeConfig): MongodProps = {
36 | val mongodExe: MongodExecutable = mongodExec(port, version, runtimeConfig)
37 | MongodProps(mongodExe.start(), mongodExe)
38 | }
39 |
40 | protected def mongoStop( mongodProps: MongodProps ) = {
41 | Option(mongodProps).foreach( _.mongodProcess.stop() )
42 | Option(mongodProps).foreach( _.mongodExe.stop() )
43 | }
44 |
45 | protected def withEmbedMongoFixture(dataset: List[DBObject],
46 | port: Int = mongoPort,
47 | version: IFeatureAwareVersion = Version.Main.PRODUCTION,
48 | runtimeConfig: IRuntimeConfig = runtimeConfig)
49 | (fixture: MongodProps => Any) {
50 | val mongodProps = mongoStart(port, version, runtimeConfig)
51 | if (!dataset.isEmpty) populateDatabase(port, dataset)
52 | try { fixture(mongodProps) } finally { Option(mongodProps).foreach( mongoStop ) }
53 | }
54 |
55 | private def runtime(config: IRuntimeConfig): MongodStarter = MongodStarter.getInstance(config)
56 |
57 | private def mongodExec(port: Int, version: IFeatureAwareVersion, runtimeConfig: IRuntimeConfig): MongodExecutable =
58 | runtime(runtimeConfig).prepare(
59 | new MongodConfigBuilder()
60 | .version(version)
61 | .net(new Net(port, Network.localhostIsIPv6()))
62 | .build()
63 | )
64 |
65 | private def populateDatabase(port: Int, dataset: List[DBObject]) = {
66 | import scala.collection.JavaConverters._
67 |
68 | val mongo: MongoClient = new MongoClient("localhost", port)
69 | val col: DBCollection = mongo.getDB(db).getCollection("testCol")
70 | col.insert(dataset.asJava)
71 | }
72 | }
73 |
74 | case class MongodProps(mongodProcess: MongodProcess, mongodExe: MongodExecutable)
--------------------------------------------------------------------------------
/doc/pom.xml:
--------------------------------------------------------------------------------
1 |
18 |
21 | 4.0.0
22 | com.stratio.spark-mongodb.doc
23 | spark-mongodb-doc
24 | Stratio Spark-MongoDB documentation
25 | pom
26 |
27 | com.stratio.datasource
28 | spark-mongodb-parent
29 | 0.13.0-SNAPSHOT
30 |
31 |
32 | true
33 | ${project.version}
34 |
35 |
36 |
37 |
38 | org.apache.maven.plugins
39 | maven-deploy-plugin
40 | 2.8.2
41 |
42 | true
43 |
44 |
45 |
46 | org.apache.maven.plugins
47 | maven-site-plugin
48 | 3.4
49 |
50 |
51 |
52 | org.apache.maven.plugins
53 | maven-project-info-reports-plugin
54 | 2.8
55 |
56 |
57 |
58 |
59 |
60 |
61 |
62 | com.stratio.maven
63 | sphinx-maven-plugin
64 | 2.0.0
65 |
66 |
67 | html
68 |
69 |
70 |
71 | src/site/sphinx
72 | true
73 |
74 | conf.py
75 |
76 |
77 |
78 |
79 |
80 |
81 |
82 |
83 |
84 |
85 |
86 |
--------------------------------------------------------------------------------
/spark-mongodb/src/test/scala/com/stratio/datasource/mongodb/MongoClusterEmbedDatabase.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb
17 |
18 | import com.mongodb.casbah.commons.MongoDBObject
19 | import com.mongodb.{ServerAddress, DBCollection, MongoClient, DBObject}
20 | import com.stratio.datasource.MongodbTestConstants
21 | import de.flapdoodle.embed.mongo.config._
22 | import de.flapdoodle.embed.mongo.distribution.Version
23 | import de.flapdoodle.embed.mongo.tests.MongosSystemForTestFactory
24 | import de.flapdoodle.embed.process.runtime.Network
25 |
26 | import scala.collection.JavaConversions._
27 | import scala.collection.JavaConverters._
28 |
29 | /**
30 | * Deploys an embedded cluster composed by:
31 | * - A mongo
32 | * - A list of config. servers
33 | * - A list of replica sets (Mongods)
34 | */
35 | trait MongoClusterEmbedDatabase extends MongodbTestConstants {
36 |
37 | // Current system
38 |
39 | type Port = Int
40 | type Host = String
41 | type MB = Int
42 | type ReplicaSetName = String
43 |
44 | val configServerPorts: List[Port]
45 | val database: String
46 | val collection: String
47 | val shardKey: String
48 | val shardMaxSize: MB
49 | val chunkSize: MB
50 | val mongoPort: Port
51 | val currentHost: Host
52 | val replicaSets: Map[ReplicaSetName, List[Port]]
53 |
54 | lazy val databaseConnection = replicaSets.flatMap {
55 | case (replicaSet, port :: restPorts) =>
56 | s"$replicaSet/$currentHost:$port" :: restPorts.map { p =>
57 | s"$currentHost:$p"
58 | }
59 | case _ => ""
60 | }.mkString(",")
61 |
62 | lazy val system = new MongosSystemForTestFactory(
63 | mongoConfig(
64 | currentHost,
65 | configServerPorts.map(p => s"$currentHost:$p").mkString(","))(mongoPort),
66 | replicaSets.map {
67 | case (rs, ports) =>
68 | rs -> shardConfig(rs, ports.map(p => (currentHost, p))).asJava
69 | },
70 | configServerPorts.map(mongodConfig(currentHost, _)),
71 | database,
72 | collection,
73 | shardKey)
74 |
75 | // Config builders
76 |
77 | private def mongoConfig(
78 | host: String,
79 | databaseLocation: String)(mongoPort: Int) =
80 | new MongosConfigBuilder()
81 | .version(Version.Main.PRODUCTION)
82 | .net(new Net(host, mongoPort, Network.localhostIsIPv6()))
83 | .configDB(databaseLocation)
84 | .build()
85 |
86 | private def mongodConfig(host: String, mongodPort: Int) =
87 | new MongodConfigBuilder()
88 | .version(Version.Main.PRODUCTION)
89 | .configServer(true)
90 | .net(new Net(host, mongodPort, Network.localhostIsIPv6()))
91 | .build()
92 |
93 | private def shardConfig(
94 | replicaSet: String,
95 | shardPorts: List[(String, Int)]): List[IMongodConfig] =
96 | shardPorts.map { case (host, port) =>
97 | new MongodConfigBuilder()
98 | .version(Version.Main.PRODUCTION)
99 | .replication(new Storage(null, replicaSet, shardMaxSize))
100 | .net(new Net(host, port, Network.localhostIsIPv6()))
101 | .build()
102 | }
103 |
104 | // Helpers
105 |
106 | protected def withCluster[T](
107 | f: MongosSystemForTestFactory => T): T = {
108 | system.start()
109 | val mongo = client()
110 | mongo.getDB(database).command(MongoDBObject("chunksize" -> chunkSize))
111 | val t = f(system)
112 | system.stop()
113 | t
114 | }
115 |
116 | protected def client(): MongoClient =
117 | new MongoClient(
118 | replicaSets.values.flatMap(ports =>
119 | ports.map(port =>
120 | new ServerAddress(currentHost, port))).toList)
121 |
122 |
123 | protected def populateDatabase(dataset: List[DBObject]) {
124 | import scala.collection.JavaConverters._
125 |
126 | val mongo = client()
127 | val col: DBCollection = mongo.getDB(database).getCollection(collection)
128 | col.insert(dataset.asJava)
129 | }
130 |
131 | }
--------------------------------------------------------------------------------
/spark-mongodb/src/test/scala/com/stratio/datasource/mongodb/schema/MongodbSchemaIT.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.schema
17 |
18 | import java.text.SimpleDateFormat
19 | import java.util.Locale
20 |
21 | import com.stratio.datasource.MongodbTestConstants
22 | import com.stratio.datasource.mongodb.config.{MongodbConfig, MongodbConfigBuilder}
23 | import com.stratio.datasource.mongodb.partitioner.MongodbPartitioner
24 | import com.stratio.datasource.mongodb.rdd.MongodbRDD
25 | import com.stratio.datasource.mongodb._
26 | import org.apache.spark.sql.mongodb.{TemporaryTestSQLContext, TestSQLContext}
27 | import org.apache.spark.sql.types.{ArrayType, StringType, StructField, TimestampType}
28 | import org.junit.runner.RunWith
29 | import org.scalatest._
30 | import org.scalatest.junit.JUnitRunner
31 |
32 | @RunWith(classOf[JUnitRunner])
33 | class MongodbSchemaIT extends FlatSpec
34 | with Matchers
35 | with MongoEmbedDatabase
36 | with TestBsonData
37 | with MongodbTestConstants {
38 |
39 | private val host: String = "localhost"
40 | private val collection: String = "testCol"
41 | private val readPreference = "secondaryPreferred"
42 |
43 | val testConfig = MongodbConfigBuilder()
44 | .set(MongodbConfig.Host,List(host + ":" + mongoPort))
45 | .set(MongodbConfig.Database,db)
46 | .set(MongodbConfig.Collection,collection)
47 | .set(MongodbConfig.SamplingRatio,1.0)
48 | .set(MongodbConfig.ReadPreference, readPreference)
49 | .build()
50 |
51 | val mongodbPartitioner = new MongodbPartitioner(testConfig)
52 |
53 | val mongodbRDD = new MongodbRDD(TemporaryTestSQLContext, testConfig, mongodbPartitioner)
54 |
55 | behavior of "A schema"
56 |
57 | it should "be inferred from rdd with primitives" + scalaBinaryVersion in {
58 | withEmbedMongoFixture(primitiveFieldAndType) { mongodProc =>
59 | val schema = MongodbSchema(mongodbRDD, 1.0).schema()
60 |
61 | schema.fields should have size 7
62 | schema.fieldNames should contain allOf("string", "integer", "long", "double", "boolean", "null")
63 |
64 | schema.printTreeString()
65 | }
66 | }
67 |
68 | it should "be inferred from rdd with complex fields" + scalaBinaryVersion in {
69 | withEmbedMongoFixture(complexFieldAndType1) { mongodProc =>
70 | val schema = MongodbSchema(mongodbRDD, 1.0).schema()
71 |
72 | schema.fields should have size 13
73 |
74 | schema.fields filter {
75 | case StructField(name, ArrayType(StringType, _), _, _) => Set("arrayOfNull", "arrayEmpty") contains name
76 | case _ => false
77 | } should have size 2
78 |
79 | schema.printTreeString()
80 | }
81 | }
82 |
83 | it should "resolve type conflicts between fields" + scalaBinaryVersion in {
84 | withEmbedMongoFixture(primitiveFieldValueTypeConflict) { mongodProc =>
85 | val schema = MongodbSchema(mongodbRDD, 1.0).schema()
86 |
87 | schema.fields should have size 7
88 |
89 | schema.printTreeString()
90 | }
91 | }
92 |
93 | it should "be inferred from rdd with more complex fields" + scalaBinaryVersion in {
94 | withEmbedMongoFixture(complexFieldAndType2) { mongodProc =>
95 | val schema = MongodbSchema(mongodbRDD, 1.0).schema()
96 |
97 | schema.fields should have size 5
98 |
99 | schema.printTreeString()
100 | }
101 | }
102 |
103 | it should "read java.util.Date fields as timestamptype" + scalaBinaryVersion in {
104 | val dfunc = (s: String) => new SimpleDateFormat("EEE MMM dd HH:mm:ss Z yyyy", Locale.ENGLISH).parse(s)
105 | import com.mongodb.casbah.Imports.DBObject
106 | val stringAndDate = List(DBObject("string" -> "this is a simple string.", "date" -> dfunc("Mon Aug 10 07:52:49 EDT 2015")))
107 | withEmbedMongoFixture(stringAndDate) { mongodProc =>
108 | val schema = MongodbSchema(mongodbRDD, 1.0).schema()
109 |
110 | schema.fields should have size 3
111 | schema.fields.filter(_.name == "date").head.dataType should equal(TimestampType)
112 | schema.printTreeString()
113 | }
114 | }
115 | }
116 |
--------------------------------------------------------------------------------
/spark-mongodb-examples/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
19 |
21 |
22 | com.stratio
23 | parent
24 | 0.4.0
25 |
26 |
27 | 4.0.0
28 | spark-mongodb-examples
29 | spark-mongodb examples
30 | MongoDB examples
31 | http://github.com/Stratio/spark-mongodb
32 | jar
33 |
34 |
35 |
36 | Apache License, Version 2.0
37 | http://www.apache.org/licenses/LICENSE-2.0
38 |
39 |
40 |
41 |
42 | 2.10
43 | 0.12.0-SNAPSHOT
44 | 1.5.2
45 |
46 |
47 |
48 |
49 | com.stratio.datasource
50 | spark-mongodb_${scala.binary.version}
51 | ${mongodb.datasource.version}
52 |
53 |
54 | org.apache.spark
55 | spark-sql_${scala.binary.version}
56 | ${spark.version}
57 |
58 |
59 |
60 |
61 | src/main/scala
62 |
63 |
64 | org.scala-tools
65 | maven-scala-plugin
66 | 2.15.2
67 |
68 |
69 | org.apache.maven.plugins
70 | maven-compiler-plugin
71 | 3.3
72 |
73 | true
74 | 1.7
75 | 1.7
76 |
77 |
78 |
79 | net.alchim31.maven
80 | scala-maven-plugin
81 | 3.2.0
82 |
83 | false
84 |
85 | -Xmax-classfile-name
86 | 130
87 |
88 |
89 |
90 |
91 | compile
92 |
93 | compile
94 |
95 | compile
96 |
97 |
98 | test-compile
99 |
100 | testCompile
101 |
102 | test-compile
103 |
104 |
105 | process-resources
106 |
107 | compile
108 |
109 |
110 |
111 |
112 |
113 |
114 |
115 |
116 |
117 |
118 |
--------------------------------------------------------------------------------
/spark-mongodb/src/test/scala/com/stratio/datasource/mongodb/partitioner/MongodbPartitionerIT.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.partitioner
17 |
18 | import com.mongodb.DBObject
19 | import com.mongodb.util.JSON
20 | import com.stratio.datasource.MongodbTestConstants
21 | import com.stratio.datasource.mongodb._
22 | import com.stratio.datasource.mongodb.client.MongodbClientFactory
23 | import com.stratio.datasource.mongodb.config.{MongodbConfig, MongodbConfigBuilder}
24 | import com.stratio.datasource.mongodb.config.MongodbConfig._
25 | import org.junit.runner.RunWith
26 | import org.scalatest.junit.JUnitRunner
27 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, Matchers, FlatSpec}
28 |
29 | @RunWith(classOf[JUnitRunner])
30 | class MongodbPartitionerIT extends FlatSpec
31 | with BeforeAndAfter
32 | with Matchers
33 | with MongoClusterEmbedDatabase
34 | with MongoEmbedDatabase
35 | with TestBsonData
36 | with MongodbTestConstants
37 | with BeforeAndAfterAll {
38 |
39 | val configServerPorts = List(mongoPort+10)
40 | val database = "database-1"
41 | val collection = "collection-1"
42 | val shardKey = "_id"
43 | val shardMaxSize = 1
44 | val chunkSize = 1
45 | val currentHost = "localhost"
46 | val replicaSets = Map(
47 | "replicaSet1" -> List(mongoPort+1, mongoPort+2, mongoPort+3),
48 | "replicaSet2" -> List(mongoPort+4, mongoPort+5, mongoPort+6))
49 |
50 | behavior of "MongodbPartitioner"
51 | it should "get proper partition ranges when connecting" + " to a sharded cluster" + scalaBinaryVersion in {
52 |
53 | val testConfig = MongodbConfigBuilder()
54 | .set(MongodbConfig.Host, replicaSets.values.flatMap(
55 | ports => ports.take(1).map(
56 | p => s"$currentHost:$p")))
57 | .set(MongodbConfig.Database, database)
58 | .set(MongodbConfig.Collection, collection)
59 | .set(MongodbConfig.SamplingRatio, 1.0)
60 | .build()
61 |
62 | withCluster { system =>
63 | val partitioner1 = new MongodbPartitioner(testConfig)
64 | val (partition :: Nil) = partitioner1.computePartitions().toList
65 | partition.index should equal(0)
66 | partition.partitionRange.minKey should equal(None)
67 | partition.partitionRange.maxKey should equal(None)
68 | //TODO: Check what happens when shard is enable due to get over max chunk size
69 | }
70 |
71 | }
72 |
73 | def objectSample(amount: Long): Stream[DBObject] = {
74 | def longs: Stream[Long] = 0 #:: longs.map(_ + 1)
75 | longs.map { n =>
76 | n -> JSON.parse(
77 | s"""{"string":"this is a simple string.",
78 | "integer":10,
79 | "long":$n,
80 | "double":1.7976931348623157E308,
81 | "boolean":true,
82 | "null":null
83 | }""").asInstanceOf[DBObject]
84 | }.takeWhile {
85 | case (idx, _) => idx <= amount
86 | }.map(_._2)
87 | }
88 |
89 | override def afterAll {
90 | MongodbClientFactory.closeAll(false)
91 | }
92 |
93 | it should "get proper partition ranges using splitVector with bounds" + scalaBinaryVersion in {
94 |
95 | import com.mongodb.casbah.Imports.MongoDBObject
96 | val dataSet = (1 to 15000).map(n=> MongoDBObject("name" -> s"name$n" , "id" -> n)).toList
97 |
98 | withEmbedMongoFixture(dataSet) { mongoProc =>
99 | val mongoClient = com.mongodb.casbah.MongoClient("localhost", mongoPort)
100 |
101 | val coll = mongoClient(db)("testCol"
102 | )
103 | // to run splitVector index by the splitKey field is needed
104 | coll.createIndex(MongoDBObject("id" ->1))
105 |
106 | val testConfig = MongodbConfigBuilder(Map(
107 | Host -> List(s"localhost:$mongoPort"),
108 | Database -> db,
109 | Collection -> "testCol",
110 | SamplingRatio -> 1.0,
111 | SplitSize -> 1,
112 | SplitKey -> "id",
113 | SplitSize -> "1",
114 | SplitKeyType -> "int",
115 | SplitKeyMin -> "500",
116 | SplitKeyMax -> "14000")
117 | ).build()
118 |
119 | val partitioner = new MongodbPartitioner(testConfig)
120 | val partitions = partitioner.computePartitions().toList.size
121 |
122 | //With the dataSet for this test and this splitVector config, 3 partitions would be created
123 | partitions should equal(3)
124 | }
125 | }
126 |
127 |
128 | }
--------------------------------------------------------------------------------
/spark-mongodb/src/test/scala/com/stratio/datasource/mongodb/MongodbRelationIT.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb
17 |
18 | import com.mongodb.WriteConcern
19 | import com.stratio.datasource.MongodbTestConstants
20 | import com.stratio.datasource.mongodb.client.MongodbClientFactory
21 | import com.stratio.datasource.mongodb.config.{MongodbCredentials, MongodbConfig, MongodbConfigBuilder}
22 | import org.apache.spark.sql.mongodb.{TemporaryTestSQLContext, TestSQLContext}
23 | import org.apache.spark.sql.types._
24 | import org.junit.runner.RunWith
25 | import org.scalatest.junit.JUnitRunner
26 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FlatSpec, Matchers}
27 |
28 | @RunWith(classOf[JUnitRunner])
29 | class MongodbRelationIT extends FlatSpec
30 | with Matchers
31 | with MongodbTestConstants
32 | with BeforeAndAfterAll {
33 |
34 | private val host: String = "localhost"
35 | private val port: Int = 12345
36 | private val port2: Int = 67890
37 | private val database: String = "testDb"
38 | private val database2: String = "testDb2"
39 | private val collection: String = "testCol"
40 | private val collection2: String = "testCol2"
41 | private val writeConcern : WriteConcern = WriteConcern.NORMAL
42 |
43 |
44 | val testConfig = MongodbConfigBuilder()
45 | .set(MongodbConfig.Host, List(host + ":" + port))
46 | .set(MongodbConfig.Database, database)
47 | .set(MongodbConfig.Collection, collection)
48 | .build()
49 |
50 | val testConfig2 = MongodbConfigBuilder()
51 | .set(MongodbConfig.Host, List(host + ":" + port))
52 | .set(MongodbConfig.Collection, collection)
53 | .set(MongodbConfig.Database, database)
54 | .build()
55 |
56 | val testConfig3 = MongodbConfigBuilder()
57 | .set(MongodbConfig.Collection, collection2)
58 | .set(MongodbConfig.Database, database2)
59 | .set(MongodbConfig.SamplingRatio, 1.0)
60 | .set(MongodbConfig.WriteConcern, writeConcern)
61 | .set(MongodbConfig.Host, List(host + ":" + port2))
62 | .build()
63 |
64 | val testConfig4 = MongodbConfigBuilder()
65 | .set(MongodbConfig.Host, List(host + ":" + port))
66 | .set(MongodbConfig.Database, database)
67 | .set(MongodbConfig.Collection, collection)
68 | .set(MongodbConfig.Credentials, List(MongodbCredentials("user","database", "password".toCharArray)))
69 | .build()
70 |
71 | val schema = new StructType(Array(new StructField(
72 | "att1",IntegerType,false),
73 | new StructField(
74 | "att2",DoubleType,false),
75 | new StructField(
76 | "att3",StringType,false),
77 | new StructField(
78 | "att4",StringType,true),
79 | new StructField(
80 | "att5",new ArrayType(IntegerType,false),false),
81 | new StructField(
82 | "att6",new StructType(Array(
83 | new StructField("att61",IntegerType ,false),
84 | new StructField("att62",IntegerType,true)
85 | )),false)))
86 |
87 | behavior of "MongodbRelation"
88 |
89 | it should "prune schema to adapt it to required columns" + scalaBinaryVersion in {
90 |
91 | MongodbRelation.pruneSchema(schema,Array[String]()) should equal(
92 | new StructType(Array()))
93 |
94 | MongodbRelation.pruneSchema(schema,Array("fakeAtt")) should equal(
95 | new StructType(Array()))
96 |
97 | MongodbRelation.pruneSchema(schema,Array("att1")) should equal(
98 | new StructType(Array(
99 | new StructField(
100 | "att1",IntegerType,false))))
101 |
102 | MongodbRelation.pruneSchema(schema,Array("att3","att1")) should equal(
103 | new StructType(Array(
104 | new StructField(
105 | "att3",StringType,false),
106 | new StructField(
107 | "att1",IntegerType,false))))
108 |
109 | }
110 |
111 | val mongodbrelation = new MongodbRelation(testConfig, Some(schema))(TemporaryTestSQLContext)
112 | val mongodbrelation2 = new MongodbRelation(testConfig2, Some(schema))(TemporaryTestSQLContext)
113 | val mongodbrelation3 = new MongodbRelation(testConfig3, Some(schema))(TemporaryTestSQLContext)
114 | val mongodbrelation4 = new MongodbRelation(testConfig4, Some(schema))(TemporaryTestSQLContext)
115 |
116 | it should "provide info about equality in MongodbRelation" + scalaBinaryVersion in {
117 | mongodbrelation.equals(mongodbrelation) shouldEqual true
118 | mongodbrelation.equals(mongodbrelation2) shouldEqual true
119 | mongodbrelation.equals(mongodbrelation3) shouldEqual false
120 | mongodbrelation.equals(mongodbrelation4) shouldEqual false
121 | }
122 |
123 | override def afterAll {
124 | MongodbClientFactory.closeAll(false)
125 | }
126 |
127 | }
128 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/util/Config.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.util
17 |
18 | import com.stratio.datasource.util.Config.Property
19 |
20 | import scala.reflect.ClassTag
21 |
22 | /**
23 | * Abstract config builder, used to set a bunch of properties a build
24 | * a config object from them.
25 | *
26 | * @param properties Map of any-type properties.
27 | * @tparam Builder Current Builder implementation type.
28 | */
29 | abstract class ConfigBuilder[Builder<:ConfigBuilder[Builder] ](
30 | val properties: Map[Property,Any] = Map()) extends Serializable { builder =>
31 |
32 | /**
33 | * Required properties to build a Mongo config object.
34 | * At build time, if these properties are not set, an assert
35 | * exception will be thrown.
36 | */
37 | val requiredProperties: List[Property]
38 |
39 | /**
40 | * Instantiate a brand new Builder from given properties map
41 | *
42 | * @param props Map of any-type properties.
43 | * @return The new builder
44 | */
45 | def apply(props: Map[Property,Any]): Builder
46 |
47 | /**
48 | * Set (override if exists) a single property value given a new one.
49 | *
50 | * @param property Property to be set
51 | * @param value New value for given property
52 | * @tparam T Property type
53 | * @return A new builder that includes new value of the specified property
54 | */
55 | def set[T](property: Property, value: T): Builder =
56 | apply(properties + (property -> value))
57 |
58 | /**
59 | * Build the config object from current builder properties.
60 | *
61 | * @return The Mongo configuration object.
62 | */
63 | def build(): Config = new Config {
64 |
65 | // TODO Review when refactoring config
66 | val properties = builder.properties.map { case (k, v) => k.toLowerCase -> v }
67 | val reqProperties = requiredProperties.map(_.toLowerCase)
68 |
69 | require(
70 | reqProperties.forall(properties.isDefinedAt),
71 | s"Not all properties are defined! : ${
72 | reqProperties.diff(
73 | properties.keys.toList.intersect(requiredProperties))
74 | }")
75 |
76 | /**
77 | * Compare if two Configs have the same properties.
78 | * @param other Object to compare
79 | * @return Boolean
80 | */
81 | override def equals(other: Any): Boolean = other match {
82 | case that: Config =>
83 | properties == that.properties
84 | case _ => false
85 | }
86 |
87 | override def hashCode(): Int = {
88 | val state = Seq(properties)
89 | state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b)
90 | }
91 |
92 | }
93 |
94 | }
95 |
96 | /**
97 | * Mongo standard configuration object
98 | */
99 | trait Config extends Serializable {
100 |
101 | /**
102 | * Contained properties in configuration object
103 | */
104 | val properties: Map[Property, Any]
105 |
106 | /** Returns the value associated with a key, or a default value if the key is not contained in the configuration object.
107 |
108 | * @param key Desired property.
109 | * @param default Value in case no binding for `key` is found in the map.
110 | * @tparam T Result type of the default computation.
111 | * @return the value associated with `key` if it exists,
112 | * otherwise the result of the `default` computation.
113 | */
114 | def getOrElse[T](key: Property, default: => T): T =
115 | properties.get(key.toLowerCase) collect { case v: T => v } getOrElse default
116 |
117 | /**
118 | * Gets specified property from current configuration object
119 | * @param property Desired property
120 | * @tparam T Property expected value type.
121 | * @return An optional value of expected type
122 | */
123 | def get[T: ClassTag](property: Property): Option[T] =
124 | properties.get(property.toLowerCase).map(_.asInstanceOf[T])
125 |
126 | /**
127 | * Gets specified property from current configuration object.
128 | * It will fail if property is not previously set.
129 | * @param property Desired property
130 | * @tparam T Property expected value type
131 | * @return Expected type value
132 | */
133 | def apply[T: ClassTag](property: Property): T = {
134 | get[T](property).get
135 | }
136 |
137 | }
138 |
139 | object Config {
140 |
141 | type Property = String
142 |
143 | /**
144 | * Defines how to act in case any parameter is not set
145 | * @param key Key that couldn't be obtained
146 | * @tparam T Expected type (used to fit in 'getOrElse' cases).
147 | * @return Throws an IllegalStateException.
148 | */
149 | def notFound[T](key: String): T =
150 | throw new IllegalStateException(s"Parameter $key not specified")
151 |
152 | }
153 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/reader/MongodbReader.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.reader
17 |
18 | import com.mongodb.casbah.Imports._
19 | import com.mongodb.casbah.MongoCursorBase
20 | import com.stratio.datasource.mongodb.client.MongodbClientFactory
21 | import com.stratio.datasource.mongodb.config.{MongodbConfig, MongodbCredentials, MongodbSSLOptions}
22 | import com.stratio.datasource.mongodb.partitioner.MongodbPartition
23 | import com.stratio.datasource.mongodb.query.FilterSection
24 | import com.stratio.datasource.util.Config
25 | import org.apache.spark.Partition
26 |
27 | import scala.util.Try
28 |
29 | /**
30 | *
31 | * @param config Configuration object.
32 | * @param requiredColumns Pruning fields
33 | * @param filters Added query filters
34 | */
35 | class MongodbReader(config: Config,
36 | requiredColumns: Array[String],
37 | filters: FilterSection) {
38 |
39 | private var mongoClient: Option[MongodbClientFactory.Client] = None
40 |
41 | private var mongoClientKey: Option[String] = None
42 |
43 | private var dbCursor: Option[MongoCursorBase] = None
44 |
45 | private val batchSize = config.getOrElse[Int](MongodbConfig.CursorBatchSize, MongodbConfig.DefaultCursorBatchSize)
46 |
47 | private val connectionsTime = config.get[String](MongodbConfig.ConnectionsTime).map(_.toLong)
48 |
49 |
50 | def close(): Unit = {
51 | dbCursor.fold(ifEmpty = ()) { cursor =>
52 | cursor.close()
53 | dbCursor = None
54 | }
55 |
56 | mongoClient.fold(ifEmpty = ()) { client =>
57 | mongoClientKey.fold({
58 | MongodbClientFactory.closeByClient(client)
59 | }) {key =>
60 | MongodbClientFactory.closeByKey(key)
61 | }
62 |
63 | mongoClient = None
64 | }
65 | }
66 |
67 | def hasNext: Boolean = {
68 | dbCursor.fold(ifEmpty = false)(cursor => cursor.hasNext)
69 | }
70 |
71 | def next(): DBObject = {
72 | dbCursor.fold(ifEmpty = throw new IllegalStateException("DbCursor is not initialized"))(cursor => cursor.next())
73 | }
74 |
75 | /**
76 | * Initialize MongoDB reader
77 | * @param partition Where to read from
78 | */
79 | def init(partition: Partition): Unit = {
80 | Try {
81 | val mongoPartition = partition.asInstanceOf[MongodbPartition]
82 | val hosts = mongoPartition.hosts.map(add => new ServerAddress(add)).toList
83 | val credentials = config.getOrElse[List[MongodbCredentials]](MongodbConfig.Credentials, MongodbConfig.DefaultCredentials).map {
84 | case MongodbCredentials(user, database, password) =>
85 | MongoCredential.createCredential(user, database, password)
86 | }
87 | val sslOptions = config.get[MongodbSSLOptions](MongodbConfig.SSLOptions)
88 | val clientOptions = config.properties.filterKeys(_.contains(MongodbConfig.ListMongoClientOptions))
89 |
90 | val mongoClientResponse = MongodbClientFactory.getClient(hosts, credentials, sslOptions, clientOptions)
91 | mongoClient = Option(mongoClientResponse.clientConnection)
92 | mongoClientKey = Option(mongoClientResponse.key)
93 |
94 | val emptyFilter = MongoDBObject(List())
95 | val filter = Try(queryPartition(filters)).getOrElse(emptyFilter)
96 |
97 | dbCursor = (for {
98 | client <- mongoClient
99 | collection <- Option(client(config(MongodbConfig.Database))(config(MongodbConfig.Collection)))
100 | dbCursor <- Option(collection.find(filter, selectFields(requiredColumns)))
101 | } yield {
102 | mongoPartition.partitionRange.minKey.foreach(min => dbCursor.addSpecial("$min", min))
103 | mongoPartition.partitionRange.maxKey.foreach(max => dbCursor.addSpecial("$max", max))
104 | dbCursor.batchSize(batchSize)
105 | }).headOption
106 | }.recover {
107 | case throwable =>
108 | throw MongodbReadException(throwable.getMessage, throwable)
109 | }
110 | }
111 |
112 | /**
113 | * Create query partition using given filters.
114 | *
115 | * @param filters the Spark filters to be converted to Mongo filters
116 | * @return the dB object
117 | */
118 | private def queryPartition(filters: FilterSection): DBObject = {
119 | implicit val c: Config = config
120 | filters.filtersToDBObject()
121 | }
122 |
123 | /**
124 | *
125 | * Prepared DBObject used to specify required fields in mongodb 'find'
126 | * @param fields Required fields
127 | * @return A mongodb object that represents required fields.
128 | */
129 | private def selectFields(fields: Array[String]): DBObject =
130 | MongoDBObject(
131 | if (fields.isEmpty) List()
132 | else fields.toList.filterNot(_ == "_id").map(_ -> 1) ::: {
133 | List("_id" -> fields.find(_ == "_id").fold(0)(_ => 1))
134 | })
135 | }
136 |
137 | case class MongodbReadException(
138 | msg: String,
139 | causedBy: Throwable) extends RuntimeException(msg, causedBy)
140 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/schema/MongodbSchema.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.schema
17 |
18 | import com.mongodb.casbah.Imports._
19 | import com.stratio.datasource.schema.SchemaProvider
20 | import org.apache.spark.rdd.RDD
21 | import org.apache.spark.sql.catalyst.analysis.TypeCoercion
22 | import org.apache.spark.sql.types._
23 | import org.apache.spark.unsafe.types.UTF8String
24 |
25 | /**
26 | * A custom RDD schema for MongoDB.
27 | * @param rdd RDD used to infer the schema
28 | * @param samplingRatio Sampling ratio used to scan the RDD and extract
29 | * used fields.
30 | */
31 | case class MongodbSchema[T <: RDD[DBObject]](
32 | rdd: T,
33 | samplingRatio: Double) extends SchemaProvider with Serializable {
34 |
35 | override def schema(): StructType = {
36 | val schemaData =
37 | if (samplingRatio > 0.99) rdd
38 | else rdd.sample(withReplacement = false, samplingRatio, 1)
39 |
40 | val structFields = schemaData.flatMap {
41 | dbo => {
42 | val doc: Map[String, AnyRef] = dbo.seq.toMap
43 | val fields = doc.mapValues(f => convertToStruct(f))
44 | fields
45 | }
46 | }.reduceByKey(compatibleType).aggregate(Seq[StructField]())({
47 | case (fields, (name, tpe)) =>
48 | val newType = tpe match {
49 | case ArrayType(NullType, containsNull) => ArrayType(StringType, containsNull)
50 | case other => other
51 | }
52 | fields :+ StructField(name, newType)
53 | }, (oldFields, newFields) => oldFields ++ newFields)
54 | StructType(structFields)
55 | }
56 |
57 | private def convertToStruct(dataType: Any): DataType = dataType match {
58 | case bl: BasicDBList =>
59 | typeOfArray(bl)
60 |
61 | case bo: DBObject =>
62 | val fields = bo.map {
63 | case (k, v) =>
64 | StructField(k, convertToStruct(v))
65 | }.toSeq
66 | StructType(fields)
67 |
68 | case elem =>
69 | elemType(elem)
70 |
71 | }
72 |
73 | /**
74 | * It looks for the most compatible type between two given DataTypes.
75 | * i.e.: {{{
76 | * val dataType1 = IntegerType
77 | * val dataType2 = DoubleType
78 | * assert(compatibleType(dataType1,dataType2)==DoubleType)
79 | * }}}
80 | * @param t1 First DataType to compare
81 | * @param t2 Second DataType to compare
82 | * @return Compatible type for both t1 and t2
83 | */
84 | private def compatibleType(t1: DataType, t2: DataType): DataType = {
85 | TypeCoercion.findTightestCommonTypeOfTwo(t1, t2) match {
86 | case Some(commonType) => commonType
87 |
88 | case None =>
89 | // t1 or t2 is a StructType, ArrayType, or an unexpected type.
90 | (t1, t2) match {
91 | case (other: DataType, NullType) => other
92 | case (NullType, other: DataType) => other
93 | case (StructType(fields1), StructType(fields2)) =>
94 | val newFields = (fields1 ++ fields2)
95 | .groupBy(field => field.name)
96 | .map { case (name, fieldTypes) =>
97 | val dataType = fieldTypes
98 | .map(field => field.dataType)
99 | .reduce(compatibleType)
100 | StructField(name, dataType, nullable = true)
101 |
102 | }
103 | StructType(newFields.toSeq.sortBy(_.name))
104 |
105 | case (ArrayType(elementType1, containsNull1), ArrayType(elementType2, containsNull2)) =>
106 | ArrayType(
107 | compatibleType(elementType1, elementType2),
108 | containsNull1 || containsNull2)
109 |
110 | case (_, _) => StringType
111 | }
112 | }
113 | }
114 |
115 | private def typeOfArray(l: Seq[Any]): ArrayType = {
116 | val containsNull = l.contains(null)
117 | val elements = l.flatMap(v => Option(v))
118 | if (elements.isEmpty) {
119 | // If this JSON array is empty, we use NullType as a placeholder.
120 | // If this array is not empty in other JSON objects, we can resolve
121 | // the type after we have passed through all JSON objects.
122 | ArrayType(NullType, containsNull)
123 | } else {
124 | val elementType = elements
125 | .map(convertToStruct)
126 | .reduce(compatibleType)
127 | ArrayType(elementType, containsNull)
128 | }
129 | }
130 |
131 | private def elemType: PartialFunction[Any, DataType] = {
132 | case obj: Boolean => BooleanType
133 | case obj: Array[Byte] => BinaryType
134 | case obj: String => StringType
135 | case obj: UTF8String => StringType
136 | case obj: Byte => ByteType
137 | case obj: Short => ShortType
138 | case obj: Int => IntegerType
139 | case obj: Long => LongType
140 | case obj: Float => FloatType
141 | case obj: Double => DoubleType
142 | case obj: java.sql.Date => DateType
143 | case obj: java.math.BigDecimal => DecimalType.SYSTEM_DEFAULT
144 | case obj: Decimal => DecimalType.SYSTEM_DEFAULT
145 | case obj: java.sql.Timestamp => TimestampType
146 | case null => NullType
147 | case date: java.util.Date => TimestampType
148 | case _ => StringType
149 | }
150 | }
151 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/schema/MongodbRowConverter.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.schema
17 |
18 | import com.mongodb.casbah.Imports._
19 | import com.stratio.datasource.schema.RowConverter
20 | import org.apache.spark.rdd.RDD
21 | import org.apache.spark.sql.Row
22 | import org.apache.spark.sql.catalyst.expressions.{GenericRowWithSchema, GenericRow}
23 | import org.apache.spark.sql.types._
24 |
25 | import scala.collection.immutable.ListMap
26 | import scala.collection.mutable.ArrayBuffer
27 |
28 | /**
29 | * MongodbRowConverter support RDD transformations
30 | * from DBObject to Row and vice versa
31 | */
32 | object MongodbRowConverter extends RowConverter[DBObject]
33 | with JsonSupport
34 | with Serializable {
35 |
36 | /**
37 | *
38 | * @param schema RDD native schema
39 | * @param rdd Current native RDD
40 | * @return A brand new RDD of Spark SQL Row type.
41 | */
42 | def asRow(schema: StructType, rdd: RDD[DBObject]): RDD[Row] = {
43 | rdd.map { record =>
44 | recordAsRow(dbObjectToMap(record), schema)
45 | }
46 | }
47 |
48 | /**
49 | *
50 | * @param schema native schema
51 | * @param array Current mongodb result
52 | * @return An array of Spark SQL Row type.
53 | */
54 | def asRow(schema: StructType, array: Array[DBObject]): Array[Row] = {
55 | array.map { record =>
56 | recordAsRow(dbObjectToMap(record), schema)
57 | }
58 | }
59 |
60 | /**
61 | * Given a schema, it converts a JSON object (as map) into a Row
62 | * @param json DBObject map
63 | * @param schema Schema
64 | * @return The converted row
65 | */
66 | def recordAsRow(
67 | json: Map[String, AnyRef],
68 | schema: StructType): Row = {
69 |
70 | val values: Seq[Any] = schema.fields.map {
71 | case StructField(name, et, _, mdata)
72 | if(mdata.contains("idx") && mdata.contains("colname")) =>
73 | val colName = mdata.getString("colname")
74 | val idx = mdata.getLong("idx").toInt
75 | json.get(colName).flatMap(v => Option(v)).map(toSQL(_, ArrayType(et, true))).collect {
76 | case elemsList: Seq[_] if((0 until elemsList.size) contains idx) => elemsList(idx)
77 | } orNull
78 | case StructField(name, dataType, _, _) =>
79 | json.get(name).flatMap(v => Option(v)).map(
80 | toSQL(_, dataType)).orNull
81 | }
82 | new GenericRowWithSchema(values.toArray, schema)
83 | }
84 |
85 | /**
86 | * Given a schema, it converts a Row into a DBObject
87 | * @param row Row to be converted
88 | * @param schema Schema
89 | * @return The converted DBObject
90 | */
91 | def rowAsDBObject(row: Row, schema: StructType): DBObject = {
92 | val attMap = ListMap(schema.fields.zipWithIndex.map {
93 | case (att, idx) => (att.name, toDBObject(row(idx),att.dataType))
94 | }:_*)
95 | attMap
96 | }
97 |
98 | /**
99 | * It converts some Row attribute value into
100 | * a DBObject field
101 | * @param value Row attribute
102 | * @param dataType Attribute type
103 | * @return The converted value into a DBObject field.
104 | */
105 | def toDBObject(value: Any, dataType: DataType): Any = {
106 | Option(value).map{v =>
107 | (dataType,v) match {
108 | case (ArrayType(elementType, _),array: Seq[Any@unchecked]) =>
109 | val list: List[Any] = array.map{
110 | case obj => toDBObject(obj,elementType)
111 | }.toList
112 | list
113 | case (struct: StructType,value: GenericRow) =>
114 | rowAsDBObject(value,struct)
115 | case _ => v
116 | }
117 | }.orNull
118 | }
119 |
120 | /**
121 | * It converts some DBObject attribute value into
122 | * a Row field
123 | * @param value DBObject attribute
124 | * @param dataType Attribute type
125 | * @return The converted value into a Row field.
126 | */
127 | def toSQL(value: Any, dataType: DataType): Any = {
128 | import scala.collection.JavaConversions._
129 | Option(value).map{value =>
130 | (value,dataType) match {
131 | case (dbList: BasicDBList,ArrayType(elementType, _)) =>
132 | dbList.map(toSQL(_, elementType))
133 | case (list: List[AnyRef @unchecked],ArrayType(elementType, _)) =>
134 | val dbList = new BasicDBList
135 | dbList.addAll(list)
136 | toSQL(dbList,dataType)
137 | case (_,struct: StructType) =>
138 | recordAsRow(dbObjectToMap(value.asInstanceOf[DBObject]), struct)
139 | case (_ , map: MapType) => dbObjectToMap(value.asInstanceOf[DBObject])
140 | .map(element => (toSQL(element._1, map.keyType), toSQL(element._2, map.valueType)))
141 | case _ =>
142 | //Assure value is mapped to schema constrained type.
143 | enforceCorrectType(value, dataType)
144 | }
145 | }.orNull
146 | }
147 |
148 | /**
149 | * It creates a map with dbObject attribute values.
150 | * @param dBObject Object to be splitted into attribute tuples.
151 | * @return A map with dbObject attributes.
152 | */
153 | def dbObjectToMap(dBObject: DBObject): Map[String, AnyRef] = {
154 | dBObject.seq.toMap
155 | }
156 |
157 | }
158 |
--------------------------------------------------------------------------------
/spark-mongodb/src/test/scala/com/stratio/datasource/mongodb/client/MongodbClientFactoryTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.client
17 |
18 | import com.mongodb.casbah.MongoClient
19 | import com.mongodb.{MongoCredential, ServerAddress}
20 | import com.stratio.datasource.MongodbTestConstants
21 | import com.stratio.datasource.mongodb.config.MongodbSSLOptions
22 | import org.junit.runner.RunWith
23 | import org.scalatest.junit.JUnitRunner
24 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FlatSpec, Matchers}
25 |
26 | @RunWith(classOf[JUnitRunner])
27 | class MongodbClientFactoryTest extends FlatSpec
28 | with Matchers
29 | with MongodbTestConstants
30 | with BeforeAndAfter
31 | with BeforeAndAfterAll {
32 |
33 | type Client = MongoClient
34 |
35 | val hostClient = MongodbClientFactory.getClient("127.0.0.1").clientConnection
36 |
37 | val hostPortCredentialsClient = MongodbClientFactory.getClient("127.0.0.1", 27017, "user", "database", "password").clientConnection
38 |
39 | val fullClient = MongodbClientFactory.getClient(
40 | List(new ServerAddress("127.0.0.1:27017")),
41 | List(MongoCredential.createCredential("user","database","password".toCharArray)),
42 | Some(MongodbSSLOptions(Some("/etc/ssl/mongodb.keystore"), Some("password"), "/etc/ssl/mongodb.keystore", Some("password"))),
43 | Map(
44 | "readPreference" -> "nearest",
45 | "connectTimeout"-> "50000",
46 | "socketTimeout"-> "50000",
47 | "maxWaitTime"-> "50000",
48 | "connectionsPerHost" -> "20",
49 | "threadsAllowedToBlockForConnectionMultiplier" -> "5"
50 | )
51 | ).clientConnection
52 |
53 | val gracefully = true
54 |
55 | val notGracefully = false
56 |
57 |
58 | behavior of "MongodbClientFactory"
59 |
60 | it should "Valid output type " + scalaBinaryVersion in {
61 |
62 | hostClient shouldBe a [Client]
63 | hostPortCredentialsClient shouldBe a [Client]
64 | fullClient shouldBe a [Client]
65 |
66 | MongodbClientFactory.closeAll(notGracefully)
67 | }
68 |
69 | it should "Valid clients size when getting the same client " in {
70 | val sameHostClient = MongodbClientFactory.getClient("127.0.0.1").clientConnection
71 |
72 | MongodbClientFactory.getClientPoolSize should be (1)
73 |
74 | val otherHostClient = MongodbClientFactory.getClient("127.0.0.1").clientConnection
75 |
76 | MongodbClientFactory.getClientPoolSize should be (2)
77 |
78 | MongodbClientFactory.closeAll(notGracefully)
79 | }
80 |
81 | it should "Valid clients size when getting the same client and set free " in {
82 | val sameHostClient = MongodbClientFactory.getClient("127.0.0.1").clientConnection
83 |
84 | MongodbClientFactory.getClientPoolSize should be (1)
85 |
86 | MongodbClientFactory.setFreeConnectionByClient(sameHostClient)
87 |
88 | val otherHostClient = MongodbClientFactory.getClient("127.0.0.1").clientConnection
89 |
90 | MongodbClientFactory.getClientPoolSize should be (1)
91 |
92 | MongodbClientFactory.closeAll(notGracefully)
93 | }
94 |
95 | it should "Valid clients size when closing one client gracefully " in {
96 | val sameHostClient = MongodbClientFactory.getClient("127.0.0.1").clientConnection
97 |
98 | MongodbClientFactory.getClientPoolSize should be (1)
99 |
100 | MongodbClientFactory.closeByClient(sameHostClient)
101 |
102 | MongodbClientFactory.getClientPoolSize should be (1)
103 |
104 | MongodbClientFactory.closeAll(notGracefully)
105 | }
106 |
107 | it should "Valid clients size when closing one client not gracefully " in {
108 | val sameHostClient = MongodbClientFactory.getClient("127.0.0.1").clientConnection
109 |
110 | MongodbClientFactory.getClientPoolSize should be (1)
111 |
112 | MongodbClientFactory.closeByClient(sameHostClient, notGracefully)
113 |
114 | MongodbClientFactory.getClientPoolSize should be (0)
115 |
116 | MongodbClientFactory.closeAll(notGracefully)
117 | }
118 |
119 | it should "Valid clients size when closing all clients gracefully " in {
120 | val sameHostClient = MongodbClientFactory.getClient("127.0.0.1").clientConnection
121 | val otherHostClient = MongodbClientFactory.getClient("127.0.0.1").clientConnection
122 |
123 | MongodbClientFactory.getClientPoolSize should be (2)
124 |
125 | MongodbClientFactory.closeAll(gracefully, 1)
126 |
127 | MongodbClientFactory.getClientPoolSize should be (2)
128 |
129 | MongodbClientFactory.setFreeConnectionByClient(sameHostClient)
130 |
131 | MongodbClientFactory.closeAll(gracefully, 1)
132 |
133 | MongodbClientFactory.getClientPoolSize should be (1)
134 |
135 | MongodbClientFactory.closeAll(notGracefully)
136 | }
137 |
138 | it should "Valid clients size when closing all clients not gracefully " in {
139 | val sameHostClient = MongodbClientFactory.getClient("127.0.0.1").clientConnection
140 | val otherHostClient = MongodbClientFactory.getClient("127.0.0.1").clientConnection
141 | val gracefully = false
142 |
143 | MongodbClientFactory.getClientPoolSize should be (2)
144 |
145 | MongodbClientFactory.closeAll(notGracefully)
146 |
147 | MongodbClientFactory.getClientPoolSize should be (0)
148 |
149 | MongodbClientFactory.closeAll(notGracefully)
150 | }
151 | }
152 |
--------------------------------------------------------------------------------
/spark-mongodb/src/test/scala/com/stratio/datasource/mongodb/TestBsonData.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb
17 |
18 | import com.mongodb.DBObject
19 | import com.mongodb.util.JSON
20 |
21 | trait TestBsonData {
22 |
23 | val primitiveFieldAndType =
24 | JSON.parse(
25 | """{"string":"this is a simple string.",
26 | "integer":10,
27 | "long":21474836470,
28 | "double":1.7976931348623157E308,
29 | "boolean":true,
30 | "null":null
31 | }""").asInstanceOf[DBObject] :: Nil
32 |
33 | val primitiveFieldAndType5rows =
34 | JSON.parse(
35 | """{"string":"this is a simple string.",
36 | "integer":10,
37 | "long":21474836470,
38 | "double":1.7976931348623157E308,
39 | "boolean":true,
40 | "null":null
41 | }""").asInstanceOf[DBObject] ::
42 | JSON.parse(
43 | """{"string":"this is another simple string.",
44 | "integer":11,
45 | "long":21474836471,
46 | "double":2.7976931348623157E308,
47 | "boolean":false,
48 | "null":null
49 | }""").asInstanceOf[DBObject] ::
50 | JSON.parse(
51 | """{"string":"this is the third simple string.",
52 | "integer":12,
53 | "long":21474836472,
54 | "double":3.7976931348623157E308,
55 | "boolean":true,
56 | "null":null
57 | }""").asInstanceOf[DBObject] ::
58 | JSON.parse(
59 | """{"string":"this is the forth simple string.",
60 | "integer":13,
61 | "long":21474836473,
62 | "double":4.7976931348623157E308,
63 | "boolean":true,
64 | "null":null
65 | }""").asInstanceOf[DBObject] ::
66 | JSON.parse(
67 | """{"string":"this is the fifth simple string.",
68 | "integer":14,
69 | "long":21474836474,
70 | "double":5.7976931348623157E308,
71 | "boolean":false,
72 | "null":null
73 | }""").asInstanceOf[DBObject] :: Nil
74 |
75 | val primitiveFieldValueTypeConflict =
76 | JSON.parse(
77 | """{"num_num_1":11, "num_num_2":null, "num_num_3": 1.1,
78 | "num_bool":true, "num_str":13.1, "str_bool":"str1"}""").asInstanceOf[DBObject] ::
79 | JSON.parse(
80 | """{"num_num_1":null, "num_num_2":21474836470.9, "num_num_3": null,
81 | "num_bool":12, "num_str":null, "str_bool":true}""" ).asInstanceOf[DBObject] ::
82 | JSON.parse(
83 | """{"num_num_1":21474836470, "num_num_2":9223372036854, "num_num_3": 100,
84 | "num_bool":false, "num_str":"str1", "str_bool":false}""" ).asInstanceOf[DBObject] ::
85 | JSON.parse(
86 | """{"num_num_1":21474836570, "num_num_2":1.1, "num_num_3": 21474836470,
87 | "num_bool":null, "num_str":9223372036854775807, "str_bool":null}""").asInstanceOf[DBObject] :: Nil
88 |
89 | val complexFieldAndType1 =
90 | JSON.parse(
91 | """{"struct":{"field1": true, "field2": 9223372036854775807},
92 | "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]},
93 | "arrayOfString":["str1", "str2"],
94 | "arrayOfInteger":[1, 2147483647, -2147483648],
95 | "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808],
96 | "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308],
97 | "arrayOfBoolean":[true, false, true],
98 | "arrayOfNull":[null, null, null, null],
99 | "arrayEmpty":[],
100 | "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}],
101 | "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]],
102 | "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]]
103 | }""").asInstanceOf[DBObject] :: Nil
104 |
105 | val complexFieldAndType2 =
106 | JSON.parse(
107 | """{"arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}],
108 | "complexArrayOfStruct": [
109 | {
110 | "field1": [
111 | {
112 | "inner1": "str1"
113 | },
114 | {
115 | "inner2": ["str2", "str22"]
116 | }],
117 | "field2": [[1, 2], [3, 4]]
118 | },
119 | {
120 | "field1": [
121 | {
122 | "inner2": ["str3", "str33"]
123 | },
124 | {
125 | "inner1": "str4"
126 | }],
127 | "field2": [[5, 6], [7, 8]]
128 | }],
129 | "arrayOfArray1": [
130 | [
131 | [5]
132 | ],
133 | [
134 | [6, 7],
135 | [8]
136 | ]],
137 | "arrayOfArray2": [
138 | [
139 | [
140 | {
141 | "inner1": "str1"
142 | }
143 | ]
144 | ],
145 | [
146 | [],
147 | [
148 | {"inner2": ["str3", "str33"]},
149 | {"inner2": ["str4"], "inner1": "str11"}
150 | ]
151 | ],
152 | [
153 | [
154 | {"inner3": [[{"inner4": 2}]]}
155 | ]
156 | ]]
157 | }""").asInstanceOf[DBObject] :: Nil
158 | }
159 |
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
19 |
20 |
21 | 4.0.0
22 | com.stratio.datasource
23 | spark-mongodb-parent
24 | 0.13.0-SNAPSHOT
25 | pom
26 | Stratio Spark Mongodb Datasource
27 | A Spark SQL library for MongoDB
28 | http://github.com/Stratio/spark-mongodb
29 | 2015
30 |
31 |
32 | com.stratio
33 | parent
34 | 0.8.2
35 |
36 |
37 |
38 | doc
39 | spark-mongodb_2.10
40 | spark-mongodb_2.11
41 |
42 |
43 |
44 | UTF-8
45 |
46 |
47 |
48 |
49 | rmorandeira
50 | Roberto Morandeira
51 | rmorandeira@stratio.com
52 |
53 | architect
54 | developer
55 | contributor
56 |
57 |
58 |
59 | jsantos
60 | Javier santos
61 | jsantos@stratio.com
62 |
63 | architect
64 | developer
65 | contributor
66 |
67 |
68 |
69 | lfernandez
70 | Loreto Fernandez
71 | lfernandez@stratio.com
72 |
73 | architect
74 | developer
75 | contributor
76 |
77 |
78 |
79 | pmadrigal
80 | Pedro Madrigal
81 | pmadrigal@stratio.com
82 |
83 | architect
84 | developer
85 | contributor
86 |
87 |
88 |
89 | ccaballero
90 | Cristian Caballero
91 | ccaballero@stratio.com
92 |
93 | architect
94 | developer
95 | contributor
96 |
97 |
98 |
99 |
100 |
101 |
102 | The Apache Software License, Version 2.0
103 | http://www.apache.org/licenses/LICENSE-2.0.txt
104 | repo
105 |
106 |
107 |
108 |
109 | scm:git:git@github.com:Stratio/spark-mongodb.git
110 | scm:git:git@github.com:Stratio/spark-mongodb.git
111 | https://github.com/Stratio/spark-mongodb
112 |
113 |
114 |
115 |
116 |
117 | org.apache.maven.plugins
118 | maven-surefire-plugin
119 |
120 |
121 | org.apache.maven.plugins
122 | maven-failsafe-plugin
123 |
124 |
125 |
126 | org.jacoco
127 | jacoco-maven-plugin
128 |
129 |
130 | org.apache.maven.plugins
131 | maven-source-plugin
132 | 2.2.1
133 |
134 |
135 | attach-sources
136 |
137 | jar
138 |
139 |
140 |
141 |
142 |
143 | org.apache.maven.plugins
144 | maven-jar-plugin
145 | 2.2
146 |
147 |
148 |
149 | test-jar
150 |
151 |
152 |
153 |
154 |
155 | com.mycila
156 | license-maven-plugin
157 |
158 |
159 |
160 |
161 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/schema/JsonSupport.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.schema
17 |
18 | import java.sql.{Date, Timestamp}
19 |
20 | import org.apache.spark.sql.types._
21 |
22 | /**
23 | * Json - Scala object transformation support.
24 | * Used to convert from DBObjects to Spark SQL Row field types.
25 | * Disclaimer: As explained in NOTICE.md, some of this product includes
26 | * software developed by The Apache Software Foundation (http://www.apache.org/).
27 | */
28 | trait JsonSupport {
29 |
30 | /**
31 | * Tries to convert some scala value to another compatible given type
32 | * @param value Value to be converted
33 | * @param desiredType Destiny type
34 | * @return Converted value
35 | */
36 |
37 |
38 | protected def enforceCorrectType(value: Any, desiredType: DataType): Any =
39 | Option(value).map{ _ => desiredType match {
40 | case StringType => toString(value)
41 | case _ if value == "" => null // guard the non string type
42 | case ByteType => toByte(value)
43 | case BinaryType => toBinary(value)
44 | case ShortType => toShort(value)
45 | case IntegerType => toInt(value)
46 | case LongType => toLong(value)
47 | case DoubleType => toDouble(value)
48 | case DecimalType() => toDecimal(value)
49 | case FloatType => toFloat(value)
50 | case BooleanType => value.asInstanceOf[Boolean]
51 | case DateType => toDate(value)
52 | case TimestampType => toTimestamp(value)
53 | case NullType => null
54 | case _ =>
55 | sys.error(s"Unsupported datatype conversion [Value: ${value}] of ${value.getClass}] to ${desiredType}]")
56 | value
57 | }
58 | }.getOrElse(null)
59 |
60 | private def toBinary(value: Any): Array[Byte] = {
61 | value match {
62 | case value: org.bson.types.Binary => value.getData
63 | case value: Array[Byte] => value
64 | }
65 | }
66 |
67 | private def toByte(value: Any): Byte = {
68 | value match {
69 | case value: java.lang.Integer => value.byteValue()
70 | case value: java.lang.Long => value.byteValue()
71 | }
72 | }
73 |
74 | private def toShort(value: Any): Short = {
75 | value match {
76 | case value: java.lang.Integer => value.toShort
77 | case value: java.lang.Long => value.toShort
78 | }
79 | }
80 |
81 | private def toInt(value: Any): Int = {
82 | import scala.language.reflectiveCalls
83 | value match {
84 | case value: String => value.toInt
85 | case _ => value.asInstanceOf[ {def toInt: Int}].toInt
86 | }
87 | }
88 |
89 | private def toLong(value: Any): Long = {
90 | value match {
91 | case value: java.lang.Integer => value.asInstanceOf[Int].toLong
92 | case value: java.lang.Long => value.asInstanceOf[Long]
93 | case value: java.lang.Double => value.asInstanceOf[Double].toLong
94 | }
95 | }
96 |
97 | private def toDouble(value: Any): Double = {
98 | value match {
99 | case value: java.lang.Integer => value.asInstanceOf[Int].toDouble
100 | case value: java.lang.Long => value.asInstanceOf[Long].toDouble
101 | case value: java.lang.Double => value.asInstanceOf[Double]
102 | }
103 | }
104 |
105 | private def toDecimal(value: Any): java.math.BigDecimal = {
106 | value match {
107 | case value: java.lang.Integer => new java.math.BigDecimal(value)
108 | case value: java.lang.Long => new java.math.BigDecimal(value)
109 | case value: java.lang.Double => new java.math.BigDecimal(value)
110 | case value: java.math.BigInteger => new java.math.BigDecimal(value)
111 | case value: java.math.BigDecimal => value
112 | }
113 | }
114 |
115 | private def toFloat(value: Any): Float = {
116 | value match {
117 | case value: java.lang.Integer => value.toFloat
118 | case value: java.lang.Long => value.toFloat
119 | case value: java.lang.Double => value.toFloat
120 | }
121 | }
122 |
123 | private def toTimestamp(value: Any): Timestamp = {
124 | value match {
125 | case value: java.util.Date => new Timestamp(value.getTime)
126 | }
127 | }
128 |
129 | private def toDate(value: Any): Date = {
130 | value match {
131 | case value: java.util.Date => new Date(value.getTime)
132 | // TODO Parse string to date when a String type arrives
133 | // case value: java.lang.String => ???
134 | }
135 | }
136 |
137 | private def toJsonArrayString(seq: Seq[Any]): String = {
138 | val builder = new StringBuilder
139 | builder.append("[")
140 | var count = 0
141 | seq.foreach {
142 | element =>
143 | if (count > 0) builder.append(",")
144 | count += 1
145 | builder.append(toString(element))
146 | }
147 | builder.append("]")
148 |
149 | builder.toString()
150 | }
151 |
152 | private def toJsonObjectString(map: Map[String, Any]): String = {
153 | val builder = new StringBuilder
154 | builder.append("{")
155 | var count = 0
156 | map.foreach {
157 | case (key, value) =>
158 | if (count > 0) builder.append(",")
159 | count += 1
160 | val stringValue = if (value.isInstanceOf[String]) s"""\"$value\"""" else toString(value)
161 | builder.append(s"""\"$key\":$stringValue""")
162 | }
163 | builder.append("}")
164 |
165 | builder.toString()
166 | }
167 |
168 | private def toString(value: Any): String = {
169 | value match {
170 | case value: Map[_, _] => toJsonObjectString(value.asInstanceOf[Map[String, Any]])
171 | case value: Seq[_] => toJsonArrayString(value)
172 | case v => Option(v).map(_.toString).orNull
173 | }
174 | }
175 |
176 | }
--------------------------------------------------------------------------------
/spark-mongodb/src/test/scala/com/stratio/datasource/mongodb/schema/MongodbRowConverterIT.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.schema
17 |
18 | import com.mongodb.DBObject
19 | import com.mongodb.util.JSON
20 | import com.stratio.datasource.MongodbTestConstants
21 | import com.stratio.datasource.mongodb.client.MongodbClientFactory
22 | import com.stratio.datasource.mongodb.config.{MongodbConfig, MongodbConfigBuilder}
23 | import com.stratio.datasource.mongodb.partitioner.MongodbPartitioner
24 | import com.stratio.datasource.mongodb.rdd.MongodbRDD
25 | import com.stratio.datasource.mongodb.schema.MongodbRowConverter._
26 | import com.stratio.datasource.mongodb._
27 |
28 | import org.apache.spark.sql.catalyst.expressions.GenericRow
29 | import org.apache.spark.sql.mongodb.{TemporaryTestSQLContext, TestSQLContext}
30 | import org.apache.spark.sql.types._
31 | import org.junit.runner.RunWith
32 | import org.scalatest.junit.JUnitRunner
33 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FlatSpec, Matchers}
34 |
35 | import scala.collection.mutable
36 | import scala.collection.mutable.ArrayBuffer
37 |
38 | @RunWith(classOf[JUnitRunner])
39 | class MongodbRowConverterIT extends FlatSpec
40 | with Matchers
41 | with MongoEmbedDatabase
42 | with TestBsonData
43 | with MongodbTestConstants
44 | with BeforeAndAfterAll {
45 |
46 | private val host: String = "localhost"
47 | private val collection: String = "testCol"
48 |
49 | val testConfig = MongodbConfigBuilder()
50 | .set(MongodbConfig.Host,List(host + ":" + mongoPort))
51 | .set(MongodbConfig.Database,db)
52 | .set(MongodbConfig.Collection,collection)
53 | .set(MongodbConfig.SamplingRatio,1.0)
54 | .build()
55 |
56 | // Sample values
57 | val valueWithType: List[(Any, StructField)] = List(
58 | 1 -> new StructField(
59 | "att1",IntegerType,false),
60 | 2.0 -> new StructField(
61 | "att2",DoubleType,false),
62 | "hi" -> new StructField(
63 | "att3",StringType,false),
64 | null.asInstanceOf[Any] -> new StructField(
65 | "att4",StringType,true),
66 | new ArrayBuffer[Int]().+=(1).+=(2).+=(3) -> new StructField(
67 | "att5",new ArrayType(IntegerType,false),false),
68 | new GenericRow(List(1,null).toArray) -> new StructField(
69 | "att6",new StructType(Array(
70 | new StructField("att61",IntegerType ,false),
71 | new StructField("att62",IntegerType,true)
72 | )),false),
73 | // Subdocument
74 | new GenericRow(List(1, new GenericRow(List(1, "b").toArray)).toArray) ->
75 | new StructField(
76 | "att7", new StructType(Array(
77 | new StructField("att71",IntegerType,false),
78 | new StructField("att72",new StructType(Array(
79 | new StructField("att721", IntegerType, false),
80 | new StructField("att722", StringType, false)
81 | )),false))), false),
82 | // Subdocument with List of a document
83 | new GenericRow(List(1, new ArrayBuffer[Any]().+=(new GenericRow(List(2,"b").toArray))).toArray) ->
84 | new StructField("att8", new StructType(
85 | Array(StructField("att81", IntegerType, false), StructField("att82",
86 | new ArrayType(StructType(Array(StructField("att821", IntegerType, false),StructField("att822", StringType, false))), false)
87 | ,false))), false),
88 | // Subdocument with List of a document with wrapped array
89 | new GenericRow(List(1, new mutable.WrappedArray.ofRef[AnyRef](Array(
90 | new GenericRow(List(2,"b").toArray)
91 | ))).toArray) ->
92 | new StructField("att9", new StructType(
93 | Array(StructField("att91", IntegerType, false), StructField("att92",
94 | new ArrayType(StructType(Array(StructField("att921", IntegerType, false),StructField("att922", StringType, false))), false)
95 | ,false))), false)
96 | )
97 |
98 | val rowSchema = new StructType(valueWithType.map(_._2).toArray)
99 |
100 | val row = new GenericRow(valueWithType.map(_._1).toArray)
101 |
102 | val dbObject = JSON.parse(
103 | """{ "att5" : [ 1 , 2 , 3] ,
104 | "att4" : null ,
105 | "att3" : "hi" ,
106 | "att6" : { "att61" : 1 , "att62" : null } ,
107 | "att2" : 2.0 ,
108 | "att1" : 1,
109 | "att7" : {"att71": 1, "att72":{"att721":1, "att722":"b"}},
110 | "att8" : {"att81": 1, "att82":[{"att821":2, "att822":"b"}]},
111 | "att9" : {"att91": 1, "att92":[{"att921":2, "att922":"b"}]}
112 | }
113 | """).asInstanceOf[DBObject]
114 |
115 | behavior of "The MongodbRowConverter"
116 |
117 | it should "be able to convert any value from a row into a dbobject field" + scalaBinaryVersion in{
118 | toDBObject(row, rowSchema) should equal(dbObject)
119 | }
120 |
121 | it should "be able to convert any value from a dbobject field into a row field" + scalaBinaryVersion in{
122 | toSQL(dbObject,rowSchema) should equal(row)
123 | }
124 |
125 | it should "apply dbobject to row mapping in a RDD context" + scalaBinaryVersion in {
126 | withEmbedMongoFixture(complexFieldAndType2) { mongodProc =>
127 | val mongodbPartitioner = new MongodbPartitioner(testConfig)
128 | val mongodbRDD = new MongodbRDD(TemporaryTestSQLContext, testConfig, mongodbPartitioner)
129 | val schema = MongodbSchema(mongodbRDD, 1.0).schema()
130 | println("\n\nschema")
131 | schema.fieldNames.foreach(println)
132 | val collected = toSQL(complexFieldAndType2.head,schema)
133 | MongodbRowConverter
134 | .asRow(schema,mongodbRDD)
135 | .collect().toList should equal(List(collected))
136 | }
137 | }
138 |
139 | override def afterAll {
140 | MongodbClientFactory.closeAll(false)
141 | }
142 |
143 | }
144 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/query/FilterSection.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.query
17 |
18 | import java.util.regex.Pattern
19 |
20 | import com.mongodb.QueryBuilder
21 | import com.mongodb.casbah.Imports
22 | import com.mongodb.casbah.Imports._
23 | import com.stratio.datasource.mongodb.sources.{NearSphere, Near}
24 | import org.apache.spark.sql.sources._
25 | import com.stratio.datasource.util.Config
26 | import com.stratio.datasource.mongodb.config.MongodbConfig
27 |
28 | object FilterSection {
29 |
30 | /**
31 | * Implicit conversion to pass from an array of [[Filter]] to [[FilterSection]] filter description object.
32 | *
33 | * @param sFilters
34 | * @param config
35 | * @return [[FilterSection]] built from `sFilters`
36 | */
37 | implicit def srcFilArr2filSel(sFilters: Array[Filter])(implicit config: Config): FilterSection =
38 | new SourceFilters(sFilters)
39 |
40 | //Factory methods
41 |
42 | def apply(sFilters: Array[Filter])(implicit config: Config): FilterSection =
43 | srcFilArr2filSel(sFilters)
44 |
45 | def apply(): FilterSection = NoFilters
46 | }
47 |
48 | /**
49 | * Trait to be implemented to those classes describing the Filter section of a MongoDB query.
50 | */
51 | trait FilterSection {
52 | /**
53 | * @return a [[DBObject]] describing the filters to apply to a partition.
54 | */
55 | def filtersToDBObject(): DBObject
56 | }
57 |
58 | /**
59 | * Filter described by a [[DBObject]] as it is used by Casbah (https://mongodb.github.io/casbah/)
60 | * @param filterDesc
61 | */
62 | case class RawFilter(filterDesc: DBObject) extends FilterSection {
63 | override def filtersToDBObject(): Imports.DBObject = filterDesc
64 | }
65 |
66 | /**
67 | * No filter to be applied
68 | */
69 | case object NoFilters extends FilterSection {
70 | override def filtersToDBObject(): Imports.DBObject = QueryBuilder.start.get()
71 | }
72 |
73 | /**
74 | * This [[FilterSection]] is described by an array of [[org.apache.spark.sql.sources.Filter]] where each
75 | * element is a restriction.
76 | *
77 | * @param sFilters All filters to be applied (AND)
78 | * @param parentFilterIsNot `true` iff the filter is negated: NOT (sFilters[0] AND ... AND sFilters[n-1])
79 | * @param config Access configuration
80 | */
81 | case class SourceFilters(
82 | sFilters: Array[Filter],
83 | parentFilterIsNot: Boolean = false
84 | )(implicit config: Config) extends FilterSection {
85 |
86 | override def filtersToDBObject: DBObject = {
87 | val queryBuilder: QueryBuilder = QueryBuilder.start
88 |
89 | if (parentFilterIsNot) queryBuilder.not()
90 |
91 | sFilters.foreach {
92 | case EqualTo(attribute, value) =>
93 | queryBuilder.put(attribute).is(checkObjectID(attribute, value))
94 | case GreaterThan(attribute, value) =>
95 | queryBuilder.put(attribute).greaterThan(checkObjectID(attribute, value))
96 | case GreaterThanOrEqual(attribute, value) =>
97 | queryBuilder.put(attribute).greaterThanEquals(checkObjectID(attribute, value))
98 | case In(attribute, values) =>
99 | queryBuilder.put(attribute).in(values.map(value => checkObjectID(attribute, value)))
100 | case LessThan(attribute, value) =>
101 | queryBuilder.put(attribute).lessThan(checkObjectID(attribute, value))
102 | case LessThanOrEqual(attribute, value) =>
103 | queryBuilder.put(attribute).lessThanEquals(checkObjectID(attribute, value))
104 | case IsNull(attribute) =>
105 | queryBuilder.put(attribute).is(null)
106 | case IsNotNull(attribute) =>
107 | queryBuilder.put(attribute).notEquals(null)
108 | case And(leftFilter, rightFilter) if !parentFilterIsNot =>
109 | queryBuilder.and(
110 | SourceFilters(Array(leftFilter)).filtersToDBObject(),
111 | SourceFilters(Array(rightFilter)).filtersToDBObject()
112 | )
113 | case Or(leftFilter, rightFilter) if !parentFilterIsNot =>
114 | queryBuilder.or(
115 | SourceFilters(Array(leftFilter)).filtersToDBObject(),
116 | SourceFilters(Array(rightFilter)).filtersToDBObject()
117 | )
118 | case StringStartsWith(attribute, value) if !parentFilterIsNot =>
119 | queryBuilder.put(attribute).regex(Pattern.compile("^" + value + ".*$"))
120 | case StringEndsWith(attribute, value) if !parentFilterIsNot =>
121 | queryBuilder.put(attribute).regex(Pattern.compile("^.*" + value + "$"))
122 | case StringContains(attribute, value) if !parentFilterIsNot =>
123 | queryBuilder.put(attribute).regex(Pattern.compile(".*" + value + ".*"))
124 | case Near(attribute, x, y, None) =>
125 | queryBuilder.put(attribute).near(x, y)
126 | case Near(attribute, x, y, Some(max)) =>
127 | queryBuilder.put(attribute).near(x, y, max)
128 | case NearSphere(attribute, longitude, latitude, None) =>
129 | queryBuilder.put(attribute).nearSphere(longitude, latitude)
130 | case NearSphere(attribute, longitude, latitude, Some(maxDistance)) =>
131 | queryBuilder.put(attribute).nearSphere(longitude, latitude, maxDistance)
132 | case Not(filter) =>
133 | SourceFilters(Array(filter), true).filtersToDBObject()
134 | }
135 |
136 | queryBuilder.get
137 | }
138 |
139 | /**
140 | * Check if the field is "_id" and if the user wants to filter by this field as an ObjectId
141 | *
142 | * @param attribute Name of the file
143 | * @param value Value for the attribute
144 | * @return The value in the correct data type
145 | */
146 | private def checkObjectID(attribute: String, value: Any)(implicit config: Config) : Any = attribute match {
147 | case "_id" if idAsObjectId => new ObjectId(value.toString)
148 | case _ => value
149 | }
150 |
151 | lazy val idAsObjectId: Boolean =
152 | config.getOrElse[String](MongodbConfig.IdAsObjectId, MongodbConfig.DefaultIdAsObjectId).equalsIgnoreCase("true")
153 |
154 | }
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/MongodbRelation.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb
17 |
18 | import com.mongodb.casbah.Imports._
19 | import com.stratio.datasource.mongodb.client.MongodbClientFactory
20 | import com.stratio.datasource.mongodb.config.{MongodbConfig, MongodbConfigReader}
21 | import com.stratio.datasource.mongodb.partitioner.MongodbPartitioner
22 | import com.stratio.datasource.mongodb.rdd.MongodbRDD
23 | import com.stratio.datasource.mongodb.schema.{MongodbRowConverter, MongodbSchema}
24 | import com.stratio.datasource.mongodb.util.usingMongoClient
25 | import com.stratio.datasource.util.Config
26 | import org.apache.spark.rdd.RDD
27 | import org.apache.spark.sql.sources.{BaseRelation, Filter, InsertableRelation, PrunedFilteredScan}
28 | import org.apache.spark.sql.types._
29 | import org.apache.spark.sql.{DataFrame, Row, SQLContext}
30 |
31 | /**
32 | * A MongoDB baseRelation that can eliminate unneeded columns
33 | * and filter using selected predicates before producing
34 | * an RDD containing all matching tuples as Row objects.
35 | * @param config A Mongo configuration with needed properties for MongoDB
36 | * @param schemaProvided The optionally provided schema. If not provided,
37 | * it will be inferred from the whole field projection
38 | * of the specified table in Spark SQL statement using
39 | * a sample ratio (as JSON Data Source does).
40 | * @param sqlContext An existing Spark SQL context.
41 | */
42 | class MongodbRelation(private val config: Config,
43 | schemaProvided: Option[StructType] = None)(
44 | @transient val sqlContext: SQLContext) extends BaseRelation
45 | with PrunedFilteredScan with InsertableRelation {
46 |
47 | implicit val _: Config = config
48 |
49 | import MongodbConfigReader._
50 | import MongodbRelation._
51 |
52 | private val rddPartitioner: MongodbPartitioner =
53 | new MongodbPartitioner(config)
54 |
55 | /**
56 | * Default schema to be used in case no schema was provided before.
57 | * It scans the RDD generated by Spark SQL statement,
58 | * using specified sample ratio.
59 | */
60 | @transient private lazy val lazySchema =
61 | MongodbSchema(
62 | new MongodbRDD(sqlContext, config, rddPartitioner),
63 | config.get[Any](MongodbConfig.SamplingRatio).fold(MongodbConfig.DefaultSamplingRatio)(_.toString.toDouble)).schema()
64 |
65 | override val schema: StructType = schemaProvided.getOrElse(lazySchema)
66 |
67 | override def buildScan(
68 | requiredColumns: Array[String],
69 | filters: Array[Filter]): RDD[Row] = {
70 |
71 | val rdd = new MongodbRDD(
72 | sqlContext,
73 | config,
74 | rddPartitioner,
75 | requiredColumns,
76 | filters)
77 |
78 | MongodbRowConverter.asRow(pruneSchema(schema, requiredColumns), rdd)
79 | }
80 |
81 | /**
82 | * Indicates if a collection is empty.
83 | * @return Boolean
84 | */
85 | def isEmptyCollection: Boolean =
86 | usingMongoClient(MongodbClientFactory.getClient(config.hosts, config.credentials, config.sslOptions, config.clientOptions).clientConnection) { mongoClient =>
87 | dbCollection(mongoClient).isEmpty
88 | }
89 |
90 |
91 |
92 |
93 |
94 | /**
95 | * Insert data into the specified DataSource.
96 | * @param data Data to insert.
97 | * @param overwrite Boolean indicating whether to overwrite the data.
98 | */
99 | def insert(data: DataFrame, overwrite: Boolean): Unit = {
100 | if (overwrite) {
101 | usingMongoClient(MongodbClientFactory.getClient(config.hosts, config.credentials, config.sslOptions, config.clientOptions).clientConnection) { mongoClient =>
102 | dbCollection(mongoClient).dropCollection()
103 | }
104 | }
105 |
106 | data.saveToMongodb(config)
107 | }
108 |
109 | /**
110 | * Compare if two MongodbRelation are the same.
111 | * @param other Object to compare
112 | * @return Boolean
113 | */
114 |
115 | override def equals(other: Any): Boolean = other match {
116 | case that: MongodbRelation =>
117 | schema == that.schema && config == that.config
118 | case _ => false
119 | }
120 |
121 | override def hashCode(): Int = {
122 | val state = Seq(schema, config)
123 | state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b)
124 | }
125 |
126 | /**
127 | * A MongoDB collection created from the specified database and collection.
128 | */
129 | private def dbCollection(mongoClient: MongoClient): MongoCollection =
130 | mongoClient(config(MongodbConfig.Database))(config(MongodbConfig.Collection))
131 | }
132 |
133 | object MongodbRelation {
134 |
135 | /**
136 | * Prune whole schema in order to fit with
137 | * required columns in Spark SQL statement.
138 | * @param schema Whole field projection schema.
139 | * @param requiredColumns Required fields in statement
140 | * @return A new pruned schema
141 | */
142 | def pruneSchema(
143 | schema: StructType,
144 | requiredColumns: Array[String]): StructType =
145 | pruneSchema(schema, requiredColumns.map(_ -> None): Array[(String, Option[Int])])
146 |
147 |
148 | /**
149 | * Prune whole schema in order to fit with
150 | * required columns taking in consideration nested columns (array elements) in Spark SQL statement.
151 | * @param schema Whole field projection schema.
152 | * @param requiredColumnsWithIndex Required fields in statement including index within field for random accesses.
153 | * @return A new pruned schema
154 | */
155 | def pruneSchema(
156 | schema: StructType,
157 | requiredColumnsWithIndex: Array[(String, Option[Int])]): StructType = {
158 |
159 | val name2sfield: Map[String, StructField] = schema.fields.map(f => f.name -> f).toMap
160 | StructType(
161 | requiredColumnsWithIndex.flatMap {
162 | case (colname, None) => name2sfield.get(colname)
163 | case (colname, Some(idx)) => name2sfield.get(colname) collect {
164 | case field @ StructField(name, ArrayType(et,_), nullable, _) =>
165 | val mdataBuilder = new MetadataBuilder
166 | //Non-functional area
167 | mdataBuilder.putLong("idx", idx.toLong)
168 | mdataBuilder.putString("colname", name)
169 | //End of non-functional area
170 | StructField(s"$name[$idx]", et, true, mdataBuilder.build())
171 | }
172 | }
173 | )
174 | }
175 |
176 | }
177 |
--------------------------------------------------------------------------------
/spark-mongodb_2.11/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
19 |
22 | 4.0.0
23 | spark-mongodb_2.11
24 | jar
25 |
26 | spark-mongodb-parent
27 | com.stratio.datasource
28 | 0.13.0-SNAPSHOT
29 |
30 |
31 | 2.11.8
32 | 2.11
33 | spark-mongodb
34 | 2.2.5
35 | 3.2.1
36 | 2.0.0
37 | 2.8.0
38 | 2.3.11
39 |
40 |
41 |
42 | org.scala-lang
43 | scala-library
44 | ${scala.version}
45 |
46 |
47 | org.apache.spark
48 | spark-core_${scala.binary.version}
49 | ${spark.version}
50 | provided
51 |
52 |
53 | org.apache.spark
54 | spark-sql_${scala.binary.version}
55 | ${spark.version}
56 | provided
57 |
58 |
59 | com.typesafe.akka
60 | akka-actor_${scala.binary.version}
61 | ${akka.version}
62 |
63 |
64 | org.mongodb
65 | casbah-commons_${scala.binary.version}
66 | ${casbah.version}
67 |
68 |
69 | org.mongodb
70 | casbah-query_${scala.binary.version}
71 | ${casbah.version}
72 |
73 |
74 | org.mongodb
75 | casbah-core_${scala.binary.version}
76 | ${casbah.version}
77 |
78 |
79 | de.flapdoodle.embed
80 | de.flapdoodle.embed.mongo
81 | 1.46.4
82 | test
83 |
84 |
85 | junit
86 | junit
87 | 4.11
88 | test
89 |
90 |
91 | org.scalatest
92 | scalatest_${scala.binary.version}
93 | ${scala.test.version}
94 | test
95 |
96 |
97 |
98 |
99 |
100 | ../spark-mongodb/src/test/resources
101 | true
102 |
103 |
104 | ../spark-mongodb/src/main/scala
105 | ../spark-mongodb/src/test/scala
106 |
107 |
108 | org.apache.maven.plugins
109 | maven-dependency-plugin
110 | 2.8
111 |
112 |
113 | copy-dependencies
114 | package
115 |
116 | copy-dependencies
117 |
118 |
119 | ${project.build.directory}/alternateLocation
120 | false
121 | false
122 | true
123 | ::*
124 |
125 |
126 |
127 |
128 |
129 | net.alchim31.maven
130 | scala-maven-plugin
131 | 3.2.1
132 |
133 | false
134 | ${scala.version}
135 | incremental
136 |
137 |
138 |
139 | scala-compile-first
140 | process-resources
141 |
142 | add-source
143 |
144 |
145 |
146 | scala-compile
147 | compile
148 |
149 | compile
150 |
151 |
152 |
153 | scala-testCompile
154 | test-compile
155 |
156 | testCompile
157 |
158 |
159 |
160 | scala-doc
161 | prepare-package
162 |
163 | doc
164 | doc-jar
165 |
166 |
167 |
168 |
169 |
170 |
171 |
172 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/config/MongodbConfig.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.config
17 |
18 | import com.mongodb.casbah.Imports._
19 | import com.mongodb.{MongoClientOptions => JavaMongoClientOptions}
20 | import com.stratio.datasource.util.Config._
21 |
22 | /**
23 | * Values and Functions for access and parse the configuration parameters
24 | */
25 | // TODO Review when refactoring config
26 | object MongodbConfig {
27 |
28 | // Parameter names
29 | val Host = "host"
30 | val Database = "database"
31 | val Collection = "collection"
32 | val SSLOptions = "sslOptions"
33 | val ReadPreference = "readPreference"
34 | val ConnectTimeout = "connectTimeout"
35 | val ConnectionsPerHost = "connectionsPerHost"
36 | val MaxWaitTime = "maxWaitTime"
37 | val SocketTimeout = "socketTimeout"
38 | val ThreadsAllowedToBlockForConnectionMultiplier = "threadsAllowedToBlockForConnectionMultiplier"
39 | val WriteConcern = "writeConcern"
40 | val Credentials = "credentials"
41 | val SamplingRatio = "schema_samplingRatio"
42 | val SplitSize = "splitSize"
43 | val SplitKey = "splitKey"
44 | val SplitKeyType = "splitKeyType"
45 | val SplitKeyMin = "splitKeyMin"
46 | val SplitKeyMax = "splitKeyMax"
47 | val UpdateFields = "updateFields"
48 | val Language = "language"
49 | val ConnectionsTime = "connectionsTime"
50 | val CursorBatchSize = "cursorBatchSize"
51 | val BulkBatchSize = "bulkBatchSize"
52 | val IdAsObjectId = "idAsObjectId"
53 |
54 | // List of parameters for mongoClientOptions
55 | val ListMongoClientOptions = List(
56 | ReadPreference,
57 | ConnectionsPerHost,
58 | ConnectTimeout,
59 | MaxWaitTime,
60 | ThreadsAllowedToBlockForConnectionMultiplier,
61 | ConnectionsTime
62 | )
63 |
64 | // Mandatory
65 | val required = List(
66 | Host,
67 | Database,
68 | Collection
69 | )
70 |
71 | // Default MongoDB values
72 | val DefaultMongoClientOptions = new JavaMongoClientOptions.Builder().build()
73 | val DefaultReadPreference = com.mongodb.casbah.ReadPreference.Nearest
74 | val DefaultConnectTimeout = DefaultMongoClientOptions.getConnectTimeout
75 | val DefaultConnectionsPerHost = DefaultMongoClientOptions.getConnectionsPerHost
76 | val DefaultMaxWaitTime = DefaultMongoClientOptions.getMaxWaitTime
77 | val DefaultSocketTimeout = DefaultMongoClientOptions.getSocketTimeout
78 | val DefaultThreadsAllowedToBlockForConnectionMultiplier= DefaultMongoClientOptions.getThreadsAllowedToBlockForConnectionMultiplier
79 | val DefaultCredentials = List[MongodbCredentials]()
80 | val DefaultWriteConcern = com.mongodb.WriteConcern.ACKNOWLEDGED
81 |
82 | // Default datasource specific values
83 | val DefaultSamplingRatio = 1.0
84 | val DefaultSplitSize = 10
85 | val DefaultSplitKey = "_id"
86 | val DefaultConnectionsTime = 10000L
87 | val DefaultCursorBatchSize = 101
88 | val DefaultBulkBatchSize = 1000
89 | val DefaultIdAsObjectId = "true"
90 |
91 | /**
92 | * Parse Map of string parameters to Map with the correct objects used in MongoDb Datasource functions
93 | * @param parameters List of parameters
94 | * @return List of parameters parsed to correct mongoDb configurations
95 | */
96 | // TODO Review when refactoring config
97 | def parseParameters(parameters : Map[String,String]): Map[String, Any] = {
98 |
99 | // required properties
100 | /** We will assume hosts are provided like 'host:port,host2:port2,...' */
101 | val properties: Map[String, Any] = parameters.updated(Host, parameters.getOrElse(Host, notFound[String](Host)).split(",").toList)
102 | if (!parameters.contains(Database)) notFound(Database)
103 | if (!parameters.contains(Collection)) notFound(Collection)
104 |
105 | //optional parseable properties
106 | val optionalProperties: List[String] = List(Credentials,SSLOptions, UpdateFields)
107 |
108 | (properties /: optionalProperties){
109 | /** We will assume credentials are provided like 'user,database,password;user,database,password;...' */
110 | case (properties,Credentials) =>
111 | parameters.get(Credentials).map{ credentialInput =>
112 | val credentials = credentialInput.split(";").map(_.split(",")).toList
113 | .map(credentials => MongodbCredentials(credentials(0), credentials(1), credentials(2).toCharArray))
114 | properties + (Credentials -> credentials)
115 | } getOrElse properties
116 |
117 | /** We will assume ssloptions are provided like '/path/keystorefile,keystorepassword,/path/truststorefile,truststorepassword' */
118 | case (properties,SSLOptions) =>
119 | parameters.get(SSLOptions).map{ ssloptionsInput =>
120 |
121 | val ssloption = ssloptionsInput.split(",")
122 | val ssloptions = MongodbSSLOptions(Some(ssloption(0)), Some(ssloption(1)), ssloption(2), Some(ssloption(3)))
123 | properties + (SSLOptions -> ssloptions)
124 | } getOrElse properties
125 |
126 | /** We will assume fields are provided like 'user,database,password...' */
127 | case (properties, UpdateFields) => {
128 | parameters.get(UpdateFields).map{ updateInputs =>
129 | val updateFields = updateInputs.split(",")
130 | properties + (UpdateFields -> updateFields)
131 | } getOrElse properties
132 | }
133 | }
134 | }
135 |
136 | /**
137 | * Parse one key to the associated readPreference
138 | * @param readPreference string key for identify the correct object
139 | * @return readPreference object
140 | */
141 | // TODO Review when refactoring config
142 | def parseReadPreference(readPreference: String): ReadPreference = {
143 | readPreference.toUpperCase match {
144 | case "PRIMARY" => com.mongodb.casbah.ReadPreference.Primary
145 | case "SECONDARY" => com.mongodb.casbah.ReadPreference.Secondary
146 | case "NEAREST" => com.mongodb.casbah.ReadPreference.Nearest
147 | case "PRIMARYPREFERRED" => com.mongodb.casbah.ReadPreference.primaryPreferred
148 | case "SECONDARYPREFERRED" => com.mongodb.casbah.ReadPreference.SecondaryPreferred
149 | case _ => com.mongodb.casbah.ReadPreference.Nearest
150 | }
151 | }
152 |
153 | /**
154 | * Parse one key to the associated writeConcern
155 | * @param writeConcern string key for identify the correct object
156 | * @return writeConcern object
157 | */
158 | // TODO Review when refactoring config
159 | def parseWriteConcern(writeConcern: String): WriteConcern = {
160 | writeConcern.toUpperCase match {
161 | case "SAFE" | "ACKNOWLEDGED" => com.mongodb.WriteConcern.SAFE
162 | case "NORMAL" | "UNACKNOWLEDGED" => com.mongodb.WriteConcern.NORMAL
163 | case "REPLICAS_SAFE" | "REPLICA_ACKNOWLEDGED" => com.mongodb.WriteConcern.REPLICAS_SAFE
164 | case "FSYNC_SAFE" | "FSYNCED" => com.mongodb.WriteConcern.FSYNC_SAFE
165 | case "MAJORITY" => com.mongodb.WriteConcern.MAJORITY
166 | case "JOURNAL_SAFE" | "JOURNALED" => com.mongodb.WriteConcern.JOURNAL_SAFE
167 | case "NONE" | "ERRORS_IGNORED" => com.mongodb.WriteConcern.NONE
168 | case _ => DefaultWriteConcern
169 | }
170 | }
171 | }
172 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/client/MongodbClientFactory.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.client
17 |
18 | import java.util.concurrent._
19 |
20 | import akka.actor.{ActorSystem, Props}
21 | import akka.pattern.ask
22 | import akka.util.Timeout
23 | import com.mongodb.ServerAddress
24 | import com.mongodb.casbah.Imports._
25 | import com.mongodb.casbah.MongoClient
26 | import com.stratio.datasource.mongodb.client.MongodbClientActor._
27 | import com.stratio.datasource.mongodb.config.{MongodbConfig, MongodbSSLOptions}
28 | import com.typesafe.config.ConfigFactory
29 |
30 | import scala.concurrent.Await
31 | import scala.concurrent.duration._
32 |
33 |
34 | /**
35 | * Different client configurations to Mongodb database
36 | */
37 | // TODO Refactor - MongodbClientFactory should be used internally and should not delegate to other when closing/freeing connections
38 | object MongodbClientFactory {
39 |
40 | type Client = MongoClient
41 |
42 | private val CloseAttempts = 120
43 |
44 | /**
45 | * Scheduler that close connections automatically when the timeout was expired
46 | */
47 | private val actorSystem = ActorSystem("mongodbClientFactory", ConfigFactory.load(ConfigFactory.parseString("akka.daemonic=on")))
48 | private val scheduler = actorSystem.scheduler
49 | private val SecondsToCheckConnections = MongodbConfig.DefaultConnectionsTime
50 | private val mongoConnectionsActor = actorSystem.actorOf(Props(new MongodbClientActor), "mongoConnectionActor")
51 |
52 | private implicit val executor = actorSystem.dispatcher
53 | private implicit val timeout: Timeout = Timeout(3.seconds)
54 |
55 | scheduler.schedule(
56 | initialDelay = Duration(SecondsToCheckConnections, TimeUnit.SECONDS),
57 | interval = Duration(SecondsToCheckConnections, TimeUnit.SECONDS),
58 | mongoConnectionsActor,
59 | CheckConnections)
60 |
61 | /**
62 | * Get or Create one client connection to MongoDb
63 | * @param host Ip or Dns to connect
64 | * @return Client connection with identifier
65 | */
66 | private[mongodb] def getClient(host: String): ClientResponse = {
67 | val futureResult = mongoConnectionsActor ? GetClient(host)
68 | Await.result(futureResult, timeout.duration) match {
69 | case ClientResponse(key, clientConnection) => ClientResponse(key, clientConnection)
70 | }
71 | }
72 |
73 | /**
74 | * Get or Create one client connection to MongoDb
75 | * @param host Ip or Dns to connect
76 | * @param port Port to connect
77 | * @param user User for credentials
78 | * @param database Database for credentials
79 | * @param password Password for credentials
80 | * @return Client connection with identifier
81 | */
82 | private[mongodb] def getClient(host: String, port: Int, user: String, database: String, password: String): ClientResponse = {
83 | val futureResult = mongoConnectionsActor ? GetClientWithUser(host, port, user, database, password)
84 | Await.result(futureResult, timeout.duration) match {
85 | case ClientResponse(key, clientConnection) => ClientResponse(key, clientConnection)
86 | }
87 | }
88 |
89 | /**
90 | * Get or Create one client connection to MongoDb
91 | * @param hostPort Server addresses to connect to one MongoDb ReplicaSet or Sharded Cluster
92 | * @param credentials Credentials to connect
93 | * @param optionSSLOptions SSL options for secure connections
94 | * @param clientOptions All options for the client connections
95 | * @return Client connection with identifier
96 | */
97 | private[mongodb] def getClient(hostPort: List[ServerAddress],
98 | credentials: List[MongoCredential] = List(),
99 | optionSSLOptions: Option[MongodbSSLOptions] = None,
100 | clientOptions: Map[String, Any] = Map()): ClientResponse = {
101 | val futureResult =
102 | mongoConnectionsActor ? GetClientWithMongoDbConfig(hostPort, credentials, optionSSLOptions, clientOptions)
103 | Await.result(futureResult, timeout.duration) match {
104 | case ClientResponse(key, clientConnection) => ClientResponse(key, clientConnection)
105 | }
106 | }
107 |
108 |
109 | /**
110 | * Close all client connections on the concurrent map
111 | * @param gracefully Close the connections if is free
112 | */
113 | private[mongodb] def closeAll(gracefully: Boolean = true, attempts: Int = CloseAttempts): Unit = {
114 | mongoConnectionsActor ! CloseAll(gracefully, attempts)
115 | }
116 |
117 | /**
118 | * Close the connections that have the same client as the client param
119 | * @param client client value for connect to MongoDb
120 | * @param gracefully Close the connection if is free
121 | */
122 | private[mongodb] def closeByClient(client: Client, gracefully: Boolean = true): Unit = {
123 | mongoConnectionsActor ! CloseByClient(client, gracefully)
124 | }
125 |
126 | /**
127 | * Close the connections that have the same key as the clientKey param
128 | * @param clientKey key pre calculated with the connection options
129 | * @param gracefully Close the connection if is free
130 | */
131 | private[mongodb] def closeByKey(clientKey: String, gracefully: Boolean = true): Unit = {
132 | mongoConnectionsActor ! CloseByKey(clientKey, gracefully)
133 | }
134 |
135 | /**
136 | * Set Free the connection that have the same client as the client param
137 | * @param client client value for connect to MongoDb
138 | */
139 | private[mongodb] def setFreeConnectionByClient(client: Client, extendedTime: Option[Long] = None): Unit = {
140 | mongoConnectionsActor ! SetFreeConnectionsByClient(client, extendedTime)
141 | }
142 |
143 | /**
144 | * Set Free the connection that have the same key as the clientKey param
145 | * @param clientKey key pre calculated with the connection options
146 | */
147 | private[mongodb] def setFreeConnectionByKey(clientKey: String, extendedTime: Option[Long] = None): Unit = {
148 | mongoConnectionsActor ! SetFreeConnectionByKey(clientKey, extendedTime)
149 | }
150 |
151 | private[client] def getClientPoolSize: Int = {
152 | val futureResult = mongoConnectionsActor ? GetSize
153 | Await.result(futureResult, timeout.duration) match {
154 | case size: Int => size
155 | }
156 | }
157 |
158 | // TODO Review when refactoring config
159 | def extractValue[T](options: Map[String, Any], key: String): Option[T] =
160 | options.get(key.toLowerCase).map(_.asInstanceOf[T])
161 |
162 | def sslBuilder(optionSSLOptions: Option[MongodbSSLOptions]): Boolean =
163 | optionSSLOptions.exists(sslOptions => {
164 | if (sslOptions.keyStore.nonEmpty) {
165 | System.setProperty("javax.net.ssl.keyStore", sslOptions.keyStore.get)
166 | if (sslOptions.keyStorePassword.nonEmpty)
167 | System.setProperty("javax.net.ssl.keyStorePassword", sslOptions.keyStorePassword.get)
168 | }
169 | if (sslOptions.trustStore.nonEmpty) {
170 | System.setProperty("javax.net.ssl.trustStore", sslOptions.trustStore)
171 | if (sslOptions.trustStorePassword.nonEmpty)
172 | System.setProperty("javax.net.ssl.trustStorePassword", sslOptions.trustStorePassword.get)
173 | }
174 | true
175 | })
176 |
177 | }
178 |
--------------------------------------------------------------------------------
/spark-mongodb_2.10/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
19 |
22 | 4.0.0
23 | spark-mongodb_2.10
24 | jar
25 |
26 | spark-mongodb-parent
27 | com.stratio.datasource
28 | 0.13.0-SNAPSHOT
29 |
30 |
31 | 2.10.4
32 | 2.10
33 | spark-mongodb
34 | 2.2.5
35 | 3.2.1
36 | 2.0.0
37 | 2.8.0
38 | 2.3.11
39 |
40 |
41 |
42 | org.scala-lang
43 | scala-library
44 | ${scala.version}
45 |
46 |
47 | org.apache.spark
48 | spark-core_${scala.binary.version}
49 | ${spark.version}
50 | provided
51 |
52 |
53 | org.apache.spark
54 | spark-sql_${scala.binary.version}
55 | ${spark.version}
56 | provided
57 |
58 |
59 | com.typesafe.akka
60 | akka-actor_${scala.binary.version}
61 | ${akka.version}
62 |
63 |
64 | org.mongodb
65 | casbah-commons_${scala.binary.version}
66 | ${casbah.version}
67 |
68 |
69 | org.mongodb
70 | casbah-query_${scala.binary.version}
71 | ${casbah.version}
72 |
73 |
74 | org.mongodb
75 | casbah-core_${scala.binary.version}
76 | ${casbah.version}
77 |
78 |
79 | de.flapdoodle.embed
80 | de.flapdoodle.embed.mongo
81 | 1.46.4
82 | test
83 |
84 |
85 | junit
86 | junit
87 | 4.11
88 | test
89 |
90 |
91 | org.scalatest
92 | scalatest_${scala.binary.version}
93 | ${scala.test.version}
94 | test
95 |
96 |
97 |
98 |
99 |
100 | ../spark-mongodb/src/test/resources
101 | true
102 |
103 |
104 | ../spark-mongodb/src/main/scala
105 | ../spark-mongodb/src/test/scala
106 |
107 |
108 | org.apache.maven.plugins
109 | maven-resources-plugin
110 | 2.7
111 |
112 |
113 | copy-testFilteredResources
114 |
115 | testResources
116 |
117 |
118 |
119 |
120 |
121 | org.apache.maven.plugins
122 | maven-dependency-plugin
123 | 2.8
124 |
125 |
126 | copy-dependencies
127 | package
128 |
129 | copy-dependencies
130 |
131 |
132 | ${project.build.directory}/alternateLocation
133 | false
134 | false
135 | true
136 | ::*
137 |
138 |
139 |
140 |
141 |
142 | net.alchim31.maven
143 | scala-maven-plugin
144 | 3.2.1
145 |
146 | false
147 | ${scala.version}
148 | incremental
149 |
150 |
151 |
152 | scala-compile-first
153 | process-resources
154 |
155 | add-source
156 |
157 |
158 |
159 | scala-compile
160 | compile
161 |
162 | compile
163 |
164 |
165 |
166 | scala-testCompile
167 | test-compile
168 |
169 | testCompile
170 |
171 |
172 |
173 | scala-doc
174 | prepare-package
175 |
176 | doc
177 | doc-jar
178 |
179 |
180 |
181 |
182 |
183 |
184 |
185 |
--------------------------------------------------------------------------------
/spark-mongodb/src/test/scala/com/stratio/datasource/mongodb/writer/MongodbWriterIT.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.writer
17 |
18 | import com.mongodb._
19 | import com.mongodb.casbah.commons.MongoDBObject
20 | import com.mongodb.util.JSON
21 | import com.stratio.datasource.MongodbTestConstants
22 | import com.stratio.datasource.mongodb.client.MongodbClientFactory
23 | import com.stratio.datasource.mongodb.config.{MongodbConfig, MongodbConfigBuilder}
24 | import com.stratio.datasource.mongodb.{MongoEmbedDatabase, TestBsonData}
25 | import org.junit.runner.RunWith
26 | import org.scalatest.junit.JUnitRunner
27 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FlatSpec, Matchers}
28 |
29 | @RunWith(classOf[JUnitRunner])
30 | class MongodbWriterIT extends FlatSpec
31 | with Matchers
32 | with MongoEmbedDatabase
33 | with TestBsonData
34 | with MongodbTestConstants
35 | with BeforeAndAfterAll {
36 |
37 | private val host: String = "localhost"
38 | private val collection: String = "testCol"
39 | private val writeConcern = "NORMAL"
40 | private val idField: String = "att2"
41 | private val updateField: Array[String] = Array("att3")
42 | private val wrongIdField: String = "non-existentColumn"
43 | private val language: String = "english"
44 |
45 |
46 | val testConfig = MongodbConfigBuilder()
47 | .set(MongodbConfig.Host, List(host + ":" + mongoPort))
48 | .set(MongodbConfig.Database, db)
49 | .set(MongodbConfig.Collection, collection)
50 | .set(MongodbConfig.SamplingRatio, 1.0)
51 | .set(MongodbConfig.WriteConcern, writeConcern)
52 | .build()
53 |
54 | val testConfigWithPk = MongodbConfigBuilder()
55 | .set(MongodbConfig.Host, List(host + ":" + mongoPort))
56 | .set(MongodbConfig.Database, db)
57 | .set(MongodbConfig.Collection, collection)
58 | .set(MongodbConfig.SamplingRatio, 1.0)
59 | .set(MongodbConfig.WriteConcern, writeConcern)
60 | .build()
61 |
62 | val testConfigWithLanguage = MongodbConfigBuilder()
63 | .set(MongodbConfig.Host, List(host + ":" + mongoPort))
64 | .set(MongodbConfig.Database, db)
65 | .set(MongodbConfig.Collection, collection)
66 | .set(MongodbConfig.SamplingRatio, 1.0)
67 | .set(MongodbConfig.WriteConcern, writeConcern)
68 | .set(MongodbConfig.Language, language)
69 | .build()
70 |
71 | val testConfigWithWrongPk = MongodbConfigBuilder()
72 | .set(MongodbConfig.Host, List(host + ":" + mongoPort))
73 | .set(MongodbConfig.Database, db)
74 | .set(MongodbConfig.Collection, collection)
75 | .set(MongodbConfig.SamplingRatio, 1.0)
76 | .set(MongodbConfig.WriteConcern, writeConcern)
77 | .build()
78 |
79 | val testConfigWithUpdateFields = MongodbConfigBuilder()
80 | .set(MongodbConfig.Host, List(host + ":" + mongoPort))
81 | .set(MongodbConfig.Database, db)
82 | .set(MongodbConfig.Collection, collection)
83 | .set(MongodbConfig.SamplingRatio, 1.0)
84 | .set(MongodbConfig.WriteConcern, writeConcern)
85 | .set(MongodbConfig.UpdateFields, updateField)
86 | .build()
87 |
88 | val dbObject = JSON.parse(
89 | """{ "att5" : [ 1 , 2 , 3] ,
90 | "att4" : null ,
91 | "att3" : "hi" ,
92 | "att6" : { "att61" : 1 , "att62" : null } ,
93 | "att2" : 2.0 ,
94 | "att1" : 1}""").asInstanceOf[DBObject]
95 |
96 | val listDbObject = List(
97 | JSON.parse(
98 | """{ "att5" : [ 1 , 2 , 3] ,
99 | "att4" : null ,
100 | "att3" : "hi" ,
101 | "att6" : { "att61" : 1 , "att62" : null } ,
102 | "att2" : 2.0 ,
103 | "att1" : 1}""").asInstanceOf[DBObject],
104 | JSON.parse(
105 | """{ "att5" : [ 1 , 2 , 3] ,
106 | "att4" : null ,
107 | "att3" : "holo" ,
108 | "att6" : { "att61" : 1 , "att62" : null } ,
109 | "att2" : 2.0 ,
110 | "att1" : 1}""").asInstanceOf[DBObject])
111 |
112 | val updateDbObject = List(
113 | JSON.parse(
114 | """{ "att5" : [ 1 , 2 , 3] ,
115 | "att4" : null ,
116 | "att3" : "holo" ,
117 | "att6" : { "att61" : 1 , "att62" : null } ,
118 | "att2" : 2.0 ,
119 | "att1" : 2}""").asInstanceOf[DBObject])
120 |
121 | behavior of "A writer"
122 |
123 | it should "properly write in a Mongo collection using the Simple Writer" + scalaBinaryVersion in {
124 |
125 | withEmbedMongoFixture(List()) { mongodbProc =>
126 |
127 | val mongodbSimpleWriter = new MongodbSimpleWriter(testConfig)
128 |
129 | val dbOIterator = List(dbObject).iterator
130 |
131 | mongodbSimpleWriter.saveWithPk(dbOIterator)
132 |
133 | val mongodbClient = new MongoClient(host, mongoPort)
134 |
135 | val dbCollection = mongodbClient.getDB(db).getCollection(collection)
136 |
137 | val dbCursor = dbCollection.find()
138 |
139 | import scala.collection.JavaConversions._
140 |
141 | dbCursor.iterator().toList should equal(List(dbObject))
142 |
143 | }
144 | }
145 |
146 | it should "properly write in a Mongo collection using the Batch Writer" + scalaBinaryVersion in {
147 |
148 | withEmbedMongoFixture(List()) { mongodbProc =>
149 |
150 | val mongodbBatchWriter = new MongodbBatchWriter(testConfig)
151 |
152 | val dbOIterator = List(dbObject).iterator
153 |
154 | mongodbBatchWriter.saveWithPk(dbOIterator)
155 |
156 | val mongodbClient = new MongoClient(host, mongoPort)
157 |
158 | val dbCollection = mongodbClient.getDB(db).getCollection(collection)
159 |
160 | val dbCursor = dbCollection.find()
161 |
162 | import scala.collection.JavaConversions._
163 |
164 | dbCursor.iterator().toList should equal(List(dbObject))
165 |
166 | }
167 | }
168 |
169 | it should "manage the incorrect primary key, created in a column that" +
170 | " doesn't exist, rightly" + scalaBinaryVersion in {
171 | withEmbedMongoFixture(List()) { mongodbProc =>
172 |
173 | val mongodbBatchWriter = new MongodbBatchWriter(testConfigWithWrongPk)
174 |
175 | val dbOIterator = List(dbObject).iterator
176 |
177 | mongodbBatchWriter.saveWithPk(dbOIterator)
178 |
179 | val mongodbClient = new MongoClient(host, mongoPort)
180 |
181 | val dbCollection = mongodbClient.getDB(db).getCollection(collection)
182 |
183 | val dbCursor = dbCollection.find()
184 |
185 | import scala.collection.JavaConversions._
186 |
187 | dbCursor.iterator().toList.forall { case obj: BasicDBObject =>
188 | obj.get("_id") != obj.get("non-existentColumn")
189 |
190 | } should be (true)
191 | }
192 | }
193 |
194 | it should "manage the language field for text index" + scalaBinaryVersion in {
195 | withEmbedMongoFixture(List()) { mongodbProc =>
196 |
197 | val mongodbBatchWriter = new MongodbBatchWriter(testConfigWithLanguage)
198 |
199 | val dbOIterator = List(dbObject).iterator
200 |
201 | mongodbBatchWriter.saveWithPk(dbOIterator)
202 |
203 | val mongodbClient = new MongoClient(host, mongoPort)
204 |
205 | val dbCollection = mongodbClient.getDB(db).getCollection(collection)
206 |
207 | val dbCursor = dbCollection.find()
208 |
209 | import scala.collection.JavaConversions._
210 |
211 | dbCursor.iterator().toList.forall { case obj: BasicDBObject =>
212 | obj.get("language") == language
213 | } should be (true)
214 | }
215 | }
216 |
217 | it should "manage the update fields and the update query, it has to read the same value from the fields in " +
218 | "configuration" + scalaBinaryVersion in {
219 | withEmbedMongoFixture(List()) { mongodbProc =>
220 |
221 | val mongodbBatchWriter = new MongodbBatchWriter(testConfigWithUpdateFields)
222 |
223 | val dbOIterator = listDbObject.iterator
224 |
225 | val dbUpdateIterator = updateDbObject.iterator
226 |
227 | mongodbBatchWriter.saveWithPk(dbOIterator)
228 |
229 | val mongodbClient = new MongoClient(host, mongoPort)
230 |
231 | val dbCollection = mongodbClient.getDB(db).getCollection(collection)
232 |
233 | val dbCursor = dbCollection.find(MongoDBObject("att3" -> "holo"))
234 |
235 | import scala.collection.JavaConversions._
236 |
237 | dbCursor.iterator().toList.foreach{ case obj: BasicDBObject =>
238 | obj.getInt("att1") should be (1)
239 | }
240 |
241 | mongodbBatchWriter.saveWithPk(dbUpdateIterator)
242 |
243 | val dbCursor2 = dbCollection.find(MongoDBObject("att3" -> "holo"))
244 |
245 | dbCursor2.iterator().toList.foreach { case obj: BasicDBObject =>
246 | obj.getInt("att1") should be (2)
247 | }
248 |
249 | }
250 | }
251 |
252 | override def afterAll {
253 | MongodbClientFactory.closeAll(false)
254 | }
255 |
256 | }
--------------------------------------------------------------------------------
/spark-mongodb/src/test/scala/com/stratio/datasource/mongodb/reader/MongodbReaderIT.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.reader
17 |
18 | import java.sql.Timestamp
19 | import java.text.SimpleDateFormat
20 | import java.util.Locale
21 |
22 | import com.mongodb.util.JSON
23 | import com.mongodb.{BasicDBObject, DBObject}
24 | import com.stratio.datasource.MongodbTestConstants
25 | import com.stratio.datasource.mongodb._
26 | import com.stratio.datasource.mongodb.client.MongodbClientFactory
27 | import com.stratio.datasource.mongodb.config.{MongodbConfig, MongodbConfigBuilder}
28 | import com.stratio.datasource.mongodb.partitioner.MongodbPartition
29 | import com.stratio.datasource.mongodb.query.FilterSection
30 | import com.stratio.datasource.partitioner.PartitionRange
31 | import org.apache.spark.sql.Row
32 | import org.apache.spark.sql.mongodb.{TemporaryTestSQLContext, TestSQLContext}
33 | import org.apache.spark.sql.sources.{EqualTo, Filter}
34 | import org.apache.spark.sql.types._
35 | import org.junit.runner.RunWith
36 | import org.scalatest.junit.JUnitRunner
37 | import org.scalatest.{BeforeAndAfterAll, BeforeAndAfter, FlatSpec, Matchers}
38 |
39 | @RunWith(classOf[JUnitRunner])
40 | class MongodbReaderIT extends FlatSpec
41 | with Matchers
42 | with MongoEmbedDatabase
43 | with TestBsonData
44 | with MongodbTestConstants
45 | with BeforeAndAfterAll {
46 |
47 | private val host: String = "localhost"
48 | private val collection: String = "testCol"
49 |
50 | implicit val testConfig = MongodbConfigBuilder()
51 | .set(MongodbConfig.Host, List(host + ":" + mongoPort))
52 | .set(MongodbConfig.Database, db)
53 | .set(MongodbConfig.Collection, collection)
54 | .set(MongodbConfig.SamplingRatio, "1.0")
55 | .build()
56 |
57 | behavior of "A reader"
58 |
59 | it should "throw IllegalStateException if next() operation is invoked after closing the Reader" +
60 | scalaBinaryVersion in {
61 | val mongodbReader = new MongodbReader(testConfig,Array(), FilterSection(Array()))
62 | mongodbReader.init(
63 | MongodbPartition(0,
64 | testConfig[Seq[String]](MongodbConfig.Host),
65 | PartitionRange[DBObject](None, None)))
66 |
67 | mongodbReader.close()
68 |
69 | a[IllegalStateException] should be thrownBy {
70 | mongodbReader.next()
71 | }
72 | }
73 |
74 | it should "not advance the cursor position when calling hasNext() operation" + scalaBinaryVersion in {
75 | withEmbedMongoFixture(complexFieldAndType1) { mongodbProc =>
76 |
77 | val mongodbReader = new MongodbReader(testConfig,Array(),FilterSection(Array()))
78 | mongodbReader.init(
79 | MongodbPartition(0,
80 | testConfig[Seq[String]](MongodbConfig.Host),
81 | PartitionRange[DBObject](None, None)))
82 |
83 | (1 until 20).map(_ => mongodbReader.hasNext).distinct.toList==List(true)
84 |
85 | }
86 | }
87 |
88 | it should "advance the cursor position when calling next() operation" + scalaBinaryVersion in {
89 | withEmbedMongoFixture(complexFieldAndType1) { mongodbProc =>
90 |
91 | val mongodbReader = new MongodbReader(testConfig,Array(),FilterSection(Array()))
92 | mongodbReader.init(
93 | MongodbPartition(0,
94 | testConfig[Seq[String]](MongodbConfig.Host),
95 | PartitionRange[DBObject](None, None)))
96 | val posBefore = mongodbReader.hasNext
97 | mongodbReader.next()
98 | val posAfter = mongodbReader.hasNext
99 | posBefore should equal(!posAfter)
100 |
101 | }
102 | }
103 |
104 | it should "properly read java.util.Date (mongodb Date) type as Timestamp" + scalaBinaryVersion in {
105 | val dfunc = (s: String) => new SimpleDateFormat("EEE MMM dd HH:mm:ss Z yyyy", Locale.ENGLISH).parse(s)
106 | import com.mongodb.casbah.Imports.DBObject
107 | val stringAndDate = List(DBObject("string" -> "this is a simple string.", "date" -> dfunc("Mon Aug 10 07:52:49 EDT 2015")))
108 |
109 | withEmbedMongoFixture(stringAndDate) { mongodbProc =>
110 | val back = TemporaryTestSQLContext.fromMongoDB(testConfig)
111 | back.printSchema()
112 | assert(back.schema.fields.filter(_.name == "date").head.dataType == TimestampType)
113 | val timestamp = back.first().get(2).asInstanceOf[Timestamp]
114 | val origTimestamp = new Timestamp(stringAndDate.head.get("date").asInstanceOf[java.util.Date].getTime)
115 | timestamp should equal(origTimestamp)
116 | }
117 | }
118 |
119 | it should "retrieve the data properly filtering & selecting some fields " +
120 | "from a one row table" + scalaBinaryVersion in {
121 | withEmbedMongoFixture(primitiveFieldAndType) { mongodbProc =>
122 | //Test data preparation
123 | val requiredColumns = Array("_id","string", "integer")
124 | val filters = FilterSection(Array[Filter](EqualTo("boolean", true)))
125 | val mongodbReader =
126 | new MongodbReader(testConfig, requiredColumns, filters)
127 |
128 | mongodbReader.init(
129 | MongodbPartition(0,
130 | testConfig[Seq[String]](MongodbConfig.Host),
131 | PartitionRange[DBObject](None, None)))
132 |
133 | //Data retrieving
134 | var l = List[DBObject]()
135 | while (mongodbReader.hasNext){
136 | l = l :+ mongodbReader.next()
137 | }
138 |
139 | //Data validation
140 | l.headOption.foreach{
141 | case obj: BasicDBObject =>
142 | obj.size() should equal(3)
143 | obj.get("string") should equal(
144 | primitiveFieldAndType.head.get("string"))
145 | obj.get("integer") should equal(
146 | primitiveFieldAndType.head.get("integer"))
147 |
148 | }
149 | }
150 |
151 | }
152 |
153 |
154 | it should "retrieve the data properly filtering & selecting some fields " +
155 | "from a five rows table" + scalaBinaryVersion in {
156 | withEmbedMongoFixture(primitiveFieldAndType5rows) { mongodbProc =>
157 |
158 | //Test data preparation
159 | val requiredColumns = Array("_id","string", "integer")
160 | val filters = FilterSection(Array[Filter](EqualTo("boolean", true)))
161 | val mongodbReader =
162 | new MongodbReader(testConfig, requiredColumns, filters)
163 |
164 | mongodbReader.init(
165 | MongodbPartition(0,
166 | testConfig[Seq[String]](MongodbConfig.Host),
167 | PartitionRange[DBObject](None, None)))
168 |
169 | val desiredData =
170 | JSON.parse(
171 | """{"string":"this is a simple string.",
172 | "integer":10
173 | }""").asInstanceOf[DBObject] ::
174 | JSON.parse(
175 | """{"string":"this is the third simple string.",
176 | "integer":12
177 | }""").asInstanceOf[DBObject] ::
178 | JSON.parse(
179 | """{"string":"this is the forth simple string.",
180 | "integer":13
181 | }""").asInstanceOf[DBObject] :: Nil
182 |
183 | //Data retrieving
184 | var l = List[BasicDBObject]()
185 | while (mongodbReader.hasNext){
186 | l = l :+ mongodbReader.next().asInstanceOf[BasicDBObject]
187 | }
188 |
189 | //Data validation
190 |
191 | def pruneId(dbObject: BasicDBObject):BasicDBObject ={
192 | import scala.collection.JavaConversions._
193 | import scala.collection.JavaConverters._
194 | new BasicDBObject(dbObject.toMap.asScala.filter{case (k,v) => k!="_id"})
195 | }
196 | val desiredL = l.map(pruneId)
197 |
198 | l.size should equal(3)
199 | desiredData.diff(desiredL) should equal (List())
200 | l.headOption.foreach{
201 | case obj: BasicDBObject =>
202 | obj.size() should equal(3)
203 | obj.get("string") should equal(
204 | primitiveFieldAndType5rows.head.get("string"))
205 | obj.get("integer") should equal(
206 | primitiveFieldAndType5rows.head.get("integer"))
207 |
208 | }
209 | }
210 | }
211 |
212 | it should "retrieve data correctly using a NOT filter" + scalaBinaryVersion in {
213 | withEmbedMongoFixture(primitiveFieldAndType5rows) { mongodbProc =>
214 |
215 | val mongoDF = TemporaryTestSQLContext.fromMongoDB(testConfig)
216 | mongoDF.registerTempTable("testTable")
217 |
218 | val resultNotBetween = TemporaryTestSQLContext.sql("SELECT integer FROM testTable WHERE integer NOT BETWEEN 11 AND 15").collect()
219 | resultNotBetween.head(0) should be (10)
220 |
221 | val resultEqualToAndNotBetween = TemporaryTestSQLContext.sql("SELECT integer FROM testTable WHERE integer = 11 AND integer NOT BETWEEN 12 AND 15").collect()
222 | resultEqualToAndNotBetween.head(0) should be (11)
223 |
224 | val resultNotLike = TemporaryTestSQLContext.sql("SELECT string FROM testTable WHERE string NOT LIKE '%third%'").collect()
225 |
226 | val notLike = Array(Row("this is a simple string."),
227 | Row("this is another simple string."),
228 | Row("this is the forth simple string."),
229 | Row("this is the fifth simple string."))
230 |
231 | resultNotLike should be (notLike)
232 |
233 | }
234 | }
235 |
236 |
237 | override def afterAll {
238 | MongodbClientFactory.closeAll(false)
239 | }
240 |
241 | }
242 |
--------------------------------------------------------------------------------
/spark-mongodb/src/main/scala/com/stratio/datasource/mongodb/partitioner/MongodbPartitioner.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (C) 2015 Stratio (http://stratio.com)
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 | package com.stratio.datasource.mongodb.partitioner
17 |
18 | import java.text.SimpleDateFormat
19 |
20 | import com.mongodb.casbah.Imports._
21 | import com.mongodb.{MongoCredential, ServerAddress}
22 | import com.stratio.datasource.mongodb.client.MongodbClientFactory.Client
23 | import com.stratio.datasource.mongodb.client.MongodbClientFactory
24 | import com.stratio.datasource.mongodb.config.{MongodbConfig, MongodbCredentials, MongodbSSLOptions}
25 | import com.stratio.datasource.mongodb.partitioner.MongodbPartitioner._
26 | import com.stratio.datasource.mongodb.util.usingMongoClient
27 | import com.stratio.datasource.partitioner.{PartitionRange, Partitioner}
28 | import com.stratio.datasource.util.Config
29 |
30 | import scala.util.Try
31 |
32 | /**
33 | * @param config Partition configuration
34 | */
35 | class MongodbPartitioner(config: Config) extends Partitioner[MongodbPartition] {
36 |
37 | @transient private val hosts: List[ServerAddress] =
38 | config[List[String]](MongodbConfig.Host)
39 | .map(add => new ServerAddress(add))
40 |
41 | @transient private val credentials: List[MongoCredential] =
42 | config.getOrElse[List[MongodbCredentials]](MongodbConfig.Credentials, MongodbConfig.DefaultCredentials).map {
43 | case MongodbCredentials(user, database, password) =>
44 | MongoCredential.createCredential(user, database, password)
45 | }
46 |
47 | @transient private val ssloptions: Option[MongodbSSLOptions] =
48 | config.get[MongodbSSLOptions](MongodbConfig.SSLOptions)
49 |
50 | private val clientOptions = {
51 | val lowerCaseOptions = MongodbConfig.ListMongoClientOptions.map(_.toLowerCase).toSet
52 | config.properties.filter { case (k, _) => lowerCaseOptions contains k }
53 | }
54 |
55 | private val databaseName: String = config(MongodbConfig.Database)
56 |
57 | private val collectionName: String = config(MongodbConfig.Collection)
58 |
59 | private val collectionFullName: String = s"$databaseName.$collectionName"
60 |
61 | private val connectionsTime = config.get[String](MongodbConfig.ConnectionsTime).map(_.toLong)
62 |
63 | private val cursorBatchSize = config.getOrElse[Int](MongodbConfig.CursorBatchSize, MongodbConfig.DefaultCursorBatchSize)
64 |
65 | override def computePartitions(): Array[MongodbPartition] =
66 | usingMongoClient(MongodbClientFactory.getClient(hosts, credentials, ssloptions, clientOptions).clientConnection) { mongoClient =>
67 | if (isShardedCollection(mongoClient))
68 | computeShardedChunkPartitions(mongoClient)
69 | else
70 | computeNotShardedPartitions(mongoClient)
71 | }
72 |
73 | /**
74 | * @return Whether this is a sharded collection or not
75 | */
76 | protected def isShardedCollection(mongoClient: Client): Boolean = {
77 |
78 | val collection = mongoClient(databaseName)(collectionName)
79 | val isSharded = collection.stats.ok && collection.stats.getBoolean("sharded", false)
80 |
81 | isSharded
82 | }
83 |
84 | /**
85 | * @return MongoDB partitions as sharded chunks.
86 | */
87 | protected def computeShardedChunkPartitions(mongoClient: Client): Array[MongodbPartition] = {
88 |
89 | val partitions = Try {
90 | val chunksCollection = mongoClient(ConfigDatabase)(ChunksCollection)
91 | val dbCursor = chunksCollection.find(MongoDBObject("ns" -> collectionFullName))
92 | val shards = describeShardsMap(mongoClient)
93 | val partitions = dbCursor.zipWithIndex.map {
94 | case (chunk: DBObject, i: Int) =>
95 | val lowerBound = chunk.getAs[DBObject]("min")
96 | val upperBound = chunk.getAs[DBObject]("max")
97 | val hosts: Seq[String] = (for {
98 | shard <- chunk.getAs[String]("shard")
99 | hosts <- shards.get(shard)
100 | } yield hosts).getOrElse(Seq[String]())
101 |
102 | MongodbPartition(i,
103 | hosts,
104 | PartitionRange(lowerBound, upperBound))
105 | }.toArray
106 |
107 | dbCursor.close()
108 |
109 | partitions
110 | }.recover {
111 | case _: Exception =>
112 | val serverAddressList: Seq[String] = mongoClient.allAddress.map {
113 | server => server.getHost + ":" + server.getPort
114 | }.toSeq
115 | Array(MongodbPartition(0, serverAddressList, PartitionRange(None, None)))
116 | }.get
117 |
118 | partitions
119 | }
120 |
121 | /**
122 | * @return Array of not-sharded MongoDB partitions.
123 | */
124 | protected def computeNotShardedPartitions(mongoClient: Client): Array[MongodbPartition] = {
125 | val ranges = splitRanges(mongoClient)
126 | val serverAddressList: Seq[String] = mongoClient.allAddress.map {
127 | server => server.getHost + ":" + server.getPort
128 | }.toSeq
129 | val partitions: Array[MongodbPartition] = ranges.zipWithIndex.map {
130 | case ((previous: Option[DBObject], current: Option[DBObject]), i) =>
131 | MongodbPartition(i,
132 | serverAddressList,
133 | PartitionRange(previous, current))
134 | }.toArray
135 |
136 | partitions
137 | }
138 |
139 | /**
140 | * @return A sequence of minimum and maximum DBObject in range.
141 | */
142 | protected def splitRanges(mongoClient: Client): Seq[(Option[DBObject], Option[DBObject])] = {
143 |
144 | def BoundWithCorrectType(value: String, dataType: String) : Any = dataType match {
145 | case "isoDate" => convertToISODate(value)
146 | case "int" => value.toInt
147 | case "long" => value.toLong
148 | case "double" => value.toDouble
149 | case "string" => value
150 | case _ => throw new IllegalArgumentException(s"Illegal type $dataType for ${MongodbConfig.SplitKeyType} parameter.")
151 | }
152 |
153 | def convertToISODate(value: String) : java.util.Date = {
154 | val dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'")
155 | dateFormat.parse(value)
156 | }
157 |
158 | val splitKey = config.getOrElse(MongodbConfig.SplitKey, MongodbConfig.DefaultSplitKey)
159 |
160 | val requiredCustomSplitParams = Seq(MongodbConfig.SplitKeyMin, MongodbConfig.SplitKeyMax, MongodbConfig.SplitKeyType)
161 |
162 | val customSplitIsDefined = requiredCustomSplitParams.forall(key => config.get(key).isDefined)
163 |
164 | val (splitBounds , splitKeyMin , splitKeyMax) = if(customSplitIsDefined){
165 |
166 | val keyType = config[String](MongodbConfig.SplitKeyType)
167 | val splitKeyMinValue = BoundWithCorrectType(config[String](MongodbConfig.SplitKeyMin), keyType)
168 | val splitKeyMaxValue = BoundWithCorrectType(config[String](MongodbConfig.SplitKeyMax), keyType)
169 |
170 | val splitKeyMin = MongoDBObject(splitKey -> splitKeyMinValue)
171 | val splitKeyMax = MongoDBObject(splitKey -> splitKeyMaxValue)
172 |
173 | val bounds = MongoDBObject(
174 | "min" -> splitKeyMin,
175 | "max" -> splitKeyMax
176 | )
177 |
178 | (bounds, Some(splitKeyMin), Some(splitKeyMax))
179 | }
180 | else (MongoDBObject.empty, None, None)
181 |
182 | val maxChunkSize = config.get[String](MongodbConfig.SplitSize).map(_.toInt)
183 | .getOrElse(MongodbConfig.DefaultSplitSize)
184 |
185 | val cmd: MongoDBObject = MongoDBObject(
186 | "splitVector" -> collectionFullName,
187 | "keyPattern" -> MongoDBObject(splitKey -> 1),
188 | "force" -> false,
189 | "maxChunkSize" -> maxChunkSize
190 | ) ++ splitBounds
191 |
192 | val ranges = Try {
193 | val data = mongoClient("admin").command(cmd)
194 | val splitKeys = data.as[List[DBObject]]("splitKeys").map(Option(_))
195 | val ranges = (splitKeyMin +: splitKeys) zip (splitKeys :+ splitKeyMax)
196 |
197 | ranges.toSeq
198 | }.recover {
199 | case _: Exception =>
200 | val stats = mongoClient(databaseName)(collectionName).stats
201 | val shards = mongoClient(ConfigDatabase)(ShardsCollection)
202 | .find(MongoDBObject("_id" -> stats.getString("primary"))).batchSize(cursorBatchSize)
203 | val shard = shards.next()
204 | val shardHost: String = shard.as[String]("host").replace(shard.get("_id") + "/", "")
205 |
206 | usingMongoClient(MongodbClientFactory.getClient(shardHost).clientConnection){ mongoClient =>
207 | val data = mongoClient.getDB("admin").command(cmd)
208 | val splitKeys = data.as[List[DBObject]]("splitKeys").map(Option(_))
209 | val ranges = (splitKeyMin +: splitKeys) zip (splitKeys :+ splitKeyMax )
210 |
211 | shards.close()
212 | ranges.toSeq
213 | }
214 |
215 | }.getOrElse(Seq((None, None)))
216 |
217 | ranges
218 | }
219 |
220 | /**
221 | * @return Map of shards.
222 | */
223 | protected def describeShardsMap(mongoClient: Client): Map[String, Seq[String]] = {
224 | val shardsCollection = mongoClient(ConfigDatabase)(ShardsCollection)
225 | val shardsFind = shardsCollection.find()
226 | val shards = shardsFind.map { shard =>
227 | val hosts: Seq[String] = shard.getAs[String]("host")
228 | .fold(ifEmpty = Seq[String]())(_.split(",").map(_.split("/").reverse.head).toSeq)
229 | (shard.as[String]("_id"), hosts)
230 | }.toMap
231 |
232 | shardsFind.close()
233 |
234 | shards
235 | }
236 | }
237 |
238 | object MongodbPartitioner {
239 |
240 | val ConfigDatabase = "config"
241 | val ChunksCollection = "chunks"
242 | val ShardsCollection = "shards"
243 | }
244 |
--------------------------------------------------------------------------------