├── .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 | } --------------------------------------------------------------------------------