├── .gitignore
├── LICENSE
├── README.md
├── pom.xml
└── src
└── main
└── scala
└── kr
└── ac
└── yonsei
└── delab
└── addb_srconnector
├── ADDBRelation.scala
├── Configurable.scala
├── DefaultSource.scala
├── PipelinePool.scala
├── RedisCluster.scala
├── RedisConfig.scala
├── RedisConnection.scala
├── RedisStore.scala
├── partition
└── RedisPartition.scala
├── rdd
└── ADDBRDD.scala
└── util
├── Filters.scala
├── KeyUtil.scala
└── Logging.scala
/.gitignore:
--------------------------------------------------------------------------------
1 | bin/
2 | project/
3 | target/
4 | metastore_db/
5 | tmp/
6 | *.sbt
7 | derby.log
8 | spark-warehouse/
9 |
--------------------------------------------------------------------------------
/LICENSE:
--------------------------------------------------------------------------------
1 | Apache License
2 | Version 2.0, January 2004
3 | http://www.apache.org/licenses/
4 |
5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
6 |
7 | 1. Definitions.
8 |
9 | "License" shall mean the terms and conditions for use, reproduction,
10 | and distribution as defined by Sections 1 through 9 of this document.
11 |
12 | "Licensor" shall mean the copyright owner or entity authorized by
13 | the copyright owner that is granting the License.
14 |
15 | "Legal Entity" shall mean the union of the acting entity and all
16 | other entities that control, are controlled by, or are under common
17 | control with that entity. For the purposes of this definition,
18 | "control" means (i) the power, direct or indirect, to cause the
19 | direction or management of such entity, whether by contract or
20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the
21 | outstanding shares, or (iii) beneficial ownership of such entity.
22 |
23 | "You" (or "Your") shall mean an individual or Legal Entity
24 | exercising permissions granted by this License.
25 |
26 | "Source" form shall mean the preferred form for making modifications,
27 | including but not limited to software source code, documentation
28 | source, and configuration files.
29 |
30 | "Object" form shall mean any form resulting from mechanical
31 | transformation or translation of a Source form, including but
32 | not limited to compiled object code, generated documentation,
33 | and conversions to other media types.
34 |
35 | "Work" shall mean the work of authorship, whether in Source or
36 | Object form, made available under the License, as indicated by a
37 | copyright notice that is included in or attached to the work
38 | (an example is provided in the Appendix below).
39 |
40 | "Derivative Works" shall mean any work, whether in Source or Object
41 | form, that is based on (or derived from) the Work and for which the
42 | editorial revisions, annotations, elaborations, or other modifications
43 | represent, as a whole, an original work of authorship. For the purposes
44 | of this License, Derivative Works shall not include works that remain
45 | separable from, or merely link (or bind by name) to the interfaces of,
46 | the Work and Derivative Works thereof.
47 |
48 | "Contribution" shall mean any work of authorship, including
49 | the original version of the Work and any modifications or additions
50 | to that Work or Derivative Works thereof, that is intentionally
51 | submitted to Licensor for inclusion in the Work by the copyright owner
52 | or by an individual or Legal Entity authorized to submit on behalf of
53 | the copyright owner. For the purposes of this definition, "submitted"
54 | means any form of electronic, verbal, or written communication sent
55 | to the Licensor or its representatives, including but not limited to
56 | communication on electronic mailing lists, source code control systems,
57 | and issue tracking systems that are managed by, or on behalf of, the
58 | Licensor for the purpose of discussing and improving the Work, but
59 | excluding communication that is conspicuously marked or otherwise
60 | designated in writing by the copyright owner as "Not a Contribution."
61 |
62 | "Contributor" shall mean Licensor and any individual or Legal Entity
63 | on behalf of whom a Contribution has been received by Licensor and
64 | subsequently incorporated within the Work.
65 |
66 | 2. Grant of Copyright License. Subject to the terms and conditions of
67 | this License, each Contributor hereby grants to You a perpetual,
68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
69 | copyright license to reproduce, prepare Derivative Works of,
70 | publicly display, publicly perform, sublicense, and distribute the
71 | Work and such Derivative Works in Source or Object form.
72 |
73 | 3. Grant of Patent License. Subject to the terms and conditions of
74 | this License, each Contributor hereby grants to You a perpetual,
75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
76 | (except as stated in this section) patent license to make, have made,
77 | use, offer to sell, sell, import, and otherwise transfer the Work,
78 | where such license applies only to those patent claims licensable
79 | by such Contributor that are necessarily infringed by their
80 | Contribution(s) alone or by combination of their Contribution(s)
81 | with the Work to which such Contribution(s) was submitted. If You
82 | institute patent litigation against any entity (including a
83 | cross-claim or counterclaim in a lawsuit) alleging that the Work
84 | or a Contribution incorporated within the Work constitutes direct
85 | or contributory patent infringement, then any patent licenses
86 | granted to You under this License for that Work shall terminate
87 | as of the date such litigation is filed.
88 |
89 | 4. Redistribution. You may reproduce and distribute copies of the
90 | Work or Derivative Works thereof in any medium, with or without
91 | modifications, and in Source or Object form, provided that You
92 | meet the following conditions:
93 |
94 | (a) You must give any other recipients of the Work or
95 | Derivative Works a copy of this License; and
96 |
97 | (b) You must cause any modified files to carry prominent notices
98 | stating that You changed the files; and
99 |
100 | (c) You must retain, in the Source form of any Derivative Works
101 | that You distribute, all copyright, patent, trademark, and
102 | attribution notices from the Source form of the Work,
103 | excluding those notices that do not pertain to any part of
104 | the Derivative Works; and
105 |
106 | (d) If the Work includes a "NOTICE" text file as part of its
107 | distribution, then any Derivative Works that You distribute must
108 | include a readable copy of the attribution notices contained
109 | within such NOTICE file, excluding those notices that do not
110 | pertain to any part of the Derivative Works, in at least one
111 | of the following places: within a NOTICE text file distributed
112 | as part of the Derivative Works; within the Source form or
113 | documentation, if provided along with the Derivative Works; or,
114 | within a display generated by the Derivative Works, if and
115 | wherever such third-party notices normally appear. The contents
116 | of the NOTICE file are for informational purposes only and
117 | do not modify the License. You may add Your own attribution
118 | notices within Derivative Works that You distribute, alongside
119 | or as an addendum to the NOTICE text from the Work, provided
120 | that such additional attribution notices cannot be construed
121 | as modifying the License.
122 |
123 | You may add Your own copyright statement to Your modifications and
124 | may provide additional or different license terms and conditions
125 | for use, reproduction, or distribution of Your modifications, or
126 | for any such Derivative Works as a whole, provided Your use,
127 | reproduction, and distribution of the Work otherwise complies with
128 | the conditions stated in this License.
129 |
130 | 5. Submission of Contributions. Unless You explicitly state otherwise,
131 | any Contribution intentionally submitted for inclusion in the Work
132 | by You to the Licensor shall be under the terms and conditions of
133 | this License, without any additional terms or conditions.
134 | Notwithstanding the above, nothing herein shall supersede or modify
135 | the terms of any separate license agreement you may have executed
136 | with Licensor regarding such Contributions.
137 |
138 | 6. Trademarks. This License does not grant permission to use the trade
139 | names, trademarks, service marks, or product names of the Licensor,
140 | except as required for reasonable and customary use in describing the
141 | origin of the Work and reproducing the content of the NOTICE file.
142 |
143 | 7. Disclaimer of Warranty. Unless required by applicable law or
144 | agreed to in writing, Licensor provides the Work (and each
145 | Contributor provides its Contributions) on an "AS IS" BASIS,
146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
147 | implied, including, without limitation, any warranties or conditions
148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
149 | PARTICULAR PURPOSE. You are solely responsible for determining the
150 | appropriateness of using or redistributing the Work and assume any
151 | risks associated with Your exercise of permissions under this License.
152 |
153 | 8. Limitation of Liability. In no event and under no legal theory,
154 | whether in tort (including negligence), contract, or otherwise,
155 | unless required by applicable law (such as deliberate and grossly
156 | negligent acts) or agreed to in writing, shall any Contributor be
157 | liable to You for damages, including any direct, indirect, special,
158 | incidental, or consequential damages of any character arising as a
159 | result of this License or out of the use or inability to use the
160 | Work (including but not limited to damages for loss of goodwill,
161 | work stoppage, computer failure or malfunction, or any and all
162 | other commercial damages or losses), even if such Contributor
163 | has been advised of the possibility of such damages.
164 |
165 | 9. Accepting Warranty or Additional Liability. While redistributing
166 | the Work or Derivative Works thereof, You may choose to offer,
167 | and charge a fee for, acceptance of support, warranty, indemnity,
168 | or other liability obligations and/or rights consistent with this
169 | License. However, in accepting such obligations, You may act only
170 | on Your own behalf and on Your sole responsibility, not on behalf
171 | of any other Contributor, and only if You agree to indemnify,
172 | defend, and hold each Contributor harmless for any liability
173 | incurred by, or claims asserted against, such Contributor by reason
174 | of your accepting any such warranty or additional liability.
175 |
176 | END OF TERMS AND CONDITIONS
177 |
178 | APPENDIX: How to apply the Apache License to your work.
179 |
180 | To apply the Apache License to your work, attach the following
181 | boilerplate notice, with the fields enclosed by brackets "[]"
182 | replaced with your own identifying information. (Don't include
183 | the brackets!) The text should be enclosed in the appropriate
184 | comment syntax for the file format. We also recommend that a
185 | file or class name and description of purpose be included on the
186 | same "printed page" as the copyright notice for easier
187 | identification within third-party archives.
188 |
189 | Copyright [yyyy] [name of copyright owner]
190 |
191 | Licensed under the Apache License, Version 2.0 (the "License");
192 | you may not use this file except in compliance with the License.
193 | You may obtain a copy of the License at
194 |
195 | http://www.apache.org/licenses/LICENSE-2.0
196 |
197 | Unless required by applicable law or agreed to in writing, software
198 | distributed under the License is distributed on an "AS IS" BASIS,
199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
200 | See the License for the specific language governing permissions and
201 | limitations under the License.
202 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # ADDB-SRConnector
2 |
3 | ## Requirements
4 |
5 | * Build ADDB(Redis+RocksDB), configure and run redis-server
6 | * Build [ADDB-Jedis](https://bitbucket.org/delab2017/addb-jedis/src/master/)
7 | * Spark v2.0.2 , set SPARK_HOME in .bashrc
8 | * Install maven for Scala
9 |
10 | ## How to build
11 |
12 | ```
13 | mvn clean install
14 | ```
15 |
16 | ## How to run
17 |
18 | ```
19 | spark-sql --jars=/ADDB_SRConnector_Path/target/addb-srconnector-0.0.1-jar-with-dependencies.jar
20 | ```
21 |
22 | ## SQL Example
23 | After run spark-sql,
24 |
25 | ### CREATE
26 | Set "table" option to INT type.(REQUIRED)
27 | Set "parition" option for partitioning specific column.(REQUIRED)
28 | Set "port" option used in redis-server port (REQUIRED)
29 | Set "AUTH" option if use redis requirepass (OPTIONAL)
30 |
31 | ```
32 | CREATE TABLE kv
33 | (col1 STRING, col2 INT, col3 STRING, col4 INT)
34 | USING kr.ac.yonsei.delab.addb_srconnector
35 | OPTIONS (host "127.0.0.1", port "8000", table "1", partitions "col2", AUTH "foobared");
36 | ```
37 |
38 | ### INSERT
39 |
40 | ```
41 | INSERT INTO table kv VALUES ('LJH', 1010, 'CWK', 1004);
42 | ```
43 |
44 | ### SELECT
45 | It is not necessary to apply a filter to the partition key in the "WHERE" clause when "SELECT" statement is called.
46 | But, we recommend to use a filter to the partition key in order to maximize ADDB's performance.
47 | ```
48 | SELECT * FROM kv WHERE col2>1000;
49 | ```
50 |
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
3 | 4.0.0
4 |
5 |
6 | kr.ac.yonsei.delab
7 | addb-parent
8 | 0.0.1
9 |
10 | addb-srconnector
11 | 0.0.1
12 | jar
13 |
14 | ADDB-SRConnector
15 | http://maven.apache.org
16 |
17 |
18 | UTF-8
19 |
20 |
21 |
22 |
23 | org.slf4j
24 | slf4j-simple
25 | 1.7.5
26 | compile
27 |
28 |
29 | org.apache.spark
30 | spark-core_2.10
31 | 2.0.2
32 |
33 |
34 | org.apache.spark
35 | spark-sql_2.10
36 | 2.0.2
37 |
38 |
39 | org.slf4j
40 | log4j-over-slf4j
41 | 1.7.5
42 | compile
43 |
44 |
52 |
53 | org.apache.commons
54 | commons-pool2
55 | 2.4.3
56 |
57 |
65 |
66 | kr.ac.yonsei.delab
67 | addb-jedis
68 | 0.0.2
69 |
73 |
74 |
75 |
76 |
77 |
78 | net.alchim31.maven
79 | scala-maven-plugin
80 | 3.2.2
81 |
82 |
83 | scala-compile
84 | process-resources
85 |
86 | compile
87 |
88 |
89 |
90 |
91 |
92 | org.apache.maven.plugins
93 | maven-assembly-plugin
94 | 2.4
95 |
96 |
97 | make-assembly
98 | package
99 |
100 | single
101 |
102 |
103 |
104 |
105 |
106 | jar-with-dependencies
107 |
108 |
109 |
110 |
111 | org.codehaus.mojo
112 | build-helper-maven-plugin
113 | 1.10
114 |
115 |
116 | add-scala-sources
117 | generate-sources
118 |
119 | add-source
120 |
121 |
122 |
123 | src/main/scala
124 |
125 |
126 |
127 |
128 | add-scala-test-sources
129 | generate-test-sources
130 |
131 | add-test-source
132 |
133 |
134 |
135 | src/test/scala
136 |
137 |
138 |
139 |
140 |
141 |
169 |
170 |
171 |
172 |
--------------------------------------------------------------------------------
/src/main/scala/kr/ac/yonsei/delab/addb_srconnector/ADDBRelation.scala:
--------------------------------------------------------------------------------
1 | package kr.ac.yonsei.delab.addb_srconnector
2 |
3 | import scala.collection.JavaConversions._
4 | import scala.collection.immutable.ListMap
5 | import scala.collection.mutable.Stack
6 |
7 |
8 | import org.apache.spark.rdd.RDD
9 | import org.apache.spark.sql.{DataFrame, SQLContext, Row}
10 | import org.apache.spark.sql.sources._
11 | import org.apache.spark.sql.sources.Filter
12 | import org.apache.spark.sql.types.{StructType, ByteType, ShortType,
13 | IntegerType, LongType, FloatType, DoubleType}
14 |
15 | import redis.clients.addb_jedis.Protocol
16 | import kr.ac.yonsei.delab.addb_srconnector.util.{Logging, KeyUtil}
17 | import kr.ac.yonsei.delab.addb_srconnector.ColumnType.{NumericType, StringType}
18 | import kr.ac.yonsei.delab.addb_srconnector.ConfigurationConstants.{TABLE_KEY, INDICES_KEY, PARTITION_COLUMN_KEY}
19 | import kr.ac.yonsei.delab.addb_srconnector.rdd._
20 |
21 | /*
22 | * ADDB Relation class
23 | * After CREATE TABLE in SparkSQL, [DefaultSource]
24 | * When INSERT and SELECT statement are called, insert and buildScan function operate. [ADDBRelation]
25 | */
26 | case class ADDBRelation (parameters: Map[String,String],
27 | schema: StructType)
28 | (@transient val sqlContext: SQLContext)
29 | extends BaseRelation
30 | with TableScan
31 | with PrunedScan
32 | with PrunedFilteredScan
33 | with Configurable
34 | with InsertableRelation
35 | with Logging {
36 |
37 | def getRedisConfig( configuration: Configuration ): RedisConfig = {
38 | RedisConfigPool.get( configuration );
39 | }
40 |
41 | def buildRedisTable: RedisTable = {
42 | val tableID = configuration.get(TABLE_KEY).toInt
43 | // Check whether current table is in the RedisTableList
44 | if (RedisTableList.checkList(tableID)) {
45 | // Return stored RedisTable
46 | RedisTableList.list.get(tableID).get
47 | }
48 | else {
49 | def buildNewRedisTable: RedisTable = {
50 | val columns: ListMap[String, RedisColumn] = ListMap( schema.fields.map {
51 | field=> // ListMap 타입
52 | ( field.name, new RedisColumn( field.name, field.dataType match { // column type 단순화. Column type은 RedisTable에 NumericType or StringType으로만 구분해놓음
53 | case _@ (ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType) => NumericType
54 | case _ => StringType
55 | }
56 | ) )
57 | }.toSeq:_* )
58 | // ex) { col1 -> RedisColumn(col1, string) }
59 | logDebug( s"[ADDB] Columns: $columns" )
60 |
61 | // Partition can be multiple columns while OPTIONS must get 1 'partitions' key
62 | val partitionColumnNames = configuration.get( PARTITION_COLUMN_KEY ).split(",").map(x => x.trim)
63 |
64 | logInfo( s"[ADDB] Index is not implemented yet.." )
65 | RedisTable(tableID, columns.values.toArray, partitionColumnNames);
66 | }
67 | // Build new RedisTable and insert it into RedisTableList
68 | val newRedisTable = buildNewRedisTable
69 | RedisTableList.insertTableList(tableID, newRedisTable)
70 | newRedisTable
71 | }
72 | }
73 |
74 |
75 |
76 | /** ADDB
77 | * WonKi Choi 2018-05-17
78 | * implementation for Scan operation in SparkSQL
79 | * build scan for returning RDD object.
80 | */
81 |
82 | // TableScan
83 | override def buildScan: RDD[Row] = {
84 | logDebug(s"buildScan: tableScan")
85 | buildScan(schema.fields.map( field => field.name ) )
86 | }
87 |
88 | // PrunedScan
89 | override def buildScan(requiredColumns: Array[String]): RDD[Row] = {
90 | logDebug(s"[ADDB] buildScan: prunedScan")
91 | buildScan( requiredColumns, Array())
92 | }
93 |
94 | // PrunedFilteredScan
95 | override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = {
96 | logDebug(s"[ADDB] buildScan: prunedFilterScan")
97 | // requiredColumns.foreach(x => logInfo(s"requiredColumns : $x"))
98 | logDebug(s"[ADDB] filter size : ${filters.size}")
99 | // filters.foreach(x => logInfo(s"filters : $x"))
100 |
101 | val redisConfig = getRedisConfig( configuration )
102 | val redisTable = buildRedisTable
103 | val rdd = new ADDBRDD(sqlContext.sparkContext, redisConfig, redisTable, requiredColumns, filters)
104 | new RedisRDDAdaptor(rdd, requiredColumns.map{ columnName=> schema(columnName)}, filters, schema)
105 | }
106 |
107 | // InsertableRelation
108 | override def insert(data: DataFrame, overwrite: Boolean): Unit = {
109 | logDebug(s"[ADDB] insert function")
110 | logDebug(s"[ADDB] the number of partition: "+data.rdd.partitions.length) // return 8
111 | // check OVERWRITE command
112 | if (overwrite) {
113 | logWarning(s"[ADDB] Do not implement overwrite command. Thus, operate only append")
114 | }
115 |
116 | // insert RedisRow(RedisTable+Column) into RedisStore
117 | val redisTable = buildRedisTable
118 | val columnsWithIndex = schema.fields.zipWithIndex // ( (field1:StructField, 0) , (field2, 1) , (field3, 2) ... )
119 | val rowRDD = data.rdd
120 |
121 | val redisConfig = getRedisConfig( configuration ) // get current ADDBRelation RedisConfig
122 | val redisStore = redisConfig.getRedisStore(); // ADDBRelationRedisConfig->RedisConfig->RedisStore
123 |
124 |
125 | // call pipeline function based on dataKey
126 | // *[Critical]* Be executed in each node
127 | try {
128 | rowRDD.foreachPartition {
129 | partition => // partition:Iterator[Row]
130 | logDebug(s"[ADDB] start partition loop")
131 |
132 | // 1) make each pipeline
133 | val retainingJedisPool = new RetainingJedisPool()
134 | val pipelinePool = new PipelinePool()
135 | redisStore.redisCluster.nodes.foreach{
136 | node =>
137 | val jedis = retainingJedisPool.add(node)
138 | pipelinePool.add(node.redisConnection.host+":"+node.redisConnection.port.toString, jedis)
139 | }
140 | // Since datakey and partitionInfo are duplicated,
141 | // make once only
142 | // var datakey = new StringBuilder
143 | // var partitionInfo = new StringBuilder
144 |
145 | // 2) fpwrite all rows
146 | partition.foreach {
147 | row => // row:Iterator[Row]
148 |
149 | val columns = columnsWithIndex.map{
150 | pair=>
151 | val columnValue = row.get(pair._2) // Get column from existing row
152 | if ( columnValue == null ) {
153 | ( pair._1.name, null )
154 | } else {
155 | ( pair._1.name, columnValue.toString() )
156 | }
157 | }.toMap
158 | // if (datakey.size == 0 && partitionInfo.size == 0) {
159 | // // Generate partition:= (1) (index, name) -> (2) (index, value)
160 | // val partitionIndexWithName = redisTable.partitionColumnID.zip(redisTable.partitionColumnNames)
161 | // val partitionIndexWithValue = partitionIndexWithName.map{
162 | // column => (column._1, columns.get(column._2).get)}
163 | // val (key, partition) = KeyUtil.generateDataKey(redisTable.id, partitionIndexWithValue)
164 | // partitionInfo.append(partition)
165 | // datakey.append(key)
166 | // }
167 | // redisStore.add(RedisRow(redisTable, columns), pipelinePool, datakey.toString, partitionInfo.toString)
168 | redisStore.add(RedisRow(redisTable, columns), pipelinePool)
169 | }
170 | // 3) synchronize all pipeline
171 | // 4) close all jedis connection
172 | redisStore.redisCluster.nodes.foreach{
173 | node =>
174 | val jedis = retainingJedisPool.get(node)
175 | val pipeline = pipelinePool.get(node.redisConnection.host+":"+node.redisConnection.port.toString)
176 | pipeline.sync
177 | jedis.close
178 | }
179 | }
180 | } catch {
181 | case e : Exception => throw e
182 | }
183 | }
184 | }
--------------------------------------------------------------------------------
/src/main/scala/kr/ac/yonsei/delab/addb_srconnector/Configurable.scala:
--------------------------------------------------------------------------------
1 | package kr.ac.yonsei.delab.addb_srconnector
2 |
3 | import kr.ac.yonsei.delab.addb_srconnector.util.Logging
4 | import scala.collection.immutable.HashMap
5 | import kr.ac.yonsei.delab.addb_srconnector.ConfigurationConstants.{TABLE_KEY, INDICES_KEY, PARTITION_COLUMN_KEY}
6 |
7 | /*
8 | * RedisStore, ADDBRelation is configurable
9 | * Configuration distinguish each RedisStore / each ADDBRelation
10 | */
11 | trait Configurable
12 | extends Logging{
13 | var configuration: Configuration = _
14 |
15 | def configure(conf:Configuration):Unit = {
16 | this.configuration = conf
17 | logInfo(s"[ADDB] $conf is configured")
18 | }
19 | }
20 |
21 | /*
22 | * Configuration class
23 | * parameters := From CREATE TABLE OPTIONS
24 | */
25 | case class Configuration (
26 | parameters:HashMap[String,String])
27 | extends Serializable
28 | with Logging {
29 | // Since options are already checked in createRelation function, do not change to default value
30 | def get(key:String): String = {
31 | parameters.get(key).get.toString
32 | }
33 | def getOrElse(key:String, defaultValue:String):String = {
34 | if ( parameters.get(key).isEmpty && defaultValue == null ) {
35 | null
36 | } else {
37 | parameters.getOrElse(key, defaultValue).toString
38 | }
39 | }
40 | }
41 |
42 | object ConfigurationConstants {
43 | val TABLE_KEY = "table"
44 | val INDICES_KEY = "indices"
45 | val PARTITION_COLUMN_KEY = "partitions"
46 | }
--------------------------------------------------------------------------------
/src/main/scala/kr/ac/yonsei/delab/addb_srconnector/DefaultSource.scala:
--------------------------------------------------------------------------------
1 | package kr.ac.yonsei.delab.addb_srconnector
2 |
3 | import scala.collection.JavaConversions._
4 | //import java.util.HashMap
5 | import scala.collection.immutable.HashMap
6 |
7 | import org.apache.spark.sql.sources._
8 | import org.apache.spark.sql.{SQLContext, SaveMode, DataFrame}
9 | import org.apache.spark.sql.types.StructType
10 |
11 | import kr.ac.yonsei.delab.addb_srconnector.ConfigurationConstants.{TABLE_KEY, INDICES_KEY, PARTITION_COLUMN_KEY}
12 | import kr.ac.yonsei.delab.addb_srconnector.util.Logging
13 |
14 | // When user defines relation by using SQL Statement,
15 | // DefaultSource
16 | class DefaultSource
17 | extends RelationProvider
18 | with SchemaRelationProvider
19 | with CreatableRelationProvider
20 | with DataSourceRegister with Logging{
21 |
22 | // DataSourceRegister
23 | override def shortName(): String = "addb"
24 |
25 | // Check OPTIONS := tableID, partitionInfo, indexInfo
26 | def checkOptions(configuration:Configuration, schema:StructType):Unit = {
27 | // 1) Check table name
28 | try {
29 | val tableID = configuration.get(TABLE_KEY).toInt
30 | } catch {
31 | case e : NumberFormatException => throw new IllegalArgumentException(s"[ADDB][ERROR] table option should be numeric.")
32 | }
33 | // 2) partition info
34 | // Partition can be multiple columns
35 | val partitionInfo = configuration.get(PARTITION_COLUMN_KEY).split(",").map(x => x.trim)
36 | // Check empty
37 | if (partitionInfo.isEmpty) {
38 | throw new IllegalArgumentException( s"[ADDB][ERROR] At least, one partition column is required" )
39 | }
40 | // Check whether partition column names are proper
41 | val schemaColumns = schema.fieldNames
42 | partitionInfo.foreach {
43 | partitionColumn =>
44 | if (!(schemaColumns.contains(partitionColumn))) {
45 | throw new IllegalArgumentException( s"[ADDB][ERROR] Mismatch between schema and partition column name" )
46 | }
47 | }
48 | // TO DO, index
49 | // 3) index info
50 |
51 |
52 | }
53 |
54 | // RelationProvider := do not specify schema
55 | override def createRelation (sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = {
56 | createRelation(sqlContext, parameters, null)
57 | }
58 | // SchemaRelationProvider := specified schema by user
59 | override def createRelation (sqlContext: SQLContext, parameters: Map[String, String], schema:StructType): BaseRelation = {
60 | // logInfo ( s"##[ADDB] createRelation with user schema" )
61 |
62 | // Check user schema because addb does not provide schema-inference feature
63 | if (schema == null) {
64 | throw new IllegalArgumentException( s"[ADDB][ERROR] No schema. Please enter relation schema!" )
65 | }
66 |
67 | // Set configuration based on parameters
68 | val param:HashMap[String, String] = HashMap(parameters.toSeq:_*)
69 | val configuration = Configuration(param)
70 |
71 | checkOptions(configuration, schema)
72 |
73 | // Return and store addbRelation about create relation
74 | val addbRelation = ADDBRelation(parameters, schema)(sqlContext)
75 | addbRelation.configure(configuration)
76 | addbRelation
77 | }
78 | // CreatableRelationProvider := When save DataFrame to data source
79 | // SaveMode => Overwrite, Append, ErrorIfExists, Ignore
80 | override def createRelation (sqlContext: SQLContext, mode: SaveMode, parameters: Map[String, String], data: DataFrame): BaseRelation = {
81 | createRelation(sqlContext, parameters, data.schema)
82 | }
83 | }
--------------------------------------------------------------------------------
/src/main/scala/kr/ac/yonsei/delab/addb_srconnector/PipelinePool.scala:
--------------------------------------------------------------------------------
1 | package kr.ac.yonsei.delab.addb_srconnector
2 |
3 | import java.util.concurrent.ConcurrentHashMap
4 | import scala.collection.JavaConversions._
5 | import scala.collection.mutable.ListBuffer
6 | import redis.clients.addb_jedis.{Jedis, Pipeline}
7 |
8 | /*
9 | In Spark Cluster mode, making all pipeline object in each node is better than serialization
10 | */
11 | class PipelinePool {
12 | @transient private lazy val pipelinePools: ConcurrentHashMap[String, Pipeline] = new ConcurrentHashMap[String, Pipeline]()
13 | def add(hostAndPort:String, jedis:Jedis) = {
14 | pipelinePools.getOrElseUpdate(hostAndPort, jedis.pipelined)
15 | }
16 | def get(hostAndPort:String):Pipeline = {
17 | pipelinePools.get(hostAndPort)
18 | }
19 | }
20 | class RetainingJedisPool {
21 | @transient private lazy val jedisPools: ConcurrentHashMap[RedisNode, Jedis] = new ConcurrentHashMap[RedisNode, Jedis]()
22 | def add(redisNode:RedisNode):Jedis = {
23 | jedisPools.getOrElseUpdate(redisNode, redisNode.connect)
24 | }
25 | def get(redisNode:RedisNode):Jedis = {
26 | jedisPools.get(redisNode)
27 | }
28 | }
--------------------------------------------------------------------------------
/src/main/scala/kr/ac/yonsei/delab/addb_srconnector/RedisCluster.scala:
--------------------------------------------------------------------------------
1 | package kr.ac.yonsei.delab.addb_srconnector
2 |
3 | import scala.collection.JavaConversions._
4 | import redis.clients.addb_jedis.util.SafeEncoder
5 | import kr.ac.yonsei.delab.addb_srconnector.util.Logging
6 |
7 | /*
8 | * RedisCluster class
9 | * From host ip, check related cluster nodes
10 | */
11 | class RedisCluster(val host: RedisConnection)
12 | extends Serializable
13 | with Logging {
14 |
15 | val nodes = getHosts(host)
16 |
17 | private def checkClusterEnabled(redisConnection: RedisConnection): Boolean = {
18 | val conn = redisConnection.connect()
19 | val info = conn.info.split("\n")
20 | val version = info.filter(_.contains("redis_version:"))(0)
21 | val clusterEnable = info.filter(_.contains("cluster_enabled:"))
22 | val mainVersion = version.substring(14, version.indexOf(".")).toInt
23 | val res = mainVersion>2 && clusterEnable.length>0 && clusterEnable(0).contains("1")
24 | conn.close
25 | res
26 | }
27 | // get cluster nodes from host
28 | // connect host and run clusterSlots command in redis
29 | // then, return related node
30 | private def getClusterNodes(redisConnection: RedisConnection): Array[RedisNode] = {
31 | val conn = redisConnection.connect()
32 | val res = conn.clusterSlots().flatMap {
33 | slotInfoObj => {
34 | val slotInfo = slotInfoObj.asInstanceOf[java.util.List[java.lang.Object]]
35 | val sPos = slotInfo.get(0).toString.toInt
36 | val ePos = slotInfo.get(1).toString.toInt
37 | /*
38 | * We will get all the nodes with the slots range [sPos, ePos],
39 | * and create RedisNode for each nodes, the total field of all
40 | * RedisNode are the number of the nodes whose slots range is
41 | * as above, and the idx field is just an index for each node
42 | * which will be used for adding support for slaves and so on.
43 | * And the idx of a master is always 0, we rely on this fact to
44 | * filter master.
45 | */
46 | (0 until (slotInfo.size - 2)).map(
47 | i => {
48 | val node = slotInfo(i + 2).asInstanceOf[java.util.List[java.lang.Object]]
49 | val host = SafeEncoder.encode(node.get(0).asInstanceOf[Array[scala.Byte]])
50 | val port = node.get(1).toString.toInt
51 | RedisNode(new RedisConnection(host, port, redisConnection.auth,
52 | redisConnection.dbNum, redisConnection.timeout),
53 | sPos, ePos, i, slotInfo.size - 2)
54 | })
55 | }
56 | }.toArray
57 | conn.close()
58 | res
59 | }
60 | def getNodes(redisConnection: RedisConnection): Array[RedisNode] = {
61 | if (!checkClusterEnabled(redisConnection)) {
62 | throw new UnsupportedOperationException( s"[ERROR] ADDB must be operated in cluster modes" )
63 | }
64 | getClusterNodes(redisConnection)
65 | }
66 | def getHosts(redisConnection: RedisConnection): Array[RedisNode] = {
67 | getNodes(redisConnection).filter { _.idx == 0 }
68 | }
69 |
70 | // TO DO
71 | def checkNodes(host: String, port: Int):Int = {
72 | var res = -1
73 | for (i <- 0 until nodes.size) {
74 | if (nodes(i).redisConnection.host == host &&
75 | nodes(i).redisConnection.port == port) {
76 | res = i
77 | }
78 | }
79 | res
80 | }
81 |
82 | }
--------------------------------------------------------------------------------
/src/main/scala/kr/ac/yonsei/delab/addb_srconnector/RedisConfig.scala:
--------------------------------------------------------------------------------
1 | package kr.ac.yonsei.delab.addb_srconnector
2 |
3 | import kr.ac.yonsei.delab.addb_srconnector.util.Logging
4 |
5 | /*
6 | * RedisConfig class manage RedisStore
7 | * Each RedisStore is created by configuration
8 | */
9 | class RedisConfig (val configuration:Configuration)
10 | extends Serializable
11 | with Logging {
12 |
13 | @transient private var redisStore: RedisStore = null
14 |
15 | def createRedisStore(): RedisStore = {
16 | val store = new RedisStore( this )
17 | store.configure( configuration )
18 | logDebug( s"[ADDB] $store created" )
19 | store
20 | }
21 |
22 | def getRedisStore(): RedisStore = {
23 | logDebug( s"[ADDB] get RedisStore object" )
24 | this.synchronized {
25 | if ( redisStore == null ) {
26 | redisStore = createRedisStore()
27 | redisStore
28 | } else {
29 | redisStore
30 | }
31 | }
32 | }
33 | }
34 | /*
35 | * For preventing repeated creation of RedisConfig,
36 | * maintain RedisConfigPool
37 | */
38 | object RedisConfigPool {
39 | val pool = scala.collection.mutable.Map[Configuration, RedisConfig]()
40 | def get( configuration: Configuration ): RedisConfig = {
41 | synchronized {
42 | // check whether mutable Map includes RedisConfig object
43 | // if not, add and return new RedisConfig
44 | val res = pool.get(configuration)
45 | if (res == None ) {
46 | val newRedisConfig = new RedisConfig(configuration)
47 | pool += (configuration -> newRedisConfig)
48 | newRedisConfig
49 | } else {
50 | res.get
51 | }
52 | }
53 | }
54 | }
55 |
--------------------------------------------------------------------------------
/src/main/scala/kr/ac/yonsei/delab/addb_srconnector/RedisConnection.scala:
--------------------------------------------------------------------------------
1 | package kr.ac.yonsei.delab.addb_srconnector
2 |
3 | import org.apache.spark.SparkConf
4 | import redis.clients.addb_jedis.{Jedis, JedisPoolConfig, JedisPool, Protocol}
5 | import redis.clients.addb_jedis.util.{JedisURIHelper, SafeEncoder, JedisClusterCRC16}
6 | import redis.clients.addb_jedis.exceptions.JedisConnectionException
7 | import kr.ac.yonsei.delab.addb_srconnector.util.Logging
8 |
9 | import scala.collection.JavaConversions._
10 | import java.util.concurrent.ConcurrentHashMap
11 | import java.net.URI
12 |
13 | // Redis Connection
14 | case class RedisConnection (val host: String = Protocol.DEFAULT_HOST,
15 | val port: Int = Protocol.DEFAULT_PORT,
16 | val auth: String = null,
17 | val dbNum: Int = Protocol.DEFAULT_DATABASE,
18 | val timeout: Int = Protocol.DEFAULT_TIMEOUT)
19 | extends Serializable
20 | with Logging {
21 |
22 | /**
23 | * Constructor from spark config. set params with redis.host, redis.port, redis.auth and redis.db
24 | *
25 | * @param conf spark context config
26 | */
27 | def this(conf: SparkConf) {
28 | this(
29 | conf.get("redis.host", Protocol.DEFAULT_HOST),
30 | conf.getInt("redis.port", Protocol.DEFAULT_PORT),
31 | conf.get("redis.auth", null),
32 | conf.getInt("redis.db", Protocol.DEFAULT_DATABASE),
33 | conf.getInt("redis.timeout", Protocol.DEFAULT_TIMEOUT)
34 | )
35 | }
36 |
37 | /**
38 | * Constructor with Jedis URI
39 | *
40 | * @param uri connection URI in the form of redis://:$password@$host:$port/[dbnum]
41 | */
42 | def this(uri: URI) {
43 | this(uri.getHost, uri.getPort, JedisURIHelper.getPassword(uri), JedisURIHelper.getDBIndex(uri))
44 | }
45 |
46 | /**
47 | * Constructor with Jedis URI from String
48 | *
49 | * @param uri connection URI in the form of redis://:$password@$host:$port/[dbnum]
50 | */
51 | def this(uri :String) {
52 | this(URI.create(uri))
53 | }
54 |
55 | /**
56 | * Connect tries to open a connection to the redis endpoint,
57 | * optionally authenticating and selecting a db
58 | *
59 | * @return a new Jedis instance
60 | */
61 | def connect(): Jedis = {
62 | RedisConnectionPool.connect(this)
63 | }
64 | }
65 | // Redis cluster node
66 | case class RedisNode(val redisConnection: RedisConnection,
67 | val startSlot: Int,
68 | val endSlot: Int,
69 | val idx: Int,
70 | val total: Int)
71 | extends Serializable
72 | with Logging {
73 | def connect(): Jedis = {
74 | logDebug(s"[ADDB] Redisd Node connect")
75 | redisConnection.connect()
76 | }
77 | }
78 |
79 | object RedisConnectionPool {
80 | @transient private lazy val pools: ConcurrentHashMap[RedisConnection, JedisPool] = new ConcurrentHashMap[RedisConnection, JedisPool]()
81 | // Get jedis resource from jedis pool
82 | def connect(redisConnection: RedisConnection): Jedis = {
83 | val pool = pools.getOrElseUpdate(redisConnection,
84 | {
85 | val poolConfig: JedisPoolConfig = new JedisPoolConfig();
86 | // Configuration setting
87 | poolConfig.setMaxTotal(10000)
88 | poolConfig.setMaxWaitMillis(300000)
89 | // poolConfig.setMaxIdle(32)
90 | // poolConfig.setTestOnBorrow(false)
91 | // poolConfig.setTestOnReturn(false)
92 | // poolConfig.setTestWhileIdle(false)
93 | // poolConfig.setMinEvictableIdleTimeMillis(60000)
94 | // poolConfig.setTimeBetweenEvictionRunsMillis(30000)
95 | // poolConfig.setNumTestsPerEvictionRun(-1)
96 | // if (redisConnection.auth == "null") {
97 | // new JedisPool(poolConfig, redisConnection.host, redisConnection.port,
98 | // redisConnection.timeout, null, redisConnection.dbNum)
99 | // } else {
100 | // new JedisPool(poolConfig, redisConnection.host, redisConnection.port,
101 | // redisConnection.timeout, redisConnection.auth, redisConnection.dbNum)
102 | new JedisPool(poolConfig, redisConnection.host, redisConnection.port,
103 | 30000000, redisConnection.auth, redisConnection.dbNum) // 30000sec
104 | // 300000, redisConnection.auth, redisConnection.dbNum) // 300sec
105 | // }
106 | }
107 | )
108 | var sleepTime: Int = 4
109 | var conn: Jedis = null
110 | while (conn == null) {
111 | try {
112 | conn = pool.getResource
113 | }
114 | catch {
115 | case e: JedisConnectionException if e.getCause.toString.
116 | contains("ERR max number of clients reached") => {
117 | if (sleepTime < 500) sleepTime *= 2
118 | Thread.sleep(sleepTime)
119 | }
120 | case e: Exception => throw e
121 | }
122 | }
123 | conn
124 | }
125 | }
126 |
--------------------------------------------------------------------------------
/src/main/scala/kr/ac/yonsei/delab/addb_srconnector/RedisStore.scala:
--------------------------------------------------------------------------------
1 | package kr.ac.yonsei.delab.addb_srconnector
2 |
3 | import java.util.HashSet
4 | import java.util.ArrayList
5 |
6 | import org.apache.spark.rdd.RDD
7 | import org.apache.spark.sql.Row
8 | import org.apache.spark.sql.sources._
9 | import org.apache.spark.sql.types._
10 |
11 | import scala.collection.JavaConversions._
12 | import scala.collection.JavaConverters._
13 | import scala.collection.mutable.{Stack, ArrayBuffer, ListBuffer}
14 |
15 | import redis.clients.addb_jedis.Protocol
16 | import redis.clients.addb_jedis.util.CommandArgsObject
17 | import redis.clients.addb_jedis.exceptions.JedisClusterException
18 |
19 | import kr.ac.yonsei.delab.addb_srconnector.util.KeyUtil
20 | import kr.ac.yonsei.delab.addb_srconnector.util.Filters
21 | import kr.ac.yonsei.delab.addb_srconnector.util.Logging
22 |
23 |
24 | object ColumnType extends Enumeration {
25 | type ColumnType = Value
26 | val StringType = Value( "String" )
27 | val NumericType = Value( "Numeric" )
28 | }
29 |
30 | /*
31 | * RedisRow class
32 | * represent table, column info associated with row
33 | * table := redis table including this RedisRow
34 | * columns := each columns information (column name -> value)
35 | */
36 | case class RedisRow( val table: RedisTable, val columns: Map[String, String])
37 | extends Serializable { }
38 | /*
39 | * RedisColumn class
40 | * represent each column
41 | * name := column name
42 | * columnType := column type(String | Numeric)
43 | */
44 | case class RedisColumn(val name: String, val columnType: ColumnType.Value ) { }
45 |
46 | /*
47 | * RedisTable class
48 | * include table information
49 | * id := tableID from CREATE TABLE OPTIONS table
50 | * columns := RedisColumn array
51 | */
52 | case class RedisTable (
53 | val id: Int,
54 | val columns: Array[RedisColumn],
55 | val partitionColumnNames: Array[String]) {
56 |
57 | val columnCount = columns.size
58 | val columnNameWithIndex = columns.map(_.name).zip(Stream from 1) // index sorted Array
59 | val columnNameWithID = columns.map(_.name).zip(Stream from 1).toMap // from index 1. not sorted
60 | val partitionColumnID:Array[Int] = partitionColumnNames.map(
61 | columnName => columnNameWithID(columnName)).toArray
62 | }
63 | /*
64 | * For reducing overhead when build redis table, maintain RedisTable list
65 | */
66 | object RedisTableList
67 | extends Logging {
68 | var list = Map[Int, RedisTable]()
69 | def insertTableList (tableID: Int, redisTable:RedisTable) {
70 | list += (tableID -> redisTable)
71 | }
72 |
73 | def checkList(tableID: Int):Boolean = {
74 | if (list.size == 0) false
75 | else if (list.get(tableID) == None) false
76 | else true
77 | }
78 |
79 | def getTableColumnWithIndex(tableID: Int, table:RedisTable):Map[String, Int] = {
80 | var res = list.get(tableID)
81 | if (res == None) {
82 | RedisTableList.insertTableList(tableID, table)
83 | res = list.get(tableID)
84 | throw new NoSuchElementException(s"[ADDB] Fatal error: There is no corresponding RedisTable...")
85 | }
86 | res.get.columnNameWithID
87 | }
88 | }
89 | /*
90 | * RedisStore class
91 | * run actual INSERT(add), SELECT(scan) statement from/to redis
92 | */
93 | class RedisStore (val redisConfig:RedisConfig)
94 | extends Configurable
95 | with Serializable {
96 |
97 | // Retain host's cluster node
98 | val redisCluster: RedisCluster = {
99 | new RedisCluster({
100 | val configuration = this.redisConfig.configuration
101 | val host = configuration.getOrElse("host", Protocol.DEFAULT_HOST)
102 | val port = configuration.getOrElse("port", Protocol.DEFAULT_PORT.toString).toInt
103 | val auth = configuration.getOrElse("auth", null)
104 | val dbNum = configuration.getOrElse("dbNum", Protocol.DEFAULT_DATABASE.toString).toInt
105 | val timeout = configuration.getOrElse("timeout", Protocol.DEFAULT_TIMEOUT.toString).toInt
106 | new RedisConnection(host, port, auth, dbNum, timeout)
107 | })
108 | }
109 |
110 | // Call by getPartitions
111 | def getTablePartitions(table: RedisTable, filter: Array[Filter]) : Array[(String, Array[String])] = {
112 | logDebug( s"[ADDB] : getTablePartitions called")
113 | val metaKey =KeyUtil.generateKeyForMeta(table.id)
114 | logDebug( s"[ADDB] : metaKey: $metaKey" )
115 | val sf = System.currentTimeMillis
116 | // Make filter
117 | var retbuf = new StringBuilder
118 | filter.foreach {
119 | x =>
120 | var stack = new Stack[String]
121 | Filters.makeFilterString(x, stack, table.id, table)
122 | while (!stack.isEmpty) {
123 | retbuf.append(stack.pop())
124 | }
125 | retbuf.append("$")
126 | }
127 | val ef = System.currentTimeMillis
128 | logInfo(s"[ADDB] make filterString ${(ef-sf)/1000.0f}")
129 | logDebug(s"new String for Filter = " + retbuf.toString() +", "+ retbuf.toString.isEmpty)
130 | logInfo(s"new String for Filter = " + retbuf.toString() +", "+ retbuf.toString.isEmpty)
131 |
132 | val sg = System.currentTimeMillis
133 | val ret_scala : ArrayBuffer[String] = ArrayBuffer[String]()
134 | redisCluster.nodes.foreach{
135 | x =>
136 | val conn = x.connect()
137 | conn.metakeys(metaKey, retbuf.toString).foreach (
138 | x => ret_scala += KeyUtil.getPartitionFromMeta(x) )
139 | conn.close()
140 | }
141 |
142 | // Spark partitioning := partition keys with corresponding port
143 | val partitioning = KeyUtil.groupKeysByNode(redisCluster.nodes, KeyUtil.generateDataKey(table.id, ret_scala.toArray))
144 | val eg = System.currentTimeMillis
145 | logInfo(s"[ADDB] metakeys ${(eg-sg)/1000.0f}")
146 | partitioning
147 | }
148 |
149 | /*
150 | * Add data to redis through jedis pipeline
151 | * Process INSERT(fpwrite) command according each node
152 | */
153 | def add(row: RedisRow,
154 | pipelinePool:PipelinePool
155 | /*,
156 | datakey: String,
157 | partitionInfo:String */): Unit = {
158 | //logInfo("[ADDB] add(INSERTION) function")
159 | var partitionInfo = new StringBuilder
160 | // rowForTableInfo.foreach { x => logInfo(s"rowForTableInfo: ${x.columns}") }
161 |
162 | // 1) Generate datakey and partitionInfo
163 | val datakey:String = {
164 | // Generate partition:= (1) (index, name) -> (2) (index, value)
165 | val partitionIndexWithName = row.table.partitionColumnID.zip(row.table.partitionColumnNames)
166 | val partitionIndexWithValue = partitionIndexWithName.map(
167 | column => (column._1, row.columns.get(column._2).get))
168 | val (key, partition) = KeyUtil.generateDataKey(row.table.id, partitionIndexWithValue)
169 | partitionInfo.append(partition)
170 | key
171 | }
172 | // 2) Execute pipelined command in each node
173 | // From SRC := [RedisRelation]-RedisCluster(RedisConnection)
174 | // To ADDB := [RedisStore]-RedisCluster(RedisConnection)
175 | val node = KeyUtil.getNodeForKey(redisCluster.nodes, datakey)
176 |
177 | // 3) Convert from data:String to data:List (compatible with Java List type)
178 | // Because of Map structure, need to sort data
179 | var data = new ListBuffer[String]()
180 | row.table.columns.foreach {
181 | x =>
182 | // convert null to "null" string
183 | data += row.columns.getOrElse(x.name, "null")
184 | }
185 | // 4) Get pipeline and insert fpwrite command into pipeline
186 | // System.out.println("fpwrite "+datakey+" "+row.table.columnCount.toString + " " + partitionInfo.toString + " " + data.mkString(" "))
187 | val commandArgsObject = new CommandArgsObject(datakey, row.table.columnCount.toString,
188 | partitionInfo.toString, data.toList.asJava)
189 | val pipeline = pipelinePool.get(node.redisConnection.host+":"+node.redisConnection.port.toString)
190 | pipeline.fpwrite(commandArgsObject)
191 | }
192 |
193 | def _calculateDurationSec(start: Double, end: Double): Double = {
194 | return (end - start) / 1000.0f;
195 | }
196 |
197 | def scan(
198 | table: RedisTable,
199 | location: String,
200 | datakeys: Array[String], // datakeys including partition key
201 | prunedColumns: Array[String]): Iterator[RedisRow] = {
202 |
203 | val _time_prepare_s = System.currentTimeMillis
204 | logDebug("[ADDB] scan function")
205 | val columnIndex = prunedColumns.map {
206 | columnName => "" + (table.columns.map(_.name).indexOf(columnName) + 1)
207 | }
208 | val host = KeyUtil.returnHost(location)
209 | val port = KeyUtil.returnPort(location)
210 | val nodeIndex = redisCluster.checkNodes(host, port)
211 | val _time_prepare_e = System.currentTimeMillis
212 | logInfo(s"[ADDB] prepare time ${_calculateDurationSec(_time_prepare_e, _time_prepare_s)}")
213 |
214 | val group_size = {
215 | if (datakeys.size >= 10) 10
216 | else 1
217 | }
218 |
219 | val _time_flatmapscan_s = System.currentTimeMillis
220 | val values = datakeys.grouped(group_size).flatMap { datakeyGroup =>
221 | val __time_connection_s = System.currentTimeMillis
222 | val conn = redisCluster.nodes(nodeIndex).redisConnection.connect
223 | val pipeline = conn.pipelined()
224 | val __time_connection_e = System.currentTimeMillis
225 | logInfo(s"[ADDB] connection time ${_calculateDurationSec(__time_connection_s, __time_connection_e)}")
226 |
227 | val __time_execution_s = System.currentTimeMillis
228 | datakeyGroup.foreach { dataKey =>
229 | val commandArgsObject = new CommandArgsObject(dataKey,
230 | KeyUtil.retRequiredColumnIndice(table.id, table, prunedColumns))
231 | pipeline.fpscan(commandArgsObject)
232 | }
233 | val __time_execution_e = System.currentTimeMillis
234 | logInfo(s"[ADDB] scan execution ${_calculateDurationSec(__time_execution_s, __time_execution_e)}")
235 |
236 | val __time_pipsync_s = System.currentTimeMillis
237 | // TODO(totoro): Implements syncAndReturnAll to Future API.
238 | val results = pipeline.syncAndReturnAll.flatMap { x =>
239 | logDebug(s"[ADDB] values getClass: ${x.getClass.toString()}")
240 | // If errors occur, casting exception is called
241 | try {
242 | /* For getting String data, transform original(List[Object]) data
243 | List[Object] -> List[ArrayList[String]] -> Buffer[ArrayList[String]] -> Append each String */
244 | x.asInstanceOf[ArrayList[String]]
245 | } catch {
246 | case e: java.lang.ClassCastException => {
247 | logError(s"[ADDB] Scan Error: ${x.asInstanceOf[JedisClusterException]}")
248 | throw e
249 | }
250 | }
251 | }
252 | val __time_pipsync_e = System.currentTimeMillis
253 | logInfo(s"[ADDB] pip sync ${_calculateDurationSec(__time_pipsync_s, __time_pipsync_e)}")
254 | conn.close()
255 | results
256 | }
257 | .toArray
258 |
259 | val _time_flatmapscan_e = System.currentTimeMillis
260 | logInfo(s"[ADDb] flatmap scan ${_calculateDurationSec(_time_flatmapscan_s, _time_flatmapscan_e)}")
261 |
262 | val _time_remainjob_s = System.currentTimeMillis
263 | val result = {
264 | if (prunedColumns.length != 0) {
265 | values.grouped(prunedColumns.length).map { x =>
266 | val columns: Map[String, String] = prunedColumns.zip(x).toMap
267 | new RedisRow(table, columns)
268 | }
269 | }
270 | else {
271 | values.map { x =>
272 | val columns: Map[String, String] = Map(x->x)
273 | new RedisRow(table, columns)
274 | }.toIterator
275 | }
276 | }
277 | val _time_remainjob_e = System.currentTimeMillis
278 | logInfo(s"[ADDB] remain job ${_calculateDurationSec(_time_remainjob_s, _time_remainjob_e)}")
279 |
280 | result
281 | }
282 |
283 | def add(row: RedisRow): Unit = {
284 | throw new RuntimeException(s"Unsupported method on this mode")
285 | }
286 | def get(key: String): Iterator[RedisRow] = {
287 | throw new RuntimeException(s"Unsupported method on this mode")
288 | }
289 | def getByRanges(
290 | table: String,
291 | key: String,
292 | ranges: Array[Range]
293 | ): Iterator[RedisRow] = {
294 | throw new RuntimeException(s"Unsupported method on this mode")
295 | }
296 | def remove(row: RedisRow): Unit = {
297 | throw new RuntimeException(s"Unsupported method on this mode")
298 | }
299 | }
300 |
--------------------------------------------------------------------------------
/src/main/scala/kr/ac/yonsei/delab/addb_srconnector/partition/RedisPartition.scala:
--------------------------------------------------------------------------------
1 | package kr.ac.yonsei.delab.addb_srconnector.partition
2 | import org.apache.spark.Partition
3 | import kr.ac.yonsei.delab.addb_srconnector.RedisConfig
4 | import kr.ac.yonsei.delab.addb_srconnector._
5 |
6 | class RedisPartition(
7 | override val index: Int,
8 | val redisConfig: RedisConfig,
9 | val location: String,
10 | val partition: Array[String]
11 | )
12 | extends Partition {
13 | }
14 |
--------------------------------------------------------------------------------
/src/main/scala/kr/ac/yonsei/delab/addb_srconnector/rdd/ADDBRDD.scala:
--------------------------------------------------------------------------------
1 | package kr.ac.yonsei.delab.addb_srconnector.rdd
2 |
3 | import org.apache.spark.rdd.RDD
4 | import org.apache.spark.TaskContext
5 | import org.apache.spark.SparkContext
6 | import org.apache.spark.Partition
7 | import java.math.BigDecimal
8 | import scala.reflect.ClassTag
9 | import scala.collection.JavaConversions._
10 | import scala.collection.mutable.ArrayBuffer
11 | import kr.ac.yonsei.delab.addb_srconnector._
12 | import kr.ac.yonsei.delab.addb_srconnector.partition._
13 | import kr.ac.yonsei.delab.addb_srconnector.util
14 | import org.apache.spark.sql.{DataFrame, SQLContext, Row}
15 | import org.apache.spark.sql.sources._
16 | import org.apache.spark.sql.types._
17 | import java.text.{SimpleDateFormat,NumberFormat}
18 | import java.util.Locale
19 | import java.sql.Date
20 | import scala.util.Try
21 |
22 | class ADDBRDD (
23 | @transient val sc: SparkContext,
24 | val redisConfig: RedisConfig,
25 | val redisTable: RedisTable,
26 | val requiredColumns: Array[String],
27 | val filter: Array[Filter]
28 | ) extends RDD[RedisRow] (sc, Seq.empty)
29 | {
30 |
31 | override protected def getPreferredLocations(split: Partition): Seq[String] = {
32 | // logInfo( s"[WONKI] : getPreferredLocations called ${split.asInstanceOf[RedisPartition].location}")
33 | Seq(split.asInstanceOf[RedisPartition].location)
34 | }
35 |
36 | override protected def getPartitions: Array[Partition] = {
37 | logDebug( s"[ADDB] getPartitions called")
38 | val redisStore = redisConfig.getRedisStore()
39 | val sourceinfos = redisStore.getTablePartitions(redisTable, filter) // get partition key
40 | var i = 0
41 | sourceinfos.map { mem =>
42 | val loc = mem._1
43 | logDebug( s"[ADDB] : getPartitions mem 1 : ${mem._1}")
44 | val sources : Array[String] = mem._2
45 | val size = mem._2.size
46 |
47 | var res = new ArrayBuffer[Partition]
48 | logDebug(s"[ADDB] Total size: ${mem._2.size}")
49 | var partitioningSize = {
50 | if (size>=3) size/3
51 | else 1
52 | }
53 | mem._2.grouped(partitioningSize).foreach {
54 | x =>
55 | res += new RedisPartition(i, redisConfig, loc, x)
56 | i+= 1
57 | }
58 | // sources.foreach { x => logInfo(s"RedisPartition-Partition : $x") }
59 | // logInfo( s"[WONKI] : getPartitions mem 2 : ${mem._2}")
60 | // val partition = new RedisPartition(i, redisConfig, loc, sources);
61 | // i += 1
62 | // partition
63 | res.toArray
64 | }.flatten.toArray // (RedisPartition1 , RedisPartition2, RedisPartition3)
65 | // TO DO, Need to balance (partition-node)
66 | }
67 |
68 | // Each RedisPartition from getPartitions is adapted to compute()
69 | // Thus, scan is called by each RedisPartitions
70 | override def compute(split: Partition, context: TaskContext) : Iterator[RedisRow] = {
71 | // logInfo( s"[WONKI] : compute called")
72 | val partition = split.asInstanceOf[RedisPartition]
73 | // logInfo( s"[WONKI] : partition : ${partition.index}")
74 | val redisStore = redisConfig.getRedisStore()
75 | redisStore.scan(redisTable, partition.location, partition.partition, requiredColumns)
76 | }
77 | }
78 |
79 | // Convert RDD[RedisRow] to RDD[Row] (DataFrame)
80 | class RedisRDDAdaptor(
81 | val prev: RDD[RedisRow],
82 | val requiredColumns: Array[StructField],
83 | val filters: Array[Filter],
84 | val schema: org.apache.spark.sql.types.StructType
85 | ) extends RDD[Row]( prev ) {
86 |
87 | def castToTarget(value: String, dataType: DataType) = {
88 | dataType match {
89 | case _: ByteType => value.toByte
90 | case _: ShortType => value.toShort
91 | case _: IntegerType => value.toInt
92 | case _: LongType => value.toLong
93 | case _: FloatType => Try(value.toFloat)
94 | .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(value).floatValue())
95 | case _: DoubleType => Try(value.toDouble)
96 | .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(value).doubleValue())
97 | case _: BooleanType => value.toBoolean
98 | case _: DecimalType => new BigDecimal(value.replaceAll(",", ""))
99 | case _: StringType => value
100 | case _: DateType => Date.valueOf(value)
101 | case _ => throw new RuntimeException(s"Unsupported type")
102 | }
103 | }
104 |
105 | override def getPartitions: Array[Partition] = prev.partitions
106 |
107 | override def compute(split: Partition, context: TaskContext): Iterator[Row] = {
108 | prev.compute(split, context).map { // call ADDBRDD.compute
109 | redisRow =>
110 | val columns: Array[Any] = requiredColumns.map { column =>
111 | val value = redisRow.columns.getOrElse(column.name, null)
112 | castToTarget(value, column.dataType)
113 | }
114 | val row = Row.fromSeq(columns.toSeq)
115 | row
116 | }
117 | }
118 | }
119 |
120 |
--------------------------------------------------------------------------------
/src/main/scala/kr/ac/yonsei/delab/addb_srconnector/util/Filters.scala:
--------------------------------------------------------------------------------
1 | package kr.ac.yonsei.delab.addb_srconnector.util
2 |
3 | import scala.collection.mutable.Stack
4 | import org.apache.spark.sql.sources._
5 | import kr.ac.yonsei.delab.addb_srconnector.{RedisTableList, RedisTable}
6 |
7 | object Filters {
8 | def makeFilterString(f: Filter, stack: Stack[String], tableID: Int, table:RedisTable) : Unit = {
9 | // Since "column name" should be converted into "column index"
10 | // get Table's Column name with index from RedisTableList object
11 | var columnNameWithIndex = RedisTableList.getTableColumnWithIndex(tableID, table)
12 |
13 | f match {
14 | case Or(_,_) => {
15 | stack.push("Or:")
16 | makeFilterString(f.asInstanceOf[Or].left, stack, tableID, table)
17 | makeFilterString(f.asInstanceOf[Or].right, stack, tableID, table)
18 | }
19 | case And(_,_) => {
20 | stack.push("And:")
21 | makeFilterString(f.asInstanceOf[And].left, stack, tableID, table)
22 | makeFilterString(f.asInstanceOf[And].right, stack, tableID, table)
23 | }
24 | case Not(_) => {
25 | stack.push("Not:")
26 | makeFilterString(f.asInstanceOf[Not].child, stack, tableID, table)
27 | }
28 | case EqualTo(_,_) => {
29 | stack.push("EqualTo:")
30 | stack.push(columnNameWithIndex.get(f.asInstanceOf[EqualTo].attribute).get + "*")
31 | stack.push(f.asInstanceOf[EqualTo].value.toString() + "*")
32 | }
33 |
34 | case GreaterThan(_, _) => {
35 | stack.push("GreaterThan:")
36 | stack.push(columnNameWithIndex.get(f.asInstanceOf[GreaterThan].attribute).get + "*")
37 | stack.push(f.asInstanceOf[GreaterThan].value.toString() + "*")
38 | }
39 |
40 | case GreaterThanOrEqual(_, _) => {
41 | stack.push("GreaterThanOrEqual:")
42 | stack.push(columnNameWithIndex.get(f.asInstanceOf[GreaterThanOrEqual].attribute).get + "*")
43 | stack.push(f.asInstanceOf[GreaterThanOrEqual].value.toString() + "*")
44 | }
45 | case LessThan(_, _) => {
46 | stack.push("LessThan:")
47 | stack.push(columnNameWithIndex.get(f.asInstanceOf[LessThan].attribute).get + "*")
48 | stack.push(f.asInstanceOf[LessThan].value.toString() + "*")
49 | }
50 | case LessThanOrEqual(_, _) => {
51 | stack.push("LessThanOrEqual:")
52 | stack.push(columnNameWithIndex.get(f.asInstanceOf[LessThanOrEqual].attribute).get + "*")
53 | stack.push(f.asInstanceOf[LessThanOrEqual].value.toString() + "*")
54 | }
55 | case In(_, _) => {
56 | /** Transform set of EqualTo **/
57 | //stack.push("In:")
58 | var i = 0
59 | val col = columnNameWithIndex.get(f.asInstanceOf[In].attribute).get
60 | val arrLen = f.asInstanceOf[In].values.length
61 | for (i <- 0 until arrLen - 1) {
62 | stack.push("Or:")
63 | }
64 | f.asInstanceOf[In].values.foreach{
65 | x =>
66 | stack.push("EqualTo:")
67 | stack.push(col + "*")
68 | stack.push(x.toString() + "*")
69 | }
70 | }
71 | case IsNull(_)=> {
72 | stack.push("IsNull:")
73 | stack.push(columnNameWithIndex.get(f.asInstanceOf[IsNull].attribute).get + "*")
74 | }
75 | case IsNotNull(_) => {
76 | stack.push("IsNotNull:")
77 | stack.push(columnNameWithIndex.get(f.asInstanceOf[IsNotNull].attribute).get + "*")
78 | }
79 | case StringStartsWith(_, _) => {
80 | stack.push("StringStartsWith:")
81 | stack.push(columnNameWithIndex.get(f.asInstanceOf[StringStartsWith].attribute).get + "*")
82 | stack.push(f.asInstanceOf[StringStartsWith].value + "*")
83 | }
84 | case StringEndsWith(_, _) => {
85 | stack.push("StringEndsWith:")
86 | stack.push(columnNameWithIndex.get(f.asInstanceOf[StringEndsWith].attribute).get + "*")
87 | stack.push(f.asInstanceOf[StringEndsWith].value + "*")
88 | }
89 | case StringContains(_, _) => {
90 | stack.push("StringContains:")
91 | stack.push(columnNameWithIndex.get(f.asInstanceOf[StringContains].attribute).get + "*")
92 | stack.push(f.asInstanceOf[StringContains].value + "*")
93 | }
94 | }
95 | }
96 | }
97 |
--------------------------------------------------------------------------------
/src/main/scala/kr/ac/yonsei/delab/addb_srconnector/util/KeyUtil.scala:
--------------------------------------------------------------------------------
1 | package kr.ac.yonsei.delab.addb_srconnector.util
2 |
3 | import scala.collection.mutable.{StringBuilder, ArrayBuffer}
4 | import redis.clients.addb_jedis.util.JedisClusterCRC16
5 | import kr.ac.yonsei.delab.addb_srconnector.{RedisNode, RedisTable, RedisTableList}
6 | import org.apache.spark.sql.sources._
7 | import scala.collection.mutable.Stack
8 |
9 | /*
10 | * generate full datakey := "D:{TableInfo:PartitionInfo}"
11 | */
12 | object KeyUtil {
13 | def returnHost(SourceString : String):String = {
14 | var buf:StringBuilder = new StringBuilder
15 | buf.append (SourceString.substring(0, SourceString.indexOf(":")))
16 | buf.toString()
17 | }
18 |
19 | def returnPort(SourceString : String): Int = {
20 | var buf:StringBuilder = new StringBuilder
21 | buf.append (SourceString.substring(SourceString.indexOf(":") + 1, SourceString.size))
22 | buf.toString().toInt
23 | }
24 |
25 | def generateKeyForMeta(tableID:Int):String = {
26 | var buf:StringBuilder = new StringBuilder
27 | buf.append("M:{").append(tableID+":").append("*").append("}")
28 | buf.toString()
29 | }
30 |
31 | def getPartitionFromMeta(MetaKey:String):String = {
32 | var buf:StringBuilder = new StringBuilder
33 | buf.append(MetaKey.substring(MetaKey.indexOf(":", MetaKey.indexOf("{")) + 1, MetaKey.indexOf("}")))
34 | buf.toString()
35 | }
36 |
37 | def generateDataKey(tableID: Int, partitions: Array[String]): Array[String] = {
38 | var buf: StringBuilder = null
39 | val res : ArrayBuffer[String] = ArrayBuffer[String]()
40 | partitions.foreach { partition =>
41 | buf = new StringBuilder
42 | buf.append("D:{").append(tableID + ":").append(partition).append("}")
43 | res += buf.toString
44 | }
45 | res.toArray
46 | }
47 |
48 | // return datakey and partitionInfo
49 | def generateDataKey(tableID:Int, partitionColumnInfo:Array[(Int, String)]):(String, String) = {
50 | var buf:StringBuilder = new StringBuilder
51 | var partition = ""
52 | // tableInfo
53 | buf.append("D:{").append(tableID+":")
54 | // partitionInfo
55 | if (partitionColumnInfo != null && partitionColumnInfo.size > 0) {
56 | // make 1:3142 : 2:4124
57 | // start partitionID from 1.
58 | partition = partitionColumnInfo.map(column => ((column._1)+":"+column._2).toString())
59 | .mkString(":")
60 | buf.append(partition)
61 | }
62 | buf.append("}")
63 | (buf.toString, partition)
64 | }
65 | /**
66 | * @param nodes list of RedisNode
67 | * @param keys list of keys
68 | * return (node(host+port): (datakey1, datakey2, ...), node2(host+port): (datakey3, datakey4,...), ...)
69 | */
70 | def groupKeysByNode(nodes: Array[RedisNode], keys: Array[String]): // keys: DataKey
71 | Array[(String, Array[String])] = {
72 | def getNode(key: String): RedisNode = { // get RedisNode applying datakey
73 | val slot = JedisClusterCRC16.getSlot(key)
74 | /* Master only */
75 | nodes.filter(node => { node.startSlot <= slot && node.endSlot >= slot }).filter(_.idx == 0)(0)
76 | }
77 | def makeSourceString(host:String, port: Int):String = {
78 | var buf:StringBuilder = new StringBuilder
79 | buf.append(host).append(":").append(port.toString())
80 | buf.toString()
81 | }
82 | keys.map(key => (getNode(key), key)).toArray.groupBy(_._1).
83 | map{x => (makeSourceString(x._1.redisConnection.host, x._1.redisConnection.port), x._2.map(_._2)) // (host+port, datakey:Array[String])
84 | }.toArray
85 | }
86 |
87 | def getNodeForKey(nodes: Array[RedisNode], key: String): RedisNode = { // key를 입력하고, 그 key가 어떤 node에 속하는지 찾아내어주는 함수
88 | val slot = JedisClusterCRC16.getSlot(key)
89 | /* Master only */
90 | nodes.filter(node => { node.startSlot <= slot && node.endSlot >= slot }).filter(_.idx == 0)(0)
91 | }
92 |
93 | // Make required column indice (fpscan parameter2)
94 | def makeRequiredColumnIndice (tableID:Int, table:RedisTable, prunedColumns:Array[String]):String = {
95 | val columnNameWithIndex = RedisTableList.getTableColumnWithIndex(tableID, table)
96 | val buf : ArrayBuffer[Int] = ArrayBuffer[Int]()
97 | prunedColumns.foreach { column =>
98 | buf += columnNameWithIndex(column)
99 | }
100 | if (buf.size == 0) {
101 | // get only first column
102 | "1"
103 | } else {
104 | buf.toArray.mkString(",")
105 | }
106 | }
107 | def retRequiredColumnIndice (tableID:Int, table:RedisTable, prunedColumns:Array[String]):String = {
108 | val columnNameWithIndex = table.columnNameWithID
109 | val buf : ArrayBuffer[Int] = ArrayBuffer[Int]()
110 | prunedColumns.foreach { column =>
111 | buf += columnNameWithIndex(column)
112 | }
113 | if (buf.size == 0) {
114 | // get only first column
115 | "1"
116 | } else {
117 | buf.toArray.mkString(",")
118 | }
119 | }
120 | }
--------------------------------------------------------------------------------
/src/main/scala/kr/ac/yonsei/delab/addb_srconnector/util/Logging.scala:
--------------------------------------------------------------------------------
1 | package kr.ac.yonsei.delab.addb_srconnector.util
2 |
3 | import org.apache.log4j.{Level, LogManager, PropertyConfigurator}
4 | import org.slf4j.{Logger, LoggerFactory}
5 | import org.slf4j.impl.StaticLoggerBinder
6 |
7 | import org.apache.spark.util.Utils
8 |
9 | /**
10 | * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows
11 | * logging messages at different levels using methods that only evaluate parameters lazily if the
12 | * log level is enabled.
13 | */
14 | trait Logging {
15 |
16 | // Make the log field transient so that objects with Logging can
17 | // be serialized and used on another machine
18 | @transient private var log_ : Logger = null
19 |
20 | // Method to get the logger name for this object
21 | protected def logName = {
22 | // Ignore trailing $'s in the class names for Scala objects
23 | this.getClass.getName.stripSuffix("$")
24 | }
25 |
26 | // Method to get or create the logger for this object
27 | protected def log: Logger = {
28 | if (log_ == null) {
29 | initializeLogIfNecessary(false)
30 | log_ = LoggerFactory.getLogger(logName)
31 | }
32 | log_
33 | }
34 |
35 | // Log methods that take only a String
36 | protected def logInfo(msg: => String) {
37 | if (log.isInfoEnabled) log.info(msg)
38 | }
39 |
40 | protected def logDebug(msg: => String) {
41 | if (log.isDebugEnabled) log.debug(msg)
42 | }
43 |
44 | protected def logTrace(msg: => String) {
45 | if (log.isTraceEnabled) log.trace(msg)
46 | }
47 |
48 | protected def logWarning(msg: => String) {
49 | if (log.isWarnEnabled) log.warn(msg)
50 | }
51 |
52 | protected def logError(msg: => String) {
53 | if (log.isErrorEnabled) log.error(msg)
54 | }
55 |
56 | // Log methods that take Throwables (Exceptions/Errors) too
57 | protected def logInfo(msg: => String, throwable: Throwable) {
58 | if (log.isInfoEnabled) log.info(msg, throwable)
59 | }
60 |
61 | protected def logDebug(msg: => String, throwable: Throwable) {
62 | if (log.isDebugEnabled) log.debug(msg, throwable)
63 | }
64 |
65 | protected def logTrace(msg: => String, throwable: Throwable) {
66 | if (log.isTraceEnabled) log.trace(msg, throwable)
67 | }
68 |
69 | protected def logWarning(msg: => String, throwable: Throwable) {
70 | if (log.isWarnEnabled) log.warn(msg, throwable)
71 | }
72 |
73 | protected def logError(msg: => String, throwable: Throwable) {
74 | if (log.isErrorEnabled) log.error(msg, throwable)
75 | }
76 |
77 | protected def isTraceEnabled(): Boolean = {
78 | log.isTraceEnabled
79 | }
80 |
81 | protected def initializeLogIfNecessary(isInterpreter: Boolean): Unit = {
82 | initializeLogIfNecessary(isInterpreter, silent = false)
83 | }
84 |
85 | protected def initializeLogIfNecessary(
86 | isInterpreter: Boolean,
87 | silent: Boolean = false): Boolean = {
88 | if (!Logging.initialized) {
89 | Logging.initLock.synchronized {
90 | if (!Logging.initialized) {
91 | initializeLogging(isInterpreter, silent)
92 | return true
93 | }
94 | }
95 | }
96 | false
97 | }
98 |
99 | private def initializeLogging(isInterpreter: Boolean, silent: Boolean): Unit = {
100 | // Don't use a logger in here, as this is itself occurring during initialization of a logger
101 | // If Log4j 1.2 is being used, but is not initialized, load a default properties file
102 | if (Logging.isLog4j12()) {
103 | val log4j12Initialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements
104 | // scalastyle:off println
105 | if (!log4j12Initialized) {
106 | Logging.defaultSparkLog4jConfig = true
107 | val defaultLogProps = "org/apache/spark/log4j-defaults.properties"
108 | Option(getClass.getClassLoader.getResource(defaultLogProps)) match {
109 | case Some(url) =>
110 | PropertyConfigurator.configure(url)
111 | if (!silent) {
112 | System.err.println(s"Using Spark's default log4j profile: $defaultLogProps")
113 | }
114 | case None =>
115 | System.err.println(s"Spark was unable to load $defaultLogProps")
116 | }
117 | }
118 |
119 | val rootLogger = LogManager.getRootLogger()
120 | if (Logging.defaultRootLevel == null) {
121 | Logging.defaultRootLevel = rootLogger.getLevel()
122 | }
123 |
124 | if (isInterpreter) {
125 | // Use the repl's main class to define the default log level when running the shell,
126 | // overriding the root logger's config if they're different.
127 | val replLogger = LogManager.getLogger(logName)
128 | val replLevel = Option(replLogger.getLevel()).getOrElse(Level.WARN)
129 | if (replLevel != rootLogger.getEffectiveLevel()) {
130 | if (!silent) {
131 | System.err.printf("Setting default log level to \"%s\".\n", replLevel)
132 | System.err.println("To adjust logging level use sc.setLogLevel(newLevel). " +
133 | "For SparkR, use setLogLevel(newLevel).")
134 | }
135 | rootLogger.setLevel(replLevel)
136 | }
137 | }
138 | // scalastyle:on println
139 | }
140 | Logging.initialized = true
141 |
142 | // Force a call into slf4j to initialize it. Avoids this happening from multiple threads
143 | // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html
144 | log
145 | }
146 | }
147 |
148 | object Logging {
149 | @volatile private var initialized = false
150 | @volatile private var defaultRootLevel: Level = null
151 | @volatile private var defaultSparkLog4jConfig = false
152 |
153 | val initLock = new Object()
154 | try {
155 | // We use reflection here to handle the case where users remove the
156 | // slf4j-to-jul bridge order to route their logs to JUL.
157 | val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler")
158 | bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null)
159 | val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean]
160 | if (!installed) {
161 | bridgeClass.getMethod("install").invoke(null)
162 | }
163 | } catch {
164 | case e: ClassNotFoundException => // can't log anything yet so just fail silently
165 | }
166 |
167 | /**
168 | * Marks the logging system as not initialized. This does a best effort at resetting the
169 | * logging system to its initial state so that the next class to use logging triggers
170 | * initialization again.
171 | */
172 | def uninitialize(): Unit = initLock.synchronized {
173 | if (isLog4j12()) {
174 | if (defaultSparkLog4jConfig) {
175 | defaultSparkLog4jConfig = false
176 | LogManager.resetConfiguration()
177 | } else {
178 | LogManager.getRootLogger().setLevel(defaultRootLevel)
179 | }
180 | }
181 | this.initialized = false
182 | }
183 |
184 | private def isLog4j12(): Boolean = {
185 | // This distinguishes the log4j 1.2 binding, currently
186 | // org.slf4j.impl.Log4jLoggerFactory, from the log4j 2.0 binding, currently
187 | // org.apache.logging.slf4j.Log4jLoggerFactory
188 | val binderClass = StaticLoggerBinder.getSingleton.getLoggerFactoryClassStr
189 | "org.slf4j.impl.Log4jLoggerFactory".equals(binderClass)
190 | }
191 | }
--------------------------------------------------------------------------------