From eee70dc6dac8aa1b3b9387aaa136250d340f2b46 Mon Sep 17 00:00:00 2001 From: Sourav Mazumder Date: Tue, 15 Nov 2016 15:34:14 -0800 Subject: [PATCH 01/24] [BAHIR-75] Initital code delivery for WebHDFS data source --- datasource-webhdfs/README.md | 42 ++ datasource-webhdfs/pom.xml | 89 ++++ ...pache.spark.sql.sources.DataSourceRegister | 1 + .../datasource/webhdfs/DefaultSource.scala | 265 ++++++++++ .../webhdfs/csv/WebHdfsCsvFormatter.scala | 150 ++++++ .../webhdfs/csv/WebHdfsCsvParserLibs.scala | 51 ++ .../webhdfs/csv/WebHdfsCsvRelation.scala | 294 +++++++++++ .../bahir/datasource/webhdfs/package.scala | 24 + .../webhdfs/util/SSLTrustStoreUtil.scala | 76 +++ .../webhdfs/util/WebHdfsConnector.scala | 471 ++++++++++++++++++ .../webhdfs/util/WebHdfsInferSchema.scala | 194 ++++++++ .../webhdfs/util/WebHdfsParseModes.scala | 48 ++ .../webhdfs/util/WebHdfsTypeCast.scala | 113 +++++ pom.xml | 1 + 14 files changed, 1819 insertions(+) create mode 100644 datasource-webhdfs/README.md create mode 100755 datasource-webhdfs/pom.xml create mode 100644 datasource-webhdfs/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister create mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/DefaultSource.scala create mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvFormatter.scala create mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvParserLibs.scala create mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvRelation.scala create mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/package.scala create mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala create mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala create mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsInferSchema.scala create mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsParseModes.scala create mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsTypeCast.scala diff --git a/datasource-webhdfs/README.md b/datasource-webhdfs/README.md new file mode 100644 index 00000000..18b49750 --- /dev/null +++ b/datasource-webhdfs/README.md @@ -0,0 +1,42 @@ +A custom datasource for reading data from remote hdfs using webhdfs protocol. + +## Linking + +Using SBT: + + libraryDependencies += "org.apache.bahir" %% "spark-datasource-webhdfs" % "2.1.0-SNAPSHOT" + +Using Maven: + + + org.apache.bahir + spark-datasource-webhdfs_2.11 + 2.1.0-SNAPSHOT + + +This library can also be added to Spark jobs launched through `spark-shell` or `spark-submit` by using the `--packages` command line option. +For example, to include it when starting the spark shell: + + $ bin/spark-shell --packages org.apache.bahir:spark-datasource-webhdfs_2.11:2.1.0-SNAPSHOT + +Unlike using `--jars`, using `--packages` ensures that this library and its dependencies will be added to the classpath. +The `--packages` argument can also be used with `bin/spark-submit`. + +This library is compiled for Scala 2.11 only, and intends to support Spark 2.0 onwards. + +## Examples + +A data frame can be created using this custom data source as shown below - + + val filePath = s"webhdfs:///gateway/default/webhdfs/v1/" + + val df = spark.read.format("webhdfs").option("certValidation", "Y").option("userCred", "user1:pass1").option("header", "true").option("partitions", "8").load(filePath) + + +## Configuration options. + + * `certValidation` Set this to 'Y' or 'N'. In case of 'N' this component will ignore validation of teh SSL certification. Otherwise it will dowload the certificate and validate. + * `userCred` Set this to 'userid:password' as needed by the remote hdfs for accessing a file from there. + * `partitions` This number tells the Data Source how many parallel connections to be opened to read data from HDFS in the remote cluster for each file. If this option is not specified default value is used which is 4. Recommended value for this option is same as the next nearest integer of (file size/block size) in hdfs or multiple of that. For example if file size in hdfs is 0.95 GB and block size of the file is 128 MB use 8 or multiple of 8 as number of partitions. However, number of partitions should not be more than (or may be little more than) maximum number of parallel tasks possible to spawn in your spark cluster. + * `format` Format of the file. Right now only 'csv' is supported. If this option is not specified by default 'csv' is assumed. + * `output` Specify either 'LIST' or 'Data'. By default, 'Data' is assumed which returns the actual data in the file. If a folder name is specified then data from all files in that folder would be fethed at once. If 'LIST' is specified then the files within the folder is listed. diff --git a/datasource-webhdfs/pom.xml b/datasource-webhdfs/pom.xml new file mode 100755 index 00000000..abb094f7 --- /dev/null +++ b/datasource-webhdfs/pom.xml @@ -0,0 +1,89 @@ + + + + + 4.0.0 + + org.apache.bahir + bahir-parent_2.11 + 2.1.0-SNAPSHOT + ../pom.xml + + + org.apache.bahir + datasource-webhdfs_2.11 + + datasource-webhdfs + + jar + Apache Bahir - Spark DataSource for WebHdfs + http://bahir.apache.org/ + + + + org.scalaj + scalaj-http_2.11 + 2.3.0 + + + org.apache.commons + commons-csv + 1.3 + + + org.apache.spark + spark-tags_${scala.binary.version} + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + + + org.apache.spark + spark-hive_${scala.binary.version} + ${spark.version} + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + + org.apache.maven.plugins + maven-source-plugin + + + org.apache.rat + apache-rat-plugin + 0.11 + + + **/org.apache.spark.sql.sources.DataSourceRegister + + + + + + diff --git a/datasource-webhdfs/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/datasource-webhdfs/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister new file mode 100644 index 00000000..2108bc67 --- /dev/null +++ b/datasource-webhdfs/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -0,0 +1 @@ +org.apache.bahir.datasource.webhdfs.DefaultSource diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/DefaultSource.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/DefaultSource.scala new file mode 100644 index 00000000..db14a78e --- /dev/null +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/DefaultSource.scala @@ -0,0 +1,265 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.bahir.datasource.webhdfs + +import java.text.SimpleDateFormat +import java.sql.{Timestamp, Date} + +import scala.collection.mutable.HashMap + +import org.apache.spark.sql.types.{DateType, TimestampType} + +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext} + +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types.StructType +import scala.annotation.switch + +import org.apache.bahir.datasource.webhdfs.util._ +import org.apache.bahir.datasource.webhdfs.csv._ + +/** + * This class contains functions for reading/writing data from/to remote webhdfs server in Spark DataSource + * This function is written in line with the DataSource function in com.databricks.spark.csv. +*/ + + +class DefaultSource + extends RelationProvider + with SchemaRelationProvider + with CreatableRelationProvider + with DataSourceRegister { + + override def shortName() : String = "webhdfs" + + private def checkPath(parameters: Map[String, String]): String = { + parameters.getOrElse("path", sys.error("'path' must be specified ")) + } + + /** + * Creates a new relation for data store in CSV given parameters. + * Parameters have to include 'path' and optionally 'delimiter', 'quote', and 'header' + */ + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + createRelation(sqlContext, parameters, null) + } + + /** + * Creates a new relation for data store in CSV given parameters and user supported schema. + * Parameters have to include 'path' and optionally 'delimiter', 'quote', and 'header' + */ + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String], + schema: StructType): WebHdfsCsvRelation = { + //print("In Create Relation of DefaultSource" + "\n") + val path = checkPath(parameters) + val delimiter = WebHdfsTypeCast.toChar(parameters.getOrElse("delimiter", ",")) + + val quote = parameters.getOrElse("quote", "\"") + val quoteChar: Character = if (quote == null) { + null + } else if (quote.length == 1) { + quote.charAt(0) + } else { + throw new Exception("Quotation cannot be more than one character.") + } + + val escape = parameters.getOrElse("escape", null) + val escapeChar: Character = if (escape == null) { + null + } else if (escape.length == 1) { + escape.charAt(0) + } else { + throw new Exception("Escape character cannot be more than one character.") + } + + val comment = parameters.getOrElse("comment", "#") + val commentChar: Character = if (comment == null) { + null + } else if (comment.length == 1) { + comment.charAt(0) + } else { + throw new Exception("Comment marker cannot be more than one character.") + } + + val parseMode = parameters.getOrElse("mode", "PERMISSIVE") + + val useHeader = parameters.getOrElse("header", "false") + val headerFlag = if (useHeader == "true") { + true + } else if (useHeader == "false") { + false + } else { + throw new Exception("Header flag can be true or false") + } + + // val parserLib = parameters.getOrElse("parserLib", ParserLibs.DEFAULT) + val parserLib = parameters.getOrElse("parserLib", "COMMONS") + val ignoreLeadingWhiteSpace = parameters.getOrElse("ignoreLeadingWhiteSpace", "false") + val ignoreLeadingWhiteSpaceFlag = if (ignoreLeadingWhiteSpace == "false") { + false + } else if (ignoreLeadingWhiteSpace == "true") { + //if (!ParserLibs.isUnivocityLib(parserLib)) { + // throw new Exception("Ignore white space supported for Univocity parser only") + //} + true + } else { + throw new Exception("Ignore white space flag can be true or false") + } + + val ignoreTrailingWhiteSpace = parameters.getOrElse("ignoreTrailingWhiteSpace", "false") + val ignoreTrailingWhiteSpaceFlag = if (ignoreTrailingWhiteSpace == "false") { + false + } else if (ignoreTrailingWhiteSpace == "true") { + //if (!ParserLibs.isUnivocityLib(parserLib)) { + // throw new Exception("Ignore white space supported for the Univocity parser only") + //} + true + } else { + throw new Exception("Ignore white space flag can be true or false") + } + val treatEmptyValuesAsNulls = parameters.getOrElse("treatEmptyValuesAsNulls", "false") + val treatEmptyValuesAsNullsFlag = if (treatEmptyValuesAsNulls == "false") { + false + } else if (treatEmptyValuesAsNulls == "true") { + true + } else { + throw new Exception("Treat empty values as null flag can be true or false") + } + + val charset = parameters.getOrElse("charset", WebHdfsConnector.DEFAULT_CHARSET.name()) + // TODO validate charset? + + val inferSchema = parameters.getOrElse("inferSchema", "false") + val inferSchemaFlag = if (inferSchema == "false") { + false + } else if (inferSchema == "true") { + true + } else { + throw new Exception("Infer schema flag can be true or false") + } + val nullValue = parameters.getOrElse("nullValue", "") + + val dateFormat = parameters.getOrElse("dateFormat", null) + + val codec = parameters.getOrElse("codec", null) + + val maxCharsPerColStr = parameters.getOrElse("maxCharsPerCol", "100000") + val maxCharsPerCol = try { + maxCharsPerColStr.toInt + } catch { + case e: Exception => throw new Exception("maxCharsPerCol must be a valid integer") + } + + val trustCredStr = parameters.getOrElse("certValidation", "") + val userCredStr = parameters.getOrElse("userCred", "") + if(userCredStr == "") + throw new Exception("User Credential has to be set") + + val connPropStr = parameters.getOrElse("connProp", "10000:60000") + var partitionDetailsStr = parameters.getOrElse("partitions", "4:10000") + + if(!partitionDetailsStr.contains(":")) partitionDetailsStr = partitionDetailsStr + ":10000" + + val formatDetailsStr = parameters.getOrElse("format", "csv:\n") + val formatDetailsArr = formatDetailsStr.split(":") + val formatType = formatDetailsArr(0) + var formatRecordSeparator = formatDetailsArr(1) + if(formatType != "csv" && formatRecordSeparator == "") + throw new Exception("Record Separator cannot be inferred for Format other than csv") + if(formatType == "csv" && formatRecordSeparator == "") formatRecordSeparator = "\n" + + val outputTypeStr = parameters.getOrElse("output", "Data") + + val outRdd = (outputTypeStr : @switch) match { + case "LIST" => WebHdfsConnector.listFromWebHdfs(sqlContext.sparkContext, path, trustCredStr, userCredStr, connPropStr) + case default => WebHdfsConnector.loadFromWebHdfs(sqlContext.sparkContext, path, charset, trustCredStr, userCredStr, connPropStr, partitionDetailsStr, formatRecordSeparator) + } + + val targetSchema = (outputTypeStr : @switch) match { + case "LIST" => StructType(Array(StructField("Name",StringType,true),StructField("File Size",LongType,true),StructField("Block Size",LongType,true), StructField("# of Blocks", IntegerType, true))) + case default => schema + } + + + val relation = (formatType : @switch) match { + case "csv" => + WebHdfsCsvRelation( + () => outRdd, + Some(path), + headerFlag, + delimiter, + quoteChar, + escapeChar, + commentChar, + parseMode, + parserLib, + ignoreLeadingWhiteSpaceFlag, + ignoreTrailingWhiteSpaceFlag, + treatEmptyValuesAsNullsFlag, + targetSchema, + inferSchemaFlag, + codec, + nullValue, + dateFormat, + maxCharsPerCol)(sqlContext) + case default => throw new Exception("Format Not Supported") + } + + relation + } + + override def createRelation( + sqlContext: SQLContext, + mode: SaveMode, + parameters: Map[String, String], + data: DataFrame): BaseRelation = { + + val path = checkPath(parameters) + + val trustStoreCredStr = parameters.getOrElse("certValidation", "") + + val connStr = parameters.getOrElse("connProp", "1000:10000") + + val userCredStr = parameters.getOrElse("userCred", "") + if (userCredStr == "") + throw new Exception("User Credentials cannot be null") + + val partitionStr = parameters.getOrElse("partitions", "4") + + val formatStr = parameters.getOrElse("format", "csv") + + /* + * Writre now only CSV format is supported + */ + + val rddToWrite = (formatStr : @switch) match { + case "csv" => WebHdfsCsvFormatter.convToCsvFormat(data, parameters) + case default => throw new Exception("Format Not Supported") + } + + WebHdfsConnector.writeToWebHdfs(rddToWrite, path, trustStoreCredStr, connStr, userCredStr, partitionStr) + + createRelation(sqlContext, parameters, data.schema) + } + +} diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvFormatter.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvFormatter.scala new file mode 100644 index 00000000..49cfcf44 --- /dev/null +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvFormatter.scala @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.bahir.datasource.webhdfs.csv + +import java.io.IOException +import java.text.SimpleDateFormat +import java.sql.{Timestamp, Date} + +import scala.collection.JavaConversions._ +import scala.util.control.NonFatal + +import org.apache.commons.csv.{CSVFormat, QuoteMode} +import org.apache.commons.csv._ +import org.slf4j.LoggerFactory + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql._ +import org.apache.spark.sql.sources.{PrunedScan, BaseRelation, InsertableRelation, TableScan} +import org.apache.spark.sql.types._ + + +/** + * This object contains all utility functions for converting data to CSV format + * This is copied from com.databricks.spark.csv as the required object could not be reused as it is declared as private +*/ + +private[webhdfs] object WebHdfsCsvFormatter { + + def convToCsvFormat(dataFrame: DataFrame, parameters: Map[String, String] = Map()) : RDD[String] = { + + val delimiter = parameters.getOrElse("delimiter", ",") + val dateFormat = parameters.getOrElse("dateFormat", "yyyy-MM-dd HH:mm:ss.S") + val dateFormatter: SimpleDateFormat = new SimpleDateFormat(dateFormat) + + val delimiterChar = if (delimiter.length == 1) { + delimiter.charAt(0) + } else { + throw new Exception("Delimiter cannot be more than one character.") + } + + val escape = parameters.getOrElse("escape", null) + val escapeChar: Character = if (escape == null) { + null + } else if (escape.length == 1) { + escape.charAt(0) + } else { + throw new Exception("Escape character cannot be more than one character.") + } + + val quote = parameters.getOrElse("quote", "\"") + val quoteChar: Character = if (quote == null) { + null + } else if (quote.length == 1) { + quote.charAt(0) + } else { + throw new Exception("Quotation cannot be more than one character.") + } + + val quoteModeString = parameters.getOrElse("quoteMode", "MINIMAL") + val quoteMode: QuoteMode = if (quoteModeString == null) { + null + } else { + QuoteMode.valueOf(quoteModeString.toUpperCase) + } + + val nullValue = parameters.getOrElse("nullValue", "null") + + val csvFormat = CSVFormat.DEFAULT + .withDelimiter(delimiterChar) + .withQuote(quoteChar) + .withEscape(escapeChar) + .withQuoteMode(quoteMode) + .withSkipHeaderRecord(false) + .withNullString(nullValue) + + val generateHeader = parameters.getOrElse("header", "false").toBoolean + val header = if (generateHeader) { + csvFormat.format(dataFrame.columns.map(_.asInstanceOf[AnyRef]): _*) + } else { + "" // There is no need to generate header in this case + } + + val schema = dataFrame.schema + val formatForIdx = schema.fieldNames.map(fname => schema(fname).dataType match { + case TimestampType => (timestamp: Any) => { + if (timestamp == null) { + nullValue + } else { + dateFormatter.format(new java.sql.Date(timestamp.asInstanceOf[Timestamp].getTime)) + } + } + case DateType => (date: Any) => { + if (date == null) nullValue else dateFormatter.format(date) + } + case _ => (fieldValue: Any) => fieldValue.asInstanceOf[AnyRef] + }) + + val strRDD = dataFrame.rdd.mapPartitionsWithIndex { case (index, iter) => + val csvFormat = CSVFormat.DEFAULT + .withDelimiter(delimiterChar) + .withQuote(quoteChar) + .withEscape(escapeChar) + .withQuoteMode(quoteMode) + .withSkipHeaderRecord(false) + .withNullString(nullValue) + + new Iterator[String] { + var firstRow: Boolean = generateHeader + + override def hasNext: Boolean = iter.hasNext || firstRow + + override def next: String = { + if (iter.nonEmpty) { + // try .zipWithIndex.foreach + val values: Seq[AnyRef] = iter.next().toSeq.zipWithIndex.map { + case (fieldVal, i) => formatForIdx(i)(fieldVal) + } + val row = csvFormat.format(values: _*) + if (firstRow) { + firstRow = false + header + "\n" + row + } else { + row + } + } else { + firstRow = false + header + } + } + } + } + strRDD + } +} diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvParserLibs.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvParserLibs.scala new file mode 100644 index 00000000..71798805 --- /dev/null +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvParserLibs.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.bahir.datasource.webhdfs.csv + + +/** + * This object contains all utility functions needed for determining CSV parser lib + * This is copied from com.databricks.spark.csv.util as the required object could not be reused as it is declared as private +*/ + +private[webhdfs] object WebHdfsCsvParserLibs { + val OLD = "COMMONS" + val NEW = "UNIVOCITY" + + val DEFAULT = OLD + + def isValidLib(lib: String): Boolean = { + lib.toUpperCase match { + case OLD | NEW => true + case _ => false + } + } + + def isCommonsLib(lib: String): Boolean = if (isValidLib(lib)) { + lib.toUpperCase == OLD + } else { + true // default + } + + def isUnivocityLib(lib: String): Boolean = if (isValidLib(lib)) { + lib.toUpperCase == NEW + } else { + false // not the default + } + +} diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvRelation.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvRelation.scala new file mode 100644 index 00000000..28a83c9f --- /dev/null +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvRelation.scala @@ -0,0 +1,294 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + + +package org.apache.bahir.datasource.webhdfs.csv + +import java.io.IOException +import java.text.SimpleDateFormat + +import scala.collection.JavaConversions._ +import scala.util.control.NonFatal + +import org.apache.commons.csv.{CSVFormat, QuoteMode} +import org.apache.commons.csv._ +import org.slf4j.LoggerFactory + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql._ +import org.apache.spark.sql.sources.{PrunedScan, BaseRelation, InsertableRelation, TableScan} +import org.apache.spark.sql.types._ + + +import org.apache.bahir.datasource.webhdfs.util._ + +/* + * This class contains functions for converting RDD to csv data source + * This is copied from com.databricks.spark.csv as the required object could not be reused as it is declared as private +*/ + + +case class WebHdfsCsvRelation protected[webhdfs] ( + baseRDD: () => RDD[String], + location: Option[String], + useHeader: Boolean, + delimiter: Char, + quote: Character, + escape: Character, + comment: Character, + parseMode: String, + parserLib: String, + ignoreLeadingWhiteSpace: Boolean, + ignoreTrailingWhiteSpace: Boolean, + treatEmptyValuesAsNulls: Boolean, + userSchema: StructType = null, + inferCsvSchema: Boolean, + codec: String = null, + nullValue: String = "", + dateFormat: String = null, + maxCharsPerCol: Int = 100000)(@transient val sqlContext: SQLContext) + extends BaseRelation with TableScan with PrunedScan { + + // Share date format object as it is expensive to parse date pattern. + private val dateFormatter = if (dateFormat != null) new SimpleDateFormat(dateFormat) else null + + private val logger = LoggerFactory.getLogger(WebHdfsCsvRelation.getClass) + + // Parse mode flags + if (!WebHdfsParseModes.isValidMode(parseMode)) { + logger.warn(s"$parseMode is not a valid parse mode. Using ${WebHdfsParseModes.DEFAULT}.") + } + + if ((ignoreLeadingWhiteSpace || ignoreLeadingWhiteSpace) && WebHdfsCsvParserLibs.isCommonsLib(parserLib)) { + logger.warn(s"Ignore white space options may not work with Commons parserLib option") + } + + private val failFast = WebHdfsParseModes.isFailFastMode(parseMode) + private val dropMalformed = WebHdfsParseModes.isDropMalformedMode(parseMode) + private val permissive = WebHdfsParseModes.isPermissiveMode(parseMode) + + override val schema: StructType = inferSchema() + + private def tokenRdd(header: Array[String]): RDD[Array[String]] = { + + val csvFormat = CSVFormat.DEFAULT + .withDelimiter(delimiter) + .withQuote(quote) + .withEscape(escape) + .withSkipHeaderRecord(false) + .withHeader(header: _*) + .withCommentMarker(comment) + + // If header is set, make sure firstLine is materialized before sending to executors. + val filterLine = if (useHeader) firstLine else null + + baseRDD().mapPartitions { iter => + // When using header, any input line that equals firstLine is assumed to be header + val csvIter = if (useHeader) { + iter.filter(_ != filterLine) + } else { + iter + } + parseCSV(csvIter, csvFormat) + } + } + + override def buildScan: RDD[Row] = { + val simpleDateFormatter = dateFormatter + val schemaFields = schema.fields + val rowArray = new Array[Any](schemaFields.length) + tokenRdd(schemaFields.map(_.name)).flatMap { tokens => + + if (dropMalformed && schemaFields.length != tokens.length) { + logger.warn(s"Dropping malformed line: ${tokens.mkString(",")}") + None + } else if (failFast && schemaFields.length != tokens.length) { + throw new RuntimeException(s"Malformed line in FAILFAST mode: ${tokens.mkString(",")}") + } else { + var index: Int = 0 + try { + index = 0 + while (index < schemaFields.length) { + val field = schemaFields(index) + rowArray(index) = WebHdfsTypeCast.castTo(tokens(index), field.dataType, field.nullable, + treatEmptyValuesAsNulls, nullValue, simpleDateFormatter) + index = index + 1 + } + Some(Row.fromSeq(rowArray)) + } catch { + case aiob: ArrayIndexOutOfBoundsException if permissive => + (index until schemaFields.length).foreach(ind => rowArray(ind) = null) + Some(Row.fromSeq(rowArray)) + case _: java.lang.NumberFormatException | + _: IllegalArgumentException if dropMalformed => + logger.warn("Number format exception. " + + s"Dropping malformed line: ${tokens.mkString(delimiter.toString)}") + None + case pe: java.text.ParseException if dropMalformed => + logger.warn("Parse exception. " + + s"Dropping malformed line: ${tokens.mkString(delimiter.toString)}") + None + } + } + } + } + + + /** + * This supports to eliminate unneeded columns before producing an RDD + * containing all of its tuples as Row objects. This reads all the tokens of each line + * and then drop unneeded tokens without casting and type-checking by mapping + * both the indices produced by `requiredColumns` and the ones of tokens. + */ + override def buildScan(requiredColumns: Array[String]): RDD[Row] = { + val simpleDateFormatter = dateFormatter + val schemaFields = schema.fields + val requiredFields = StructType(requiredColumns.map(schema(_))).fields + val shouldTableScan = schemaFields.deep == requiredFields.deep + val safeRequiredFields = if (dropMalformed) { + // If `dropMalformed` is enabled, then it needs to parse all the values + // so that we can decide which row is malformed. + requiredFields ++ schemaFields.filterNot(requiredFields.contains(_)) + } else { + requiredFields + } + val rowArray = new Array[Any](safeRequiredFields.length) + if (shouldTableScan) { + buildScan() + } else { + val safeRequiredIndices = new Array[Int](safeRequiredFields.length) + schemaFields.zipWithIndex.filter { + case (field, _) => safeRequiredFields.contains(field) + }.foreach { + case (field, index) => safeRequiredIndices(safeRequiredFields.indexOf(field)) = index + } + val requiredSize = requiredFields.length + tokenRdd(schemaFields.map(_.name)).flatMap { tokens => + + if (dropMalformed && schemaFields.length != tokens.length) { + logger.warn(s"Dropping malformed line: ${tokens.mkString(delimiter.toString)}") + None + } else if (failFast && schemaFields.length != tokens.length) { + throw new RuntimeException(s"Malformed line in FAILFAST mode: " + + s"${tokens.mkString(delimiter.toString)}") + } else { + val indexSafeTokens = if (permissive && schemaFields.length > tokens.length) { + tokens ++ new Array[String](schemaFields.length - tokens.length) + } else if (permissive && schemaFields.length < tokens.length) { + tokens.take(schemaFields.length) + } else { + tokens + } + try { + var index: Int = 0 + var subIndex: Int = 0 + while (subIndex < safeRequiredIndices.length) { + index = safeRequiredIndices(subIndex) + val field = schemaFields(index) + rowArray(subIndex) = WebHdfsTypeCast.castTo( + indexSafeTokens(index), + field.dataType, + field.nullable, + treatEmptyValuesAsNulls, + nullValue, + simpleDateFormatter + ) + subIndex = subIndex + 1 + } + Some(Row.fromSeq(rowArray.take(requiredSize))) + } catch { + case _: java.lang.NumberFormatException | + _: IllegalArgumentException if dropMalformed => + logger.warn("Number format exception. " + + s"Dropping malformed line: ${tokens.mkString(delimiter.toString)}") + None + case pe: java.text.ParseException if dropMalformed => + logger.warn("Parse exception. " + + s"Dropping malformed line: ${tokens.mkString(delimiter.toString)}") + None + } + } + } + } + } + + private def inferSchema(): StructType = { + if (this.userSchema != null) { + userSchema + } else { + val csvFormat = CSVFormat.DEFAULT + .withDelimiter(delimiter) + .withQuote(quote) + .withEscape(escape) + .withSkipHeaderRecord(false) + val firstRow = CSVParser.parse(firstLine, csvFormat).getRecords.head.toArray + val header = if (useHeader) { + firstRow + } else { + firstRow.zipWithIndex.map { case (value, index) => s"C$index"} + } + if (this.inferCsvSchema) { + val simpleDateFormatter = dateFormatter + WebHdfsInferSchema(tokenRdd(header), header, nullValue, simpleDateFormatter) + } else { + // By default fields are assumed to be StringType + val schemaFields = header.map { fieldName => + StructField(fieldName.toString, StringType, nullable = true) + } + StructType(schemaFields) + } + } + } + + /** + * Returns the first line of the first non-empty file in path + */ + private lazy val firstLine = { + if (comment != null) { + baseRDD().filter { line => + line.trim.nonEmpty && !line.startsWith(comment.toString) + }.first() + } else { + baseRDD().filter { line => + line.trim.nonEmpty + }.first() + } + } + + + private def parseCSV( + iter: Iterator[String], + csvFormat: CSVFormat): Iterator[Array[String]] = { + iter.flatMap { line => + try { + val records = CSVParser.parse(line, csvFormat).getRecords + if (records.isEmpty) { + logger.warn(s"Ignoring empty line: $line") + None + } else { + Some(records.head.toArray) + } + } catch { + case NonFatal(e) if !failFast => + logger.error(s"Exception while parsing line: $line. ", e) + None + } + } + } + +} diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/package.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/package.scala new file mode 100644 index 00000000..f03640af --- /dev/null +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/package.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.bahir.datasource.webhdfs + + +package object webhdfs { + +} diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala new file mode 100644 index 00000000..521c02e0 --- /dev/null +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.bahir.datasource.webhdfs.util + +import java.security._ +import javax.net.ssl.TrustManagerFactory +import java.io._ +import javax.net.ssl.SSLSocketFactory +import javax.net.ssl.SSLContext +import scala.collection.mutable.HashMap +import sys.process._ + +/** + * This Singleton is used to generate SSLTrustStore certification once. The assumption behind use of this trust store is that this code would be executed on a machine which would be accessible from all Spark executors which need to access the trust store +**/ + +object SSLTrustStore{ + + var trustStoreFileMap : HashMap[String, File] = HashMap() + + /** + * This function checks the availability of truststore for a particular site. If not it creates a new one. + **/ + + + def getCertDetails(path: String): Tuple2[File, String] = { + + val pathComp = path.split("/") + + val srvr = pathComp(2) + + val trustStorePword = "ts-password" + + val currDir = ("pwd" !!).trim + val trustStore = currDir + "/" + srvr + "_trustStore.jks" + + val os = new java.io.ByteArrayOutputStream + + val tsExist = (s"ls $trustStore" #> os ).! + + val f = if (tsExist == 0) + { + + print("Using Existing Trust Store for SSL" + "\n") + new java.io.File(trustStore) + } + else { + val cert = srvr + "_cert" + val cfl = new File(cert) + + (s"openssl s_client -showcerts -connect $srvr" #| "openssl x509 -outform PEM" #> cfl).! + (s"keytool -import -trustcacerts -alias hadoop -file $cert -keystore $trustStore -storepass $trustStorePword -noprompt").! + (s"rm -f $cert").! + new java.io.File(trustStore) + } + + new Tuple2(f, trustStorePword) + + } + +} diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala new file mode 100644 index 00000000..8f5a56de --- /dev/null +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala @@ -0,0 +1,471 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.bahir.datasource.webhdfs.util + +import java.nio.charset.Charset +import java.text.SimpleDateFormat +import java.sql.{Timestamp, Date} +import java.util.Date + +import org.apache.spark.sql.types.{DateType, TimestampType} + +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD + +import scala.annotation.switch +import scalaj.http._ +import java.security._ +import javax.net.ssl.TrustManagerFactory +import java.io._ +import javax.net.ssl.SSLSocketFactory +import javax.net.ssl.SSLContext +import org.apache.http.conn.scheme.Scheme +import java.net.HttpURLConnection +import scala.collection.mutable.HashMap +import scala.math._ +import org.apache.spark.sql.types.{DateType, TimestampType} +import org.apache.spark.sql.{DataFrame, SQLContext} +import scala.collection.mutable.ArrayBuffer +import scala.util.control.Breaks._ + + +/** + * This object contains all utility functions for reading/writing data from/to remote webhdfs server. The abstraction maintained in this layer is at the level of RDD +*/ + +private[webhdfs] object WebHdfsConnector { + + + /** + * Currently only files transferred using UTF-8 are supported + + */ + + val DEFAULT_CHARSET = Charset.forName("UTF-8") + + /** + * This function returns a Tuple for credential store which contains flag for validating Certificate, the Certificate File object and Certificate File Object password + + */ + + def createTrustStoreCredForExecutors(cred: String, path: String) : Tuple3[String, File, String] = { + + val trustStoreMap = if (cred != "") { + if (cred == "N") + new Tuple3("N", null, "") + else if(cred == "Y") + { + val tsd = SSLTrustStore.getCertDetails(path) + + new Tuple3("Y", tsd._1, tsd._2) + } + else + throw new Exception("Invalid Certificate Validation Option") + + } else { + new Tuple3("", null, "") + } + trustStoreMap + + + } + + + /** + * This function returns a SSLSocketFactory which needs to be used in HTTP connection library in case Certificate to be validated + */ + + def biocSslSocketFactory(fl: File, pswrd: String): SSLSocketFactory = { + + + val tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()) + val ks = KeyStore.getInstance("JKS") + val fis = new java.io.FileInputStream(fl) + ks.load(fis, pswrd.toCharArray()); + tmf.init(ks); + + + val sslc = SSLContext.getInstance("SSL") + + sslc.init(null, tmf.getTrustManagers(),null) + + sslc.getSocketFactory() + } + + + /** + * This function returns the details of the the files in a folder if the path passed is a folder. In case a File path is passed it returns the details of teh files. + * This returns an Array of Tuple where each Tuple represents one file with details of full file path, + * size of the file, block size of the file and number of partitions based on size of the file and blick size + **/ + + def getFilesDetails(path: String, trustCred: Tuple3[String, File, String], usrCred: Array[String], connProp: Array[Int]): Array [Tuple4[String, Long, Long, Int]] = { + + + val listStatusOpr = s"op=LISTSTATUS" + + val listStatus = callWebHdfsAPI(path, "", "GET", "BODY", trustCred, usrCred, connProp, listStatusOpr) + + if (listStatus.contains("RemoteException")) + throw new Exception(listStatus) + + val flist = scala.util.parsing.json.JSON.parseFull(listStatus).toList(0).asInstanceOf[Map[String, Map[String, Any]]].get("FileStatuses").get("FileStatus").asInstanceOf[List[Map[String, Any]]] + + val fileCount = flist.length + + var i = 0 + var j = 0L + var fileDetails = new Array[Tuple4[String, Long, Long, Int]](fileCount) + var fSuffix = "" + var fLength = 0L + var fBlocksize = 0L + var fPart = 0 + var fullFilePath = "" + + while(i < fileCount) + { + fSuffix = flist(i).get("pathSuffix").getOrElse(path).asInstanceOf[String].toString + + fullFilePath = if (fSuffix == "") path else (path + "/" + fSuffix) + + fLength = flist(i).get("length").getOrElse(0).asInstanceOf[Double].toLong + + fBlocksize = flist(i).get("blockSize").getOrElse(0).asInstanceOf[Double].toLong + if (fLength > 0) fPart = (floor((fLength/fBlocksize)).toInt+1) else fPart = 0 + fileDetails(i) = new Tuple4(fullFilePath, fLength, fBlocksize, fPart) + + i+=1 + + } + fileDetails + + } + + /** + * This function prepares the partition details for each file based on the details populated by getFilesDetails + * This partition details is further used to spawn multiple connections to get data of a file using multiple connections + **/ + + def preparePartitions(fileDetails: Array[Tuple4[String, Long, Long, Int]], baseFile: String, partitionDetails: String, recordSeparator: String): Array [Tuple7[String, Long, Long, Int, Int, Int, String]] = { + + val totalFileCount = fileDetails.length + var i = 0 + + val partitionDet = partitionDetails.split(":") + + /** + *If number of partitions used for opening connections is passed as 0 or less, partition is defaulted to 4 + **/ + + var filePartition = if (partitionDet(0).toInt < 1) 4 else partitionDet(0).toInt + + /** + *If partition span (used to resolve record boundary) is sent as less than 10 KB, it is defaulted to 10 KB. Otherwise it is kept between 10KB to 100 KB + **/ + + var partitionSpan = if(partitionDet(1).toInt < 10000) 10000 else math.min(partitionDet(1).toInt, 100000) + + i = 0 + + var partList = new ArrayBuffer[Tuple7[String, Long, Long, Int, Int, Int, String]]() + + var j = 0 + var k = 0 + var filePart = 0 + var partPath = "" + var partLength = 0L + var fileLength = 0L + var partOffset = 0L + var fileSpan = 0 + + val maxSpan = 1000000 + val minSpan = 1000 + + while(i < totalFileCount) + { + + fileLength = fileDetails(i)._2.toLong + + if (fileLength > 0) { + + partPath = fileDetails(i)._1 + + fileLength = fileDetails(i)._2.toLong + partLength = fileLength/filePartition + + + if (partLength < 1000000) + { + filePartition = 1 + partitionSpan = 0 + } + + + j = 0 + while (j < filePartition) + { + partOffset = j*partLength.toLong + + if (j+1 == filePartition) partLength = fileDetails(i)._2.toLong - j*partLength + else partLength + + partList += new Tuple7(partPath, partOffset, partLength, j+1, filePartition, partitionSpan, recordSeparator) + j+=1 + k+=1 + } + } + + i+=1 + + } + + if (k < 1) + throw new Exception("Zero File Content") + + + var finalPartList = new Array[Tuple7[String, Long, Long, Int, Int, Int, String]](k) + + partList.copyToArray(finalPartList) + + finalPartList + } + + /** + * This function returns the list of files in a folder with file details as RDD + **/ + + def listFromWebHdfs(sc: SparkContext, path: String, trustStoreCred: String, userCred: String, connProp: String): RDD[String] = { + + + + val conns = connProp.split(":") + + val conn = Array(conns(0).toInt, conns(1).toInt) + + val usrCred = userCred.split(":") + + val trustCred = createTrustStoreCredForExecutors(trustStoreCred, path) + + val fileDetails = getFilesDetails(path, trustCred, usrCred, conn) + + def g(v:Tuple4[String, Long, Long, Int]) = v._1.split("/").last + "," + v._2.toString + "," + v._3.toString + "," + v._4.toString + "\n" + + val fds = fileDetails.map(x => g(x)) + + val flRdd = sc.parallelize(fds) + + flRdd + + } + + /** + * This function returns data of a file (or data of all files in a folder with same structure) as RDD + **/ + + def loadFromWebHdfs(sc: SparkContext, path: String, charset: String, trustStoreCred: String, userCred: String, connProp: String, partitionDetails: String, recordSeparator: String): RDD[String] = { + + + val conns = connProp.split(":") + + val conn = Array(conns(0).toInt, conns(1).toInt) + + val usrCrd = userCred.split(":") + + val trustCred = createTrustStoreCredForExecutors(trustStoreCred, path) + + val fileDetails = getFilesDetails(path, trustCred, usrCrd, conn) + + val parts = preparePartitions(fileDetails, path, partitionDetails, recordSeparator) + + val input = sc.parallelize(parts, parts.length) + + input.collect() + + val fRdd = input.flatMap(x => WebHdfsConnector.getAllFiles(x, usrCrd, trustCred, conn)) + + fRdd + + } + + + /** + * This function is passed to each executor through flatMap function to spawn one http connection from each executor for get a part of the file + **/ + + def getAllFiles (partInfo : Tuple7[String, Long, Long, Int, Int, Int, String], usrCred: Array[String], trustCred: Tuple3[String, File, String], connProp: Array[Int]): Iterator[String] = { + + val foffset = partInfo._2.toLong + val flength = partInfo._3.toLong + val ffilePath = partInfo._1 + val fpartNum = partInfo._4 + val ftotalPart = partInfo._5 + val fspan = partInfo._6 + val frecordSeparator = partInfo._7 + + val fileGetOpr = if(fpartNum < ftotalPart) + { + val effLength = flength + fspan + s"op=OPEN&offset=$foffset&length=$effLength&bufferSize=$effLength" + } + else + s"op=OPEN&offset=$foffset&length=$flength&bufferSize=$flength" + + val getUrl = callWebHdfsAPI(ffilePath, "", "GET", "LOCATION", trustCred, usrCred, connProp, fileGetOpr) + val partContent = callWebHdfsAPI(getUrl, "", "GET", "BODY", trustCred, usrCred, connProp, fileGetOpr) + + val records = getTillEndOfRecord(partContent, flength, fpartNum, ftotalPart, frecordSeparator) + + records.split("\n").iterator + + } + + /** + * This function calls webhdfs API after creating all necessary parameters from different configurations + **/ + + def callWebHdfsAPI(path: String, data: String, method: String, respType: String, trustStoreCred: Tuple3[String, File, String], userCred: Array[String], connProp: Array[Int], opr: String): String = { + + + val pathComp = path.split(":") + + val uri = (if(trustStoreCred._1 != "") "https:" else "http:") + pathComp(1) + ":" + pathComp(2) + "?" + opr + + var httpc = Http(uri).auth(userCred(0), userCred(1)).timeout(connTimeoutMs = connProp(0), readTimeoutMs = connProp(1)) + + httpc = (method : @switch) match { + case "GET" => httpc + case "PUT" => httpc.put(data).header("content-type", "application/csv") + case "POST" => httpc.postData(data).header("content-type", "application/csv") + } + + httpc = (trustStoreCred._1 : @switch) match { + case "" => httpc + case "N" => httpc.option(HttpOptions.allowUnsafeSSL) + case "Y" => httpc.option(HttpOptions.sslSocketFactory(biocSslSocketFactory(trustStoreCred._2, trustStoreCred._3))) + } + + val resp = (respType : @switch) match { + case "BODY" => httpc.asString.body + case "CODE" => httpc.asString.code + case "HEADERS" => httpc.asString.headers + case "LOCATION" => httpc.asString.location.mkString(" ") + } + + resp.toString() + + } + + /** + * This function resolves record boundaries. + * Right now this only supports "\n" as record boundary . This function has to be refined to support json or xml formats for different type of record separators + **/ + + def getTillEndOfRecord (content : String, partLength: Long, partNum: Int, totalPart: Int, recordSeparator : String): String = { + + val contentBytes = content.getBytes("UTF-8") + val recordSeparatorBytes = recordSeparator.getBytes("UTF-8") + + val contentBytesLength = contentBytes.length + + var bytePosition = 0 + + var startbyte = 0 + + startbyte = if(partNum == 1) 0 else { + /* + * This part of the code has to be rewritten later on to make it more generic for supporting other formats apart from csv. Right now it supports only csv. + */ + while (contentBytes(bytePosition) != '\n') bytePosition += 1 + bytePosition + } + + val length = if (partNum == totalPart) (contentBytesLength.toInt - startbyte).toInt + else { + bytePosition = partLength.toInt + /* + * This part of the code has to be rewritten later on to make it more generic for supporting other formats apart from csv. Right now it supports only csv. + */ + while (contentBytes(bytePosition) != '\n') bytePosition += 1 + (bytePosition - startbyte) + + } + + new String(contentBytes, startbyte, length, "UTF-8") + + } + + /** + * This function writes data back to hdfs using WebHDFS using multiple parallel connections. Right now file overwrite is not supported + **/ + + def writeToWebHdfs(dataToWrite: RDD[String] ,path: String, trustStoreCredStr: String, connStr : String, userCredStr : String, partitionStr : String): Unit = { + + val trustCred = createTrustStoreCredForExecutors(trustStoreCredStr, path) + + val conns = connStr.split(":") + + val conn = Array(conns(0).toInt, conns(1).toInt) + + val usr = userCredStr.split(":") + + val webHdfsChkDirOpr = "op=GETFILESTATUS" + val returnChkDir = callWebHdfsAPI(path, "", "GET", "CODE", trustCred, usr, conn, webHdfsChkDirOpr) + + if (returnChkDir == "200") + throw new Exception("The File Already Exists : " + path + "\n") + + val dPartitions = partitionStr.toInt + + val textRdd = dataToWrite.repartition(dPartitions) + + val webHdfsMakeDirOpr = "op=MKDIRS" + val returnCreateDir = callWebHdfsAPI(path, "", "PUT", "CODE", trustCred, usr, conn, webHdfsMakeDirOpr) + + textRdd.mapPartitionsWithIndex((idx, iter) => WebHdfsConnector.saveAllFiles(idx, iter, usr, path, trustCred, conn)).collect() + + } + + /** + * This function is passed to mapPartitionsWithIndex so that each executor task can save part of the data using separate connection + **/ + + def saveAllFiles (idx: Int, data : Iterator[String], usrCred: Array[String], path: String, trustCred: Tuple3[String, File , String], connProp: Array[Int]): Iterator[String] = { + + + var dataP = data.next() + while(data.hasNext) { + dataP = dataP + "\n" + data.next() + } + + val fnameArray = path.split("/") + val fnameIdx = fnameArray.length - 1 + val fname = fnameArray(fnameIdx) + val filePath = s"$path/part-000$idx-$fname" + + val createOpr = "op=CREATE" + + val createUrl = callWebHdfsAPI(filePath, "", "PUT", "LOCATION", trustCred, usrCred, connProp, createOpr) + val created = callWebHdfsAPI(createUrl, dataP, "PUT", "CODE", trustCred, usrCred, connProp, createOpr) + + val ret = Array(created.toString) + ret.iterator + + } + +} + diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsInferSchema.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsInferSchema.scala new file mode 100644 index 00000000..4b4935e9 --- /dev/null +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsInferSchema.scala @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.bahir.datasource.webhdfs.util + +import java.sql.Timestamp +import java.text.SimpleDateFormat + +import scala.util.control.Exception._ + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.types._ + +/** + * Utility functions for infering schema + * Copied from com.databricks.spark.csv.util package as there was now way to reuse this object +*/ + + +private[webhdfs] object WebHdfsInferSchema { + + /** + * Similar to the JSON schema inference. + * [[org.apache.spark.sql.execution.datasources.json.InferSchema]] + * 1. Infer type of each row + * 2. Merge row types to find common type + * 3. Replace any null types with string type + */ + def apply( + tokenRdd: RDD[Array[String]], + header: Array[String], + nullValue: String = "", + dateFormatter: SimpleDateFormat = null): StructType = { + val startType: Array[DataType] = Array.fill[DataType](header.length)(NullType) + val rootTypes: Array[DataType] = tokenRdd.aggregate(startType)( + inferRowType(nullValue, dateFormatter), + mergeRowTypes) + + val structFields = header.zip(rootTypes).map { case (thisHeader, rootType) => + val dType = rootType match { + case z: NullType => StringType + case other => other + } + StructField(thisHeader, dType, nullable = true) + } + + StructType(structFields) + } + + private def inferRowType(nullValue: String, dateFormatter: SimpleDateFormat) + (rowSoFar: Array[DataType], next: Array[String]): Array[DataType] = { + var i = 0 + while (i < math.min(rowSoFar.length, next.length)) { // May have columns on right missing. + rowSoFar(i) = inferField(rowSoFar(i), next(i), nullValue, dateFormatter) + i+=1 + } + rowSoFar + } + + private[webhdfs] def mergeRowTypes( + first: Array[DataType], + second: Array[DataType]): Array[DataType] = { + first.zipAll(second, NullType, NullType).map { case ((a, b)) => + findTightestCommonType(a, b).getOrElse(NullType) + } + } + + /** + * Infer type of string field. Given known type Double, and a string "1", there is no + * point checking if it is an Int, as the final type must be Double or higher. + */ + private[webhdfs] def inferField(typeSoFar: DataType, + field: String, + nullValue: String = "", + dateFormatter: SimpleDateFormat = null): DataType = { + def tryParseInteger(field: String): DataType = if ((allCatch opt field.toInt).isDefined) { + IntegerType + } else { + tryParseLong(field) + } + + def tryParseLong(field: String): DataType = if ((allCatch opt field.toLong).isDefined) { + LongType + } else { + tryParseDouble(field) + } + + def tryParseDouble(field: String): DataType = { + if ((allCatch opt field.toDouble).isDefined) { + DoubleType + } else { + tryParseTimestamp(field) + } + } + + def tryParseTimestamp(field: String): DataType = { + if (dateFormatter != null) { + // This case infers a custom `dataFormat` is set. + if ((allCatch opt dateFormatter.parse(field)).isDefined){ + TimestampType + } else { + tryParseBoolean(field) + } + } else { + // We keep this for backwords competibility. + if ((allCatch opt Timestamp.valueOf(field)).isDefined) { + TimestampType + } else { + tryParseBoolean(field) + } + } + } + + def tryParseBoolean(field: String): DataType = { + if ((allCatch opt field.toBoolean).isDefined) { + BooleanType + } else { + stringType() + } + } + + // Defining a function to return the StringType constant is necessary in order to work around + // a Scala compiler issue which leads to runtime incompatibilities with certain Spark versions; + // see issue #128 for more details. + def stringType(): DataType = { + StringType + } + + if (field == null || field.isEmpty || field == nullValue) { + typeSoFar + } else { + typeSoFar match { + case NullType => tryParseInteger(field) + case IntegerType => tryParseInteger(field) + case LongType => tryParseLong(field) + case DoubleType => tryParseDouble(field) + case TimestampType => tryParseTimestamp(field) + case BooleanType => tryParseBoolean(field) + case StringType => StringType + case other: DataType => + throw new UnsupportedOperationException(s"Unexpected data type $other") + } + } + } + + /** + * Copied from internal Spark api + * [[org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion]] + */ + private val numericPrecedence: IndexedSeq[DataType] = + IndexedSeq[DataType]( + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + TimestampType, + DecimalType.USER_DEFAULT) + + + /** + * Copied from internal Spark api + * [[org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion]] + */ + val findTightestCommonType: (DataType, DataType) => Option[DataType] = { + case (t1, t2) if t1 == t2 => Some(t1) + case (NullType, t1) => Some(t1) + case (t1, NullType) => Some(t1) + case (StringType, t2) => Some(StringType) + case (t1, StringType) => Some(StringType) + + // Promote numeric types to the highest of the two and all numeric types to unlimited decimal + case (t1, t2) if Seq(t1, t2).forall(numericPrecedence.contains) => + val index = numericPrecedence.lastIndexWhere(t => t == t1 || t == t2) + Some(numericPrecedence(index)) + + case _ => None + } +} diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsParseModes.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsParseModes.scala new file mode 100644 index 00000000..332a6e38 --- /dev/null +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsParseModes.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.bahir.datasource.webhdfs.util + +/** + * Utility functions for mode of Parsing + * Copied from com.databricks.spark.csv.util package as there was now way to reuse this object +*/ + +private[webhdfs] object WebHdfsParseModes { + + val PERMISSIVE_MODE = "PERMISSIVE" + val DROP_MALFORMED_MODE = "DROPMALFORMED" + val FAIL_FAST_MODE = "FAILFAST" + + val DEFAULT = PERMISSIVE_MODE + + def isValidMode(mode: String): Boolean = { + mode.toUpperCase match { + case PERMISSIVE_MODE | DROP_MALFORMED_MODE | FAIL_FAST_MODE => true + case _ => false + } + } + + def isDropMalformedMode(mode: String): Boolean = mode.toUpperCase == DROP_MALFORMED_MODE + def isFailFastMode(mode: String): Boolean = mode.toUpperCase == FAIL_FAST_MODE + def isPermissiveMode(mode: String): Boolean = if (isValidMode(mode)) { + mode.toUpperCase == PERMISSIVE_MODE + } else { + true // We default to permissive is the mode string is not valid + } +} diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsTypeCast.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsTypeCast.scala new file mode 100644 index 00000000..26acd6bd --- /dev/null +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsTypeCast.scala @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.bahir.datasource.webhdfs.util + +import java.math.BigDecimal +import java.sql.{Date, Timestamp} +import java.text.{SimpleDateFormat, NumberFormat} +import java.util.Locale + +import org.apache.spark.sql.types._ + +import scala.util.Try + +/** + * Utility functions for type casting + * Copied from com.databricks.spark.csv.util package as there was now way to reuse this object +*/ + +object WebHdfsTypeCast { + + /** + * Casts given string datum to specified type. + * Currently we do not support complex types (ArrayType, MapType, StructType). + * + * For string types, this is simply the datum. For other types. + * For other nullable types, this is null if the string datum is empty. + * + * @param datum string value + * @param castType SparkSQL type + */ + private[webhdfs] def castTo( + datum: String, + castType: DataType, + nullable: Boolean = true, + treatEmptyValuesAsNulls: Boolean = false, + nullValue: String = "", + dateFormatter: SimpleDateFormat = null): Any = { + // if nullValue is not an empty string, don't require treatEmptyValuesAsNulls + // to be set to true + val nullValueIsNotEmpty = nullValue != "" + if (datum == nullValue && + nullable && + (!castType.isInstanceOf[StringType] || treatEmptyValuesAsNulls || nullValueIsNotEmpty) + ){ + null + } else { + castType match { + case _: ByteType => datum.toByte + case _: ShortType => datum.toShort + case _: IntegerType => datum.toInt + case _: LongType => datum.toLong + case _: FloatType => Try(datum.toFloat) + .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).floatValue()) + case _: DoubleType => Try(datum.toDouble) + .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).doubleValue()) + case _: BooleanType => datum.toBoolean + case _: DecimalType => new BigDecimal(datum.replaceAll(",", "")) + case _: TimestampType if dateFormatter != null => + new Timestamp(dateFormatter.parse(datum).getTime) + case _: TimestampType => Timestamp.valueOf(datum) + case _: DateType if dateFormatter != null => + new Date(dateFormatter.parse(datum).getTime) + case _: DateType => Date.valueOf(datum) + case _: StringType => datum + case _ => throw new RuntimeException(s"Unsupported type: ${castType.typeName}") + } + } + } + + /** + * Helper method that converts string representation of a character to actual character. + * It handles some Java escaped strings and throws exception if given string is longer than one + * character. + * + */ + @throws[IllegalArgumentException] + private[webhdfs] def toChar(str: String): Char = { + if (str.charAt(0) == '\\') { + str.charAt(1) + match { + case 't' => '\t' + case 'r' => '\r' + case 'b' => '\b' + case 'f' => '\f' + case '\"' => '\"' // In case user changes quote char and uses \" as delimiter in options + case '\'' => '\'' + case 'u' if str == """\u0000""" => '\u0000' + case _ => + throw new IllegalArgumentException(s"Unsupported special character for delimiter: $str") + } + } else if (str.length == 1) { + str.charAt(0) + } else { + throw new IllegalArgumentException(s"Delimiter cannot be more than one character: $str") + } + } +} diff --git a/pom.xml b/pom.xml index 65e6462d..6c516b60 100644 --- a/pom.xml +++ b/pom.xml @@ -80,6 +80,7 @@ sql-streaming-mqtt streaming-twitter streaming-zeromq + datasource-webhdfs From c2d53fdd55eee69120cf00dc17869786945ed93a Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Tue, 15 Nov 2016 17:00:39 -0800 Subject: [PATCH 02/24] [BAHIR-75] - fix RAT excludes for DataSourceRegister --- datasource-webhdfs/pom.xml | 41 ++++++++++++++------------------------ pom.xml | 1 + 2 files changed, 16 insertions(+), 26 deletions(-) diff --git a/datasource-webhdfs/pom.xml b/datasource-webhdfs/pom.xml index abb094f7..22b10501 100755 --- a/datasource-webhdfs/pom.xml +++ b/datasource-webhdfs/pom.xml @@ -31,35 +31,35 @@ datasource-webhdfs jar - Apache Bahir - Spark DataSource for WebHdfs + Apache Bahir - Spark DataSource WebHDFS http://bahir.apache.org/ - org.scalaj - scalaj-http_2.11 - 2.3.0 - - - org.apache.commons - commons-csv - 1.3 - - + org.scalaj + scalaj-http_2.11 + 2.3.0 + + + org.apache.commons + commons-csv + 1.3 + + org.apache.spark spark-tags_${scala.binary.version} org.apache.spark - spark-sql_${scala.binary.version} + spark-core_${scala.binary.version} ${spark.version} org.apache.spark - spark-core_${scala.binary.version} + spark-sql_${scala.binary.version} ${spark.version} - + org.apache.spark spark-hive_${scala.binary.version} ${spark.version} @@ -68,22 +68,11 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - org.apache.maven.plugins maven-source-plugin - - org.apache.rat - apache-rat-plugin - 0.11 - - - **/org.apache.spark.sql.sources.DataSourceRegister - - - - + diff --git a/pom.xml b/pom.xml index 6c516b60..960272d6 100644 --- a/pom.xml +++ b/pom.xml @@ -447,6 +447,7 @@ .classpath .project **/dependency-reduced-pom.xml + **/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister **/target/** **/README.md **/examples/data/*.txt From af805e3226cbc31b2a9993aa635795a3d1fdd8c7 Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Tue, 15 Nov 2016 17:27:08 -0800 Subject: [PATCH 03/24] [BAHIR-75] - minor README fixes --- datasource-webhdfs/README.md | 35 ++++++++++++++++++++++------------- 1 file changed, 22 insertions(+), 13 deletions(-) diff --git a/datasource-webhdfs/README.md b/datasource-webhdfs/README.md index 18b49750..614df8b4 100644 --- a/datasource-webhdfs/README.md +++ b/datasource-webhdfs/README.md @@ -1,4 +1,4 @@ -A custom datasource for reading data from remote hdfs using webhdfs protocol. +A custom data source to read and write data from and to remote HDFS clusters using the [WebHDFS](https://hadoop.apache.org/docs/r2.7.3/hadoop-project-dist/hadoop-hdfs/WebHDFS.html) protocol. ## Linking @@ -8,11 +8,13 @@ Using SBT: Using Maven: - - org.apache.bahir - spark-datasource-webhdfs_2.11 - 2.1.0-SNAPSHOT - +```xml + + org.apache.bahir + spark-datasource-webhdfs_2.11 + 2.1.0-SNAPSHOT + +``` This library can also be added to Spark jobs launched through `spark-shell` or `spark-submit` by using the `--packages` command line option. For example, to include it when starting the spark shell: @@ -28,15 +30,22 @@ This library is compiled for Scala 2.11 only, and intends to support Spark 2.0 o A data frame can be created using this custom data source as shown below - - val filePath = s"webhdfs:///gateway/default/webhdfs/v1/" - - val df = spark.read.format("webhdfs").option("certValidation", "Y").option("userCred", "user1:pass1").option("header", "true").option("partitions", "8").load(filePath) +```scala +val filePath = s"webhdfs:///gateway/default/webhdfs/v1/" +val df = spark.read + .format("webhdfs") + .option("certValidation", "Y") + .option("userCred", "user1:pass1") + .option("header", "true") + .option("partitions", "8") + .load(filePath) +``` ## Configuration options. - * `certValidation` Set this to 'Y' or 'N'. In case of 'N' this component will ignore validation of teh SSL certification. Otherwise it will dowload the certificate and validate. - * `userCred` Set this to 'userid:password' as needed by the remote hdfs for accessing a file from there. - * `partitions` This number tells the Data Source how many parallel connections to be opened to read data from HDFS in the remote cluster for each file. If this option is not specified default value is used which is 4. Recommended value for this option is same as the next nearest integer of (file size/block size) in hdfs or multiple of that. For example if file size in hdfs is 0.95 GB and block size of the file is 128 MB use 8 or multiple of 8 as number of partitions. However, number of partitions should not be more than (or may be little more than) maximum number of parallel tasks possible to spawn in your spark cluster. + * `certValidation` Set this to `'Y'` or `'N'`. In case of `'N'` this component will ignore validation of teh SSL certification. Otherwise it will download the certificate and validate. + * `userCred` Set this to `'userid:password'` as needed by the remote HDFS for accessing a file from there. + * `partitions` This number tells the Data Source how many parallel connections to be opened to read data from HDFS in the remote cluster for each file. If this option is not specified default value is used which is 4. Recommended value for this option is same as the next nearest integer of (file size/block size) in HDFS or multiples of that. For example if file size in HDFS is 0.95 GB and block size of the file is 128 MB use 8 or multiple of 8 as number of partitions. However, number of partitions should not be more than (or may be little more than) maximum number of parallel tasks possible to spawn in your Spark cluster. * `format` Format of the file. Right now only 'csv' is supported. If this option is not specified by default 'csv' is assumed. - * `output` Specify either 'LIST' or 'Data'. By default, 'Data' is assumed which returns the actual data in the file. If a folder name is specified then data from all files in that folder would be fethed at once. If 'LIST' is specified then the files within the folder is listed. + * `output` Specify either `'LIST'` or `'Data'`. By default, `'Data'` is assumed which returns the actual data in the file. If a folder name is specified then data from all files in that folder would be fetched at once. If `'LIST'` is specified then the files within the folder is listed. From 78ff29c8885534935d43a911af9c27f667725989 Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Tue, 15 Nov 2016 17:38:04 -0800 Subject: [PATCH 04/24] [BAHIR-75] - minor README fixes (2) --- datasource-webhdfs/README.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/datasource-webhdfs/README.md b/datasource-webhdfs/README.md index 614df8b4..f8d19937 100644 --- a/datasource-webhdfs/README.md +++ b/datasource-webhdfs/README.md @@ -4,7 +4,9 @@ A custom data source to read and write data from and to remote HDFS clusters usi Using SBT: - libraryDependencies += "org.apache.bahir" %% "spark-datasource-webhdfs" % "2.1.0-SNAPSHOT" +```scala +libraryDependencies += "org.apache.bahir" %% "spark-datasource-webhdfs" % "2.1.0-SNAPSHOT" +``` Using Maven: From 24e79c9743f46de296941a423dd94772174495bd Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Tue, 15 Nov 2016 19:16:49 -0800 Subject: [PATCH 05/24] [BAHIR-75] - include DataSourceRegister in Maven build --- datasource-webhdfs/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/datasource-webhdfs/pom.xml b/datasource-webhdfs/pom.xml index 22b10501..cad5a04f 100755 --- a/datasource-webhdfs/pom.xml +++ b/datasource-webhdfs/pom.xml @@ -68,6 +68,11 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + ${basedir}/src/main/resources + + org.apache.maven.plugins From 365ee1f702dc42e50c2e922e9c069f80f2d016a2 Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Wed, 16 Nov 2016 10:37:46 -0800 Subject: [PATCH 06/24] [BAHIR-75] - fix package declaration in webhdfs package object --- .../org/apache/bahir/datasource/webhdfs/package.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/package.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/package.scala index f03640af..e7ec7a28 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/package.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/package.scala @@ -15,10 +15,11 @@ * limitations under the License. */ +package org.apache.bahir.datasource -package org.apache.bahir.datasource.webhdfs - - +/* + * WebHDFS data source for Apache Spark + */ package object webhdfs { } From d4c6e56db57a9cd13b0a25dc77cc3c4179c9a7b7 Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Wed, 16 Nov 2016 14:46:39 -0800 Subject: [PATCH 07/24] [BAHIR-75] - fix 798 Scalastyle violations --- .../datasource/webhdfs/DefaultSource.scala | 417 ++++----- .../webhdfs/csv/WebHdfsCsvFormatter.scala | 232 +++-- .../webhdfs/csv/WebHdfsCsvParserLibs.scala | 9 +- .../webhdfs/csv/WebHdfsCsvRelation.scala | 36 +- .../webhdfs/util/SSLTrustStoreUtil.scala | 76 +- .../webhdfs/util/WebHdfsConnector.scala | 873 +++++++++--------- .../webhdfs/util/WebHdfsInferSchema.scala | 108 ++- .../webhdfs/util/WebHdfsParseModes.scala | 23 +- .../webhdfs/util/WebHdfsTypeCast.scala | 27 +- 9 files changed, 889 insertions(+), 912 deletions(-) diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/DefaultSource.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/DefaultSource.scala index db14a78e..d6caa269 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/DefaultSource.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/DefaultSource.scala @@ -17,249 +17,232 @@ package org.apache.bahir.datasource.webhdfs -import java.text.SimpleDateFormat -import java.sql.{Timestamp, Date} - -import scala.collection.mutable.HashMap +import scala.annotation.switch -import org.apache.spark.sql.types.{DateType, TimestampType} +import org.apache.bahir.datasource.webhdfs.csv._ +import org.apache.bahir.datasource.webhdfs.util._ -import org.apache.spark.sql.types._ import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext} - import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.StructType -import scala.annotation.switch +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructField, StructType} -import org.apache.bahir.datasource.webhdfs.util._ -import org.apache.bahir.datasource.webhdfs.csv._ /** - * This class contains functions for reading/writing data from/to remote webhdfs server in Spark DataSource - * This function is written in line with the DataSource function in com.databricks.spark.csv. -*/ - - + * A Spark data source to read/write data from/to remote WebHDFS servers. + * This function is written in line with the DataSource function in com.databricks.spark.csv + */ class DefaultSource extends RelationProvider - with SchemaRelationProvider - with CreatableRelationProvider - with DataSourceRegister { + with SchemaRelationProvider + with CreatableRelationProvider + with DataSourceRegister { override def shortName() : String = "webhdfs" private def checkPath(parameters: Map[String, String]): String = { - parameters.getOrElse("path", sys.error("'path' must be specified ")) + parameters.getOrElse("path", sys.error("'path' must be specified ")) } /** * Creates a new relation for data store in CSV given parameters. * Parameters have to include 'path' and optionally 'delimiter', 'quote', and 'header' */ - override def createRelation( - sqlContext: SQLContext, - parameters: Map[String, String]): BaseRelation = { - createRelation(sqlContext, parameters, null) + override def createRelation(sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + createRelation(sqlContext, parameters, null) } /** * Creates a new relation for data store in CSV given parameters and user supported schema. * Parameters have to include 'path' and optionally 'delimiter', 'quote', and 'header' */ - override def createRelation( - sqlContext: SQLContext, - parameters: Map[String, String], - schema: StructType): WebHdfsCsvRelation = { - //print("In Create Relation of DefaultSource" + "\n") - val path = checkPath(parameters) - val delimiter = WebHdfsTypeCast.toChar(parameters.getOrElse("delimiter", ",")) - - val quote = parameters.getOrElse("quote", "\"") - val quoteChar: Character = if (quote == null) { - null - } else if (quote.length == 1) { - quote.charAt(0) - } else { - throw new Exception("Quotation cannot be more than one character.") - } - - val escape = parameters.getOrElse("escape", null) - val escapeChar: Character = if (escape == null) { - null - } else if (escape.length == 1) { - escape.charAt(0) - } else { - throw new Exception("Escape character cannot be more than one character.") - } - - val comment = parameters.getOrElse("comment", "#") - val commentChar: Character = if (comment == null) { - null - } else if (comment.length == 1) { - comment.charAt(0) - } else { - throw new Exception("Comment marker cannot be more than one character.") - } - - val parseMode = parameters.getOrElse("mode", "PERMISSIVE") - - val useHeader = parameters.getOrElse("header", "false") - val headerFlag = if (useHeader == "true") { - true - } else if (useHeader == "false") { - false - } else { - throw new Exception("Header flag can be true or false") - } - - // val parserLib = parameters.getOrElse("parserLib", ParserLibs.DEFAULT) - val parserLib = parameters.getOrElse("parserLib", "COMMONS") - val ignoreLeadingWhiteSpace = parameters.getOrElse("ignoreLeadingWhiteSpace", "false") - val ignoreLeadingWhiteSpaceFlag = if (ignoreLeadingWhiteSpace == "false") { - false - } else if (ignoreLeadingWhiteSpace == "true") { - //if (!ParserLibs.isUnivocityLib(parserLib)) { - // throw new Exception("Ignore white space supported for Univocity parser only") - //} - true - } else { - throw new Exception("Ignore white space flag can be true or false") - } - - val ignoreTrailingWhiteSpace = parameters.getOrElse("ignoreTrailingWhiteSpace", "false") - val ignoreTrailingWhiteSpaceFlag = if (ignoreTrailingWhiteSpace == "false") { - false - } else if (ignoreTrailingWhiteSpace == "true") { - //if (!ParserLibs.isUnivocityLib(parserLib)) { - // throw new Exception("Ignore white space supported for the Univocity parser only") - //} - true - } else { - throw new Exception("Ignore white space flag can be true or false") - } - val treatEmptyValuesAsNulls = parameters.getOrElse("treatEmptyValuesAsNulls", "false") - val treatEmptyValuesAsNullsFlag = if (treatEmptyValuesAsNulls == "false") { - false - } else if (treatEmptyValuesAsNulls == "true") { - true - } else { - throw new Exception("Treat empty values as null flag can be true or false") - } - - val charset = parameters.getOrElse("charset", WebHdfsConnector.DEFAULT_CHARSET.name()) - // TODO validate charset? - - val inferSchema = parameters.getOrElse("inferSchema", "false") - val inferSchemaFlag = if (inferSchema == "false") { - false - } else if (inferSchema == "true") { - true - } else { - throw new Exception("Infer schema flag can be true or false") - } - val nullValue = parameters.getOrElse("nullValue", "") - - val dateFormat = parameters.getOrElse("dateFormat", null) - - val codec = parameters.getOrElse("codec", null) - - val maxCharsPerColStr = parameters.getOrElse("maxCharsPerCol", "100000") - val maxCharsPerCol = try { - maxCharsPerColStr.toInt - } catch { - case e: Exception => throw new Exception("maxCharsPerCol must be a valid integer") - } - - val trustCredStr = parameters.getOrElse("certValidation", "") - val userCredStr = parameters.getOrElse("userCred", "") - if(userCredStr == "") - throw new Exception("User Credential has to be set") - - val connPropStr = parameters.getOrElse("connProp", "10000:60000") - var partitionDetailsStr = parameters.getOrElse("partitions", "4:10000") - - if(!partitionDetailsStr.contains(":")) partitionDetailsStr = partitionDetailsStr + ":10000" - - val formatDetailsStr = parameters.getOrElse("format", "csv:\n") - val formatDetailsArr = formatDetailsStr.split(":") - val formatType = formatDetailsArr(0) - var formatRecordSeparator = formatDetailsArr(1) - if(formatType != "csv" && formatRecordSeparator == "") - throw new Exception("Record Separator cannot be inferred for Format other than csv") - if(formatType == "csv" && formatRecordSeparator == "") formatRecordSeparator = "\n" - - val outputTypeStr = parameters.getOrElse("output", "Data") - - val outRdd = (outputTypeStr : @switch) match { - case "LIST" => WebHdfsConnector.listFromWebHdfs(sqlContext.sparkContext, path, trustCredStr, userCredStr, connPropStr) - case default => WebHdfsConnector.loadFromWebHdfs(sqlContext.sparkContext, path, charset, trustCredStr, userCredStr, connPropStr, partitionDetailsStr, formatRecordSeparator) - } - - val targetSchema = (outputTypeStr : @switch) match { - case "LIST" => StructType(Array(StructField("Name",StringType,true),StructField("File Size",LongType,true),StructField("Block Size",LongType,true), StructField("# of Blocks", IntegerType, true))) - case default => schema - } - - - val relation = (formatType : @switch) match { - case "csv" => - WebHdfsCsvRelation( - () => outRdd, - Some(path), - headerFlag, - delimiter, - quoteChar, - escapeChar, - commentChar, - parseMode, - parserLib, - ignoreLeadingWhiteSpaceFlag, - ignoreTrailingWhiteSpaceFlag, - treatEmptyValuesAsNullsFlag, - targetSchema, - inferSchemaFlag, - codec, - nullValue, - dateFormat, - maxCharsPerCol)(sqlContext) - case default => throw new Exception("Format Not Supported") - } - - relation + override def createRelation(sqlContext: SQLContext, + parameters: Map[String, String], + schema: StructType): WebHdfsCsvRelation = { + // print("In Create Relation of DefaultSource" + "\n") + val path = checkPath(parameters) + val delimiter = WebHdfsTypeCast.toChar(parameters.getOrElse("delimiter", ",")) + + val quote = parameters.getOrElse("quote", "\"") + val quoteChar: Character = if (quote == null) { + null + } else if (quote.length == 1) { + quote.charAt(0) + } else { + throw new Exception("Quotation cannot be more than one character.") + } + + val escape = parameters.getOrElse("escape", null) + val escapeChar: Character = if (escape == null) { + null + } else if (escape.length == 1) { + escape.charAt(0) + } else { + throw new Exception("Escape character cannot be more than one character.") + } + + val comment = parameters.getOrElse("comment", "#") + val commentChar: Character = if (comment == null) { + null + } else if (comment.length == 1) { + comment.charAt(0) + } else { + throw new Exception("Comment marker cannot be more than one character.") + } + + val parseMode = parameters.getOrElse("mode", "PERMISSIVE") + + val useHeader = parameters.getOrElse("header", "false") + val headerFlag = if (useHeader == "true") { + true + } else if (useHeader == "false") { + false + } else { + throw new Exception("Header flag must be true or false") + } + + // val parserLib = parameters.getOrElse("parserLib", ParserLibs.DEFAULT) + val parserLib = parameters.getOrElse("parserLib", "COMMONS") + val ignoreLeadingWhiteSpace = parameters.getOrElse("ignoreLeadingWhiteSpace", "false") + val ignoreLeadingWhiteSpaceFlag = if (ignoreLeadingWhiteSpace == "false") { + false + } else if (ignoreLeadingWhiteSpace == "true") { + // if (!ParserLibs.isUnivocityLib(parserLib)) { + // throw new Exception("Ignore white space supported for Univocity parser only") + // } + true + } else { + throw new Exception("Ignore white space flag must be true or false") + } + + val ignoreTrailingWhiteSpace = parameters.getOrElse("ignoreTrailingWhiteSpace", "false") + val ignoreTrailingWhiteSpaceFlag = if (ignoreTrailingWhiteSpace == "false") { + false + } else if (ignoreTrailingWhiteSpace == "true") { + // if (!ParserLibs.isUnivocityLib(parserLib)) { + // throw new Exception("Ignore white space supported for the Univocity parser only") + // } + true + } else { + throw new Exception("Ignore white space flag can be true or false") + } + val treatEmptyValuesAsNulls = parameters.getOrElse("treatEmptyValuesAsNulls", "false") + val treatEmptyValuesAsNullsFlag = if (treatEmptyValuesAsNulls == "false") { + false + } else if (treatEmptyValuesAsNulls == "true") { + true + } else { + throw new Exception("Treat empty values as null flag can be true or false") + } + + val charset = parameters.getOrElse("charset", WebHdfsConnector.DEFAULT_CHARSET.name()) + // TODO validate charset? + + val inferSchema = parameters.getOrElse("inferSchema", "false") + val inferSchemaFlag = if (inferSchema == "false") { + false + } else if (inferSchema == "true") { + true + } else { + throw new Exception("Infer schema flag can be true or false") + } + val nullValue = parameters.getOrElse("nullValue", "") + val dateFormat = parameters.getOrElse("dateFormat", null) + val codec = parameters.getOrElse("codec", null) + val maxCharsPerColStr = parameters.getOrElse("maxCharsPerCol", "100000") + val maxCharsPerCol = try { + maxCharsPerColStr.toInt + } catch { + case e: Exception => throw new Exception("maxCharsPerCol must be a valid integer") + } + + val trustCredStr = parameters.getOrElse("certValidation", "") + val userCredStr = parameters.getOrElse("userCred", "") + if (userCredStr == "") { + throw new Exception("User Credential has to be set") + } + + val connPropStr = parameters.getOrElse("connProp", "10000:60000") + var partitionDetailsStr = parameters.getOrElse("partitions", "4:10000") + + if (!partitionDetailsStr.contains(":")) partitionDetailsStr = partitionDetailsStr + ":10000" + + val formatDetailsStr = parameters.getOrElse("format", "csv:\n") + val formatDetailsArr = formatDetailsStr.split(":") + var formatRecordSeparator = formatDetailsArr(1) + val outputTypeStr = parameters.getOrElse("output", "Data") + val formatType = formatDetailsArr(0) + if (formatType != "csv" && formatRecordSeparator == "") { + throw new Exception("Record Separator cannot be inferred for Format other than csv") + } + if (formatType == "csv" && formatRecordSeparator == "") formatRecordSeparator = "\n" + + val outRdd = (outputTypeStr : @switch) match { + case "LIST" => WebHdfsConnector.listFromWebHdfs(sqlContext.sparkContext, path, trustCredStr, + userCredStr, connPropStr) + case default => WebHdfsConnector.loadFromWebHdfs(sqlContext.sparkContext, path, charset, + trustCredStr, userCredStr, connPropStr, partitionDetailsStr, formatRecordSeparator) + } + + val targetSchema = (outputTypeStr : @switch) match { + case "LIST" => StructType(Array( + StructField("Name", StringType, true), + StructField("File Size", LongType, true), + StructField("Block Size", LongType, true), + StructField("# of Blocks", IntegerType, true))) + case default => schema + } + + val relation = (formatType : @switch) match { + case "csv" => + WebHdfsCsvRelation( + () => outRdd, + Some(path), + headerFlag, + delimiter, + quoteChar, + escapeChar, + commentChar, + parseMode, + parserLib, + ignoreLeadingWhiteSpaceFlag, + ignoreTrailingWhiteSpaceFlag, + treatEmptyValuesAsNullsFlag, + targetSchema, + inferSchemaFlag, + codec, + nullValue, + dateFormat, + maxCharsPerCol)(sqlContext) + case default => throw new Exception("Format Not Supported") + } + + relation } - override def createRelation( - sqlContext: SQLContext, - mode: SaveMode, - parameters: Map[String, String], - data: DataFrame): BaseRelation = { - - val path = checkPath(parameters) - - val trustStoreCredStr = parameters.getOrElse("certValidation", "") - - val connStr = parameters.getOrElse("connProp", "1000:10000") - - val userCredStr = parameters.getOrElse("userCred", "") - if (userCredStr == "") - throw new Exception("User Credentials cannot be null") - - val partitionStr = parameters.getOrElse("partitions", "4") - - val formatStr = parameters.getOrElse("format", "csv") - - /* - * Writre now only CSV format is supported - */ - - val rddToWrite = (formatStr : @switch) match { - case "csv" => WebHdfsCsvFormatter.convToCsvFormat(data, parameters) - case default => throw new Exception("Format Not Supported") - } - - WebHdfsConnector.writeToWebHdfs(rddToWrite, path, trustStoreCredStr, connStr, userCredStr, partitionStr) - - createRelation(sqlContext, parameters, data.schema) + override def createRelation(sqlContext: SQLContext, + mode: SaveMode, + parameters: Map[String, String], + data: DataFrame): BaseRelation = { + val path = checkPath(parameters) + val trustStoreCredStr = parameters.getOrElse("certValidation", "") + val connStr = parameters.getOrElse("connProp", "1000:10000") + val partitionStr = parameters.getOrElse("partitions", "4") + val formatStr = parameters.getOrElse("format", "csv") + val userCredStr = parameters.getOrElse("userCred", "") + if (userCredStr == "") { + throw new Exception("User Credentials cannot be null") + } + + // As of now only CSV format is supported + val rddToWrite = (formatStr : @switch) match { + case "csv" => WebHdfsCsvFormatter.convToCsvFormat(data, parameters) + case default => throw new Exception("Format Not Supported") + } + + WebHdfsConnector + .writeToWebHdfs(rddToWrite, path, trustStoreCredStr, connStr, userCredStr, partitionStr) + + createRelation(sqlContext, parameters, data.schema) } - } diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvFormatter.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvFormatter.scala index 49cfcf44..60fad882 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvFormatter.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvFormatter.scala @@ -15,136 +15,128 @@ * limitations under the License. */ - package org.apache.bahir.datasource.webhdfs.csv -import java.io.IOException +import java.sql.Timestamp import java.text.SimpleDateFormat -import java.sql.{Timestamp, Date} - -import scala.collection.JavaConversions._ -import scala.util.control.NonFatal import org.apache.commons.csv.{CSVFormat, QuoteMode} -import org.apache.commons.csv._ -import org.slf4j.LoggerFactory import org.apache.spark.rdd.RDD import org.apache.spark.sql._ -import org.apache.spark.sql.sources.{PrunedScan, BaseRelation, InsertableRelation, TableScan} import org.apache.spark.sql.types._ /** - * This object contains all utility functions for converting data to CSV format - * This is copied from com.databricks.spark.csv as the required object could not be reused as it is declared as private -*/ - -private[webhdfs] object WebHdfsCsvFormatter { - - def convToCsvFormat(dataFrame: DataFrame, parameters: Map[String, String] = Map()) : RDD[String] = { - - val delimiter = parameters.getOrElse("delimiter", ",") - val dateFormat = parameters.getOrElse("dateFormat", "yyyy-MM-dd HH:mm:ss.S") - val dateFormatter: SimpleDateFormat = new SimpleDateFormat(dateFormat) - - val delimiterChar = if (delimiter.length == 1) { - delimiter.charAt(0) - } else { - throw new Exception("Delimiter cannot be more than one character.") - } - - val escape = parameters.getOrElse("escape", null) - val escapeChar: Character = if (escape == null) { - null - } else if (escape.length == 1) { - escape.charAt(0) - } else { - throw new Exception("Escape character cannot be more than one character.") - } - - val quote = parameters.getOrElse("quote", "\"") - val quoteChar: Character = if (quote == null) { - null - } else if (quote.length == 1) { - quote.charAt(0) - } else { - throw new Exception("Quotation cannot be more than one character.") - } - - val quoteModeString = parameters.getOrElse("quoteMode", "MINIMAL") - val quoteMode: QuoteMode = if (quoteModeString == null) { - null - } else { - QuoteMode.valueOf(quoteModeString.toUpperCase) - } - - val nullValue = parameters.getOrElse("nullValue", "null") - - val csvFormat = CSVFormat.DEFAULT - .withDelimiter(delimiterChar) - .withQuote(quoteChar) - .withEscape(escapeChar) - .withQuoteMode(quoteMode) - .withSkipHeaderRecord(false) - .withNullString(nullValue) - - val generateHeader = parameters.getOrElse("header", "false").toBoolean - val header = if (generateHeader) { - csvFormat.format(dataFrame.columns.map(_.asInstanceOf[AnyRef]): _*) - } else { - "" // There is no need to generate header in this case - } - - val schema = dataFrame.schema - val formatForIdx = schema.fieldNames.map(fname => schema(fname).dataType match { - case TimestampType => (timestamp: Any) => { - if (timestamp == null) { - nullValue - } else { - dateFormatter.format(new java.sql.Date(timestamp.asInstanceOf[Timestamp].getTime)) - } - } - case DateType => (date: Any) => { - if (date == null) nullValue else dateFormatter.format(date) - } - case _ => (fieldValue: Any) => fieldValue.asInstanceOf[AnyRef] - }) - - val strRDD = dataFrame.rdd.mapPartitionsWithIndex { case (index, iter) => - val csvFormat = CSVFormat.DEFAULT - .withDelimiter(delimiterChar) - .withQuote(quoteChar) - .withEscape(escapeChar) - .withQuoteMode(quoteMode) - .withSkipHeaderRecord(false) - .withNullString(nullValue) - - new Iterator[String] { - var firstRow: Boolean = generateHeader - - override def hasNext: Boolean = iter.hasNext || firstRow - - override def next: String = { - if (iter.nonEmpty) { - // try .zipWithIndex.foreach - val values: Seq[AnyRef] = iter.next().toSeq.zipWithIndex.map { - case (fieldVal, i) => formatForIdx(i)(fieldVal) - } - val row = csvFormat.format(values: _*) - if (firstRow) { - firstRow = false - header + "\n" + row - } else { - row - } - } else { - firstRow = false - header - } - } - } - } - strRDD - } + * This object contains all utility functions for converting data to CSV format + * This is copied from com.databricks.spark.csv as the required object could not be reused as it is + * declared as private + */ +private[webhdfs] object WebHdfsCsvFormatter { + + def convToCsvFormat(dataFrame: DataFrame, + parameters: Map[String, String] = Map()) : RDD[String] = { + val delimiter = parameters.getOrElse("delimiter", ",") + val dateFormat = parameters.getOrElse("dateFormat", "yyyy-MM-dd HH:mm:ss.S") + val dateFormatter: SimpleDateFormat = new SimpleDateFormat(dateFormat) + + val delimiterChar = if (delimiter.length == 1) { + delimiter.charAt(0) + } else { + throw new Exception("Delimiter cannot be more than one character.") + } + + val escape = parameters.getOrElse("escape", null) + val escapeChar: Character = if (escape == null) { + null + } else if (escape.length == 1) { + escape.charAt(0) + } else { + throw new Exception("Escape character cannot be more than one character.") + } + + val quote = parameters.getOrElse("quote", "\"") + val quoteChar: Character = if (quote == null) { + null + } else if (quote.length == 1) { + quote.charAt(0) + } else { + throw new Exception("Quotation cannot be more than one character.") + } + + val quoteModeString = parameters.getOrElse("quoteMode", "MINIMAL") + val quoteMode: QuoteMode = if (quoteModeString == null) { + null + } else { + QuoteMode.valueOf(quoteModeString.toUpperCase) + } + + val nullValue = parameters.getOrElse("nullValue", "null") + + val csvFormat = CSVFormat.DEFAULT + .withDelimiter(delimiterChar) + .withQuote(quoteChar) + .withEscape(escapeChar) + .withQuoteMode(quoteMode) + .withSkipHeaderRecord(false) + .withNullString(nullValue) + + val generateHeader = parameters.getOrElse("header", "false").toBoolean + val header = if (generateHeader) { + csvFormat.format(dataFrame.columns.map(_.asInstanceOf[AnyRef]): _*) + } else { + "" // There is no need to generate header in this case + } + + val schema = dataFrame.schema + val formatForIdx = schema.fieldNames.map(fname => schema(fname).dataType match { + case TimestampType => (timestamp: Any) => { + if (timestamp == null) { + nullValue + } else { + dateFormatter.format(new java.sql.Date(timestamp.asInstanceOf[Timestamp].getTime)) + } + } + case DateType => (date: Any) => { + if (date == null) nullValue else dateFormatter.format(date) + } + case _ => (fieldValue: Any) => fieldValue.asInstanceOf[AnyRef] + }) + + val strRDD = dataFrame.rdd.mapPartitionsWithIndex { case (index, iter) => + val csvFormat = CSVFormat.DEFAULT + .withDelimiter(delimiterChar) + .withQuote(quoteChar) + .withEscape(escapeChar) + .withQuoteMode(quoteMode) + .withSkipHeaderRecord(false) + .withNullString(nullValue) + + new Iterator[String] { + var firstRow: Boolean = generateHeader + + override def hasNext: Boolean = iter.hasNext || firstRow + + override def next: String = { + if (iter.nonEmpty) { + // try .zipWithIndex.foreach + val values: Seq[AnyRef] = iter.next().toSeq.zipWithIndex.map { + case (fieldVal, i) => formatForIdx(i)(fieldVal) + } + val row = csvFormat.format(values: _*) + if (firstRow) { + firstRow = false + header + "\n" + row + } else { + row + } + } else { + firstRow = false + header + } + } + } + } + strRDD + } } diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvParserLibs.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvParserLibs.scala index 71798805..5c873404 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvParserLibs.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvParserLibs.scala @@ -19,14 +19,13 @@ package org.apache.bahir.datasource.webhdfs.csv /** - * This object contains all utility functions needed for determining CSV parser lib - * This is copied from com.databricks.spark.csv.util as the required object could not be reused as it is declared as private -*/ - + * This object contains all utility functions needed for determining CSV parser lib + * This is copied from com.databricks.spark.csv.util as the required object could not be reused as + * it is declared as private + */ private[webhdfs] object WebHdfsCsvParserLibs { val OLD = "COMMONS" val NEW = "UNIVOCITY" - val DEFAULT = OLD def isValidLib(lib: String): Boolean = { diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvRelation.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvRelation.scala index 28a83c9f..07e80d0a 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvRelation.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvRelation.scala @@ -15,34 +15,31 @@ * limitations under the License. */ - - package org.apache.bahir.datasource.webhdfs.csv -import java.io.IOException + import java.text.SimpleDateFormat +// TODO: use scala.collection.JavaConverters instead of implicit JavaConversions +// scalastyle:off javaconversions import scala.collection.JavaConversions._ +// scalastyle:on import scala.util.control.NonFatal -import org.apache.commons.csv.{CSVFormat, QuoteMode} -import org.apache.commons.csv._ +import org.apache.bahir.datasource.webhdfs.util._ +import org.apache.commons.csv.{CSVFormat, CSVParser} import org.slf4j.LoggerFactory import org.apache.spark.rdd.RDD -import org.apache.spark.sql._ -import org.apache.spark.sql.sources.{PrunedScan, BaseRelation, InsertableRelation, TableScan} -import org.apache.spark.sql.types._ - - -import org.apache.bahir.datasource.webhdfs.util._ - -/* - * This class contains functions for converting RDD to csv data source - * This is copied from com.databricks.spark.csv as the required object could not be reused as it is declared as private -*/ - - +import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.sources.{BaseRelation, PrunedScan, TableScan} +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +/** + * This class contains functions for converting RDD to csv data source + * This is copied from com.databricks.spark.csv as the required object could not be reused as it is + * declared as private + */ case class WebHdfsCsvRelation protected[webhdfs] ( baseRDD: () => RDD[String], location: Option[String], @@ -74,7 +71,8 @@ case class WebHdfsCsvRelation protected[webhdfs] ( logger.warn(s"$parseMode is not a valid parse mode. Using ${WebHdfsParseModes.DEFAULT}.") } - if ((ignoreLeadingWhiteSpace || ignoreLeadingWhiteSpace) && WebHdfsCsvParserLibs.isCommonsLib(parserLib)) { + if ((ignoreLeadingWhiteSpace || ignoreLeadingWhiteSpace) + && WebHdfsCsvParserLibs.isCommonsLib(parserLib)) { logger.warn(s"Ignore white space options may not work with Commons parserLib option") } diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala index 521c02e0..1ab14176 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala @@ -17,60 +17,54 @@ package org.apache.bahir.datasource.webhdfs.util -import java.security._ -import javax.net.ssl.TrustManagerFactory import java.io._ -import javax.net.ssl.SSLSocketFactory -import javax.net.ssl.SSLContext + import scala.collection.mutable.HashMap -import sys.process._ +import scala.sys.process._ /** - * This Singleton is used to generate SSLTrustStore certification once. The assumption behind use of this trust store is that this code would be executed on a machine which would be accessible from all Spark executors which need to access the trust store -**/ - -object SSLTrustStore{ - - var trustStoreFileMap : HashMap[String, File] = HashMap() - - /** - * This function checks the availability of truststore for a particular site. If not it creates a new one. - **/ + * This Singleton is used to generate SSLTrustStore certification once. The assumption behind use of + * this trust store is that this code would be executed on a machine which would be accessible from + * all Spark executors which need to access the trust store + */ +object SSLTrustStore { + var trustStoreFileMap: HashMap[String, File] = HashMap() - def getCertDetails(path: String): Tuple2[File, String] = { + /** + * This function checks the availability of truststore for a particular site. If not it creates + * a new one. + */ + def getCertDetails(path: String): Tuple2[File, String] = { - val pathComp = path.split("/") + val pathComp = path.split("/") - val srvr = pathComp(2) + val srvr = pathComp(2) - val trustStorePword = "ts-password" + val trustStorePword = "ts-password" - val currDir = ("pwd" !!).trim - val trustStore = currDir + "/" + srvr + "_trustStore.jks" - - val os = new java.io.ByteArrayOutputStream + val currDir = ("pwd" !!).trim + val trustStore = currDir + "/" + srvr + "_trustStore.jks" - val tsExist = (s"ls $trustStore" #> os ).! - - val f = if (tsExist == 0) - { - - print("Using Existing Trust Store for SSL" + "\n") - new java.io.File(trustStore) - } - else { - val cert = srvr + "_cert" - val cfl = new File(cert) + val os = new java.io.ByteArrayOutputStream - (s"openssl s_client -showcerts -connect $srvr" #| "openssl x509 -outform PEM" #> cfl).! - (s"keytool -import -trustcacerts -alias hadoop -file $cert -keystore $trustStore -storepass $trustStorePword -noprompt").! - (s"rm -f $cert").! - new java.io.File(trustStore) - } + val tsExist = (s"ls $trustStore" #> os).! - new Tuple2(f, trustStorePword) + val f = if (tsExist == 0) { + print("Using Existing Trust Store for SSL" + "\n") + new java.io.File(trustStore) + } + else { + val cert = srvr + "_cert" + val cfl = new File(cert) - } + (s"openssl s_client -showcerts -connect $srvr" #| "openssl x509 -outform PEM" #> cfl).! + (s"keytool -import -trustcacerts -alias hadoop -file $cert -keystore $trustStore" + + s" -storepass $trustStorePword -noprompt").! + (s"rm -f $cert").! + new java.io.File(trustStore) + } + new Tuple2(f, trustStorePword) + } } diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala index 8f5a56de..3cb62ae6 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala @@ -17,455 +17,472 @@ package org.apache.bahir.datasource.webhdfs.util +import java.io._ import java.nio.charset.Charset -import java.text.SimpleDateFormat -import java.sql.{Timestamp, Date} -import java.util.Date - -import org.apache.spark.sql.types.{DateType, TimestampType} - -import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD +import java.security._ +import javax.net.ssl.{SSLContext, SSLSocketFactory, TrustManagerFactory} import scala.annotation.switch -import scalaj.http._ -import java.security._ -import javax.net.ssl.TrustManagerFactory -import java.io._ -import javax.net.ssl.SSLSocketFactory -import javax.net.ssl.SSLContext -import org.apache.http.conn.scheme.Scheme -import java.net.HttpURLConnection -import scala.collection.mutable.HashMap -import scala.math._ -import org.apache.spark.sql.types.{DateType, TimestampType} -import org.apache.spark.sql.{DataFrame, SQLContext} import scala.collection.mutable.ArrayBuffer -import scala.util.control.Breaks._ - - -/** - * This object contains all utility functions for reading/writing data from/to remote webhdfs server. The abstraction maintained in this layer is at the level of RDD -*/ - -private[webhdfs] object WebHdfsConnector { - - - /** - * Currently only files transferred using UTF-8 are supported - - */ - - val DEFAULT_CHARSET = Charset.forName("UTF-8") - - /** - * This function returns a Tuple for credential store which contains flag for validating Certificate, the Certificate File object and Certificate File Object password - - */ - - def createTrustStoreCredForExecutors(cred: String, path: String) : Tuple3[String, File, String] = { - - val trustStoreMap = if (cred != "") { - if (cred == "N") - new Tuple3("N", null, "") - else if(cred == "Y") - { - val tsd = SSLTrustStore.getCertDetails(path) - - new Tuple3("Y", tsd._1, tsd._2) - } - else - throw new Exception("Invalid Certificate Validation Option") - - } else { - new Tuple3("", null, "") - } - trustStoreMap - - - } - - - /** - * This function returns a SSLSocketFactory which needs to be used in HTTP connection library in case Certificate to be validated - */ - - def biocSslSocketFactory(fl: File, pswrd: String): SSLSocketFactory = { - - - val tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()) - val ks = KeyStore.getInstance("JKS") - val fis = new java.io.FileInputStream(fl) - ks.load(fis, pswrd.toCharArray()); - tmf.init(ks); - - - val sslc = SSLContext.getInstance("SSL") - - sslc.init(null, tmf.getTrustManagers(),null) - - sslc.getSocketFactory() - } - - - /** - * This function returns the details of the the files in a folder if the path passed is a folder. In case a File path is passed it returns the details of teh files. - * This returns an Array of Tuple where each Tuple represents one file with details of full file path, - * size of the file, block size of the file and number of partitions based on size of the file and blick size - **/ - - def getFilesDetails(path: String, trustCred: Tuple3[String, File, String], usrCred: Array[String], connProp: Array[Int]): Array [Tuple4[String, Long, Long, Int]] = { - - - val listStatusOpr = s"op=LISTSTATUS" - - val listStatus = callWebHdfsAPI(path, "", "GET", "BODY", trustCred, usrCred, connProp, listStatusOpr) - - if (listStatus.contains("RemoteException")) - throw new Exception(listStatus) - - val flist = scala.util.parsing.json.JSON.parseFull(listStatus).toList(0).asInstanceOf[Map[String, Map[String, Any]]].get("FileStatuses").get("FileStatus").asInstanceOf[List[Map[String, Any]]] - - val fileCount = flist.length - - var i = 0 - var j = 0L - var fileDetails = new Array[Tuple4[String, Long, Long, Int]](fileCount) - var fSuffix = "" - var fLength = 0L - var fBlocksize = 0L - var fPart = 0 - var fullFilePath = "" - - while(i < fileCount) - { - fSuffix = flist(i).get("pathSuffix").getOrElse(path).asInstanceOf[String].toString - - fullFilePath = if (fSuffix == "") path else (path + "/" + fSuffix) - - fLength = flist(i).get("length").getOrElse(0).asInstanceOf[Double].toLong - - fBlocksize = flist(i).get("blockSize").getOrElse(0).asInstanceOf[Double].toLong - if (fLength > 0) fPart = (floor((fLength/fBlocksize)).toInt+1) else fPart = 0 - fileDetails(i) = new Tuple4(fullFilePath, fLength, fBlocksize, fPart) - - i+=1 - - } - fileDetails - - } - - /** - * This function prepares the partition details for each file based on the details populated by getFilesDetails - * This partition details is further used to spawn multiple connections to get data of a file using multiple connections - **/ - - def preparePartitions(fileDetails: Array[Tuple4[String, Long, Long, Int]], baseFile: String, partitionDetails: String, recordSeparator: String): Array [Tuple7[String, Long, Long, Int, Int, Int, String]] = { - - val totalFileCount = fileDetails.length - var i = 0 - - val partitionDet = partitionDetails.split(":") - - /** - *If number of partitions used for opening connections is passed as 0 or less, partition is defaulted to 4 - **/ - - var filePartition = if (partitionDet(0).toInt < 1) 4 else partitionDet(0).toInt - - /** - *If partition span (used to resolve record boundary) is sent as less than 10 KB, it is defaulted to 10 KB. Otherwise it is kept between 10KB to 100 KB - **/ - - var partitionSpan = if(partitionDet(1).toInt < 10000) 10000 else math.min(partitionDet(1).toInt, 100000) - - i = 0 - - var partList = new ArrayBuffer[Tuple7[String, Long, Long, Int, Int, Int, String]]() - - var j = 0 - var k = 0 - var filePart = 0 - var partPath = "" - var partLength = 0L - var fileLength = 0L - var partOffset = 0L - var fileSpan = 0 - - val maxSpan = 1000000 - val minSpan = 1000 - - while(i < totalFileCount) - { - - fileLength = fileDetails(i)._2.toLong - - if (fileLength > 0) { - - partPath = fileDetails(i)._1 - - fileLength = fileDetails(i)._2.toLong - partLength = fileLength/filePartition - - - if (partLength < 1000000) - { - filePartition = 1 - partitionSpan = 0 - } - - - j = 0 - while (j < filePartition) - { - partOffset = j*partLength.toLong - - if (j+1 == filePartition) partLength = fileDetails(i)._2.toLong - j*partLength - else partLength - - partList += new Tuple7(partPath, partOffset, partLength, j+1, filePartition, partitionSpan, recordSeparator) - j+=1 - k+=1 - } - } - - i+=1 - - } - - if (k < 1) - throw new Exception("Zero File Content") - - - var finalPartList = new Array[Tuple7[String, Long, Long, Int, Int, Int, String]](k) - - partList.copyToArray(finalPartList) - - finalPartList - } - - /** - * This function returns the list of files in a folder with file details as RDD - **/ - - def listFromWebHdfs(sc: SparkContext, path: String, trustStoreCred: String, userCred: String, connProp: String): RDD[String] = { - - - - val conns = connProp.split(":") - - val conn = Array(conns(0).toInt, conns(1).toInt) - - val usrCred = userCred.split(":") - - val trustCred = createTrustStoreCredForExecutors(trustStoreCred, path) - - val fileDetails = getFilesDetails(path, trustCred, usrCred, conn) - - def g(v:Tuple4[String, Long, Long, Int]) = v._1.split("/").last + "," + v._2.toString + "," + v._3.toString + "," + v._4.toString + "\n" - - val fds = fileDetails.map(x => g(x)) - - val flRdd = sc.parallelize(fds) - - flRdd - - } - - /** - * This function returns data of a file (or data of all files in a folder with same structure) as RDD - **/ - - def loadFromWebHdfs(sc: SparkContext, path: String, charset: String, trustStoreCred: String, userCred: String, connProp: String, partitionDetails: String, recordSeparator: String): RDD[String] = { - - - val conns = connProp.split(":") - - val conn = Array(conns(0).toInt, conns(1).toInt) - - val usrCrd = userCred.split(":") - - val trustCred = createTrustStoreCredForExecutors(trustStoreCred, path) - - val fileDetails = getFilesDetails(path, trustCred, usrCrd, conn) - - val parts = preparePartitions(fileDetails, path, partitionDetails, recordSeparator) - - val input = sc.parallelize(parts, parts.length) - - input.collect() - - val fRdd = input.flatMap(x => WebHdfsConnector.getAllFiles(x, usrCrd, trustCred, conn)) - - fRdd - - } - - - /** - * This function is passed to each executor through flatMap function to spawn one http connection from each executor for get a part of the file - **/ - - def getAllFiles (partInfo : Tuple7[String, Long, Long, Int, Int, Int, String], usrCred: Array[String], trustCred: Tuple3[String, File, String], connProp: Array[Int]): Iterator[String] = { - - val foffset = partInfo._2.toLong - val flength = partInfo._3.toLong - val ffilePath = partInfo._1 - val fpartNum = partInfo._4 - val ftotalPart = partInfo._5 - val fspan = partInfo._6 - val frecordSeparator = partInfo._7 - - val fileGetOpr = if(fpartNum < ftotalPart) - { - val effLength = flength + fspan - s"op=OPEN&offset=$foffset&length=$effLength&bufferSize=$effLength" - } - else - s"op=OPEN&offset=$foffset&length=$flength&bufferSize=$flength" - - val getUrl = callWebHdfsAPI(ffilePath, "", "GET", "LOCATION", trustCred, usrCred, connProp, fileGetOpr) - val partContent = callWebHdfsAPI(getUrl, "", "GET", "BODY", trustCred, usrCred, connProp, fileGetOpr) - - val records = getTillEndOfRecord(partContent, flength, fpartNum, ftotalPart, frecordSeparator) - - records.split("\n").iterator - - } - - /** - * This function calls webhdfs API after creating all necessary parameters from different configurations - **/ - - def callWebHdfsAPI(path: String, data: String, method: String, respType: String, trustStoreCred: Tuple3[String, File, String], userCred: Array[String], connProp: Array[Int], opr: String): String = { - - - val pathComp = path.split(":") - - val uri = (if(trustStoreCred._1 != "") "https:" else "http:") + pathComp(1) + ":" + pathComp(2) + "?" + opr - - var httpc = Http(uri).auth(userCred(0), userCred(1)).timeout(connTimeoutMs = connProp(0), readTimeoutMs = connProp(1)) - - httpc = (method : @switch) match { - case "GET" => httpc - case "PUT" => httpc.put(data).header("content-type", "application/csv") - case "POST" => httpc.postData(data).header("content-type", "application/csv") - } - - httpc = (trustStoreCred._1 : @switch) match { - case "" => httpc - case "N" => httpc.option(HttpOptions.allowUnsafeSSL) - case "Y" => httpc.option(HttpOptions.sslSocketFactory(biocSslSocketFactory(trustStoreCred._2, trustStoreCred._3))) - } - - val resp = (respType : @switch) match { - case "BODY" => httpc.asString.body - case "CODE" => httpc.asString.code - case "HEADERS" => httpc.asString.headers - case "LOCATION" => httpc.asString.location.mkString(" ") - } - - resp.toString() - - } - - /** - * This function resolves record boundaries. - * Right now this only supports "\n" as record boundary . This function has to be refined to support json or xml formats for different type of record separators - **/ - - def getTillEndOfRecord (content : String, partLength: Long, partNum: Int, totalPart: Int, recordSeparator : String): String = { - - val contentBytes = content.getBytes("UTF-8") - val recordSeparatorBytes = recordSeparator.getBytes("UTF-8") - - val contentBytesLength = contentBytes.length - - var bytePosition = 0 - - var startbyte = 0 - - startbyte = if(partNum == 1) 0 else { - /* - * This part of the code has to be rewritten later on to make it more generic for supporting other formats apart from csv. Right now it supports only csv. - */ - while (contentBytes(bytePosition) != '\n') bytePosition += 1 - bytePosition - } - - val length = if (partNum == totalPart) (contentBytesLength.toInt - startbyte).toInt - else { - bytePosition = partLength.toInt - /* - * This part of the code has to be rewritten later on to make it more generic for supporting other formats apart from csv. Right now it supports only csv. - */ - while (contentBytes(bytePosition) != '\n') bytePosition += 1 - (bytePosition - startbyte) - - } - - new String(contentBytes, startbyte, length, "UTF-8") - - } - - /** - * This function writes data back to hdfs using WebHDFS using multiple parallel connections. Right now file overwrite is not supported - **/ - - def writeToWebHdfs(dataToWrite: RDD[String] ,path: String, trustStoreCredStr: String, connStr : String, userCredStr : String, partitionStr : String): Unit = { - - val trustCred = createTrustStoreCredForExecutors(trustStoreCredStr, path) - - val conns = connStr.split(":") +import scala.math._ +import scala.util.parsing.json.JSON - val conn = Array(conns(0).toInt, conns(1).toInt) +import scalaj.http._ - val usr = userCredStr.split(":") +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD - val webHdfsChkDirOpr = "op=GETFILESTATUS" - val returnChkDir = callWebHdfsAPI(path, "", "GET", "CODE", trustCred, usr, conn, webHdfsChkDirOpr) +/** + * This object contains all utility functions for reading/writing data from/to remote WebHDFS + * server. The abstraction maintained in this layer is at the level of RDD. + */ +private[webhdfs] object WebHdfsConnector { - if (returnChkDir == "200") - throw new Exception("The File Already Exists : " + path + "\n") + /** + * Currently only files transferred using UTF-8 are supported + */ + val DEFAULT_CHARSET = Charset.forName("UTF-8") - val dPartitions = partitionStr.toInt + /** + * This function prepares the partition details for each file based on the details populated by + * getFilesDetails. This partition details is further used to spawn multiple connections to get + * data of a file using multiple connections. + */ + def preparePartitions( + fileDetails: Array[(String, Long, Long, Int)], + baseFile: String, + partitionDetails: String, + recordSeparator: String): Array[(String, Long, Long, Int, Int, Int, String)] = { - val textRdd = dataToWrite.repartition(dPartitions) + val totalFileCount = fileDetails.length + var i = 0 - val webHdfsMakeDirOpr = "op=MKDIRS" - val returnCreateDir = callWebHdfsAPI(path, "", "PUT", "CODE", trustCred, usr, conn, webHdfsMakeDirOpr) - - textRdd.mapPartitionsWithIndex((idx, iter) => WebHdfsConnector.saveAllFiles(idx, iter, usr, path, trustCred, conn)).collect() + val partitionDet = partitionDetails.split(":") - } + // if number of partitions used for opening connections is passed as 0 or less, partition is + // defaulted to 4 + var filePartition = if (partitionDet(0).toInt < 1) 4 else partitionDet(0).toInt - /** - * This function is passed to mapPartitionsWithIndex so that each executor task can save part of the data using separate connection - **/ + // if partition span (used to resolve record boundary) is sent as less than 10 KB, it is + // defaulted to 10 KB. Otherwise it is kept between 10KB to 100 KB + var partitionSpan = if (partitionDet(1).toInt < 10000) { + 10000 + } else { + math.min(partitionDet(1).toInt, 100000) + } - def saveAllFiles (idx: Int, data : Iterator[String], usrCred: Array[String], path: String, trustCred: Tuple3[String, File , String], connProp: Array[Int]): Iterator[String] = { + i = 0 + var partList = new ArrayBuffer[(String, Long, Long, Int, Int, Int, String)]() - var dataP = data.next() - while(data.hasNext) { - dataP = dataP + "\n" + data.next() - } + var j = 0 + var k = 0 +// var filePart = 0 + var partPath = "" + var partLength = 0L + var fileLength = 0L + var partOffset = 0L +// var fileSpan = 0 - val fnameArray = path.split("/") - val fnameIdx = fnameArray.length - 1 - val fname = fnameArray(fnameIdx) - val filePath = s"$path/part-000$idx-$fname" +// val maxSpan = 1000000 +// val minSpan = 1000 - val createOpr = "op=CREATE" + while (i < totalFileCount) { - val createUrl = callWebHdfsAPI(filePath, "", "PUT", "LOCATION", trustCred, usrCred, connProp, createOpr) - val created = callWebHdfsAPI(createUrl, dataP, "PUT", "CODE", trustCred, usrCred, connProp, createOpr) + fileLength = fileDetails(i)._2.toLong - val ret = Array(created.toString) - ret.iterator + if (fileLength > 0) { + + partPath = fileDetails(i)._1 + + fileLength = fileDetails(i)._2.toLong + partLength = fileLength / filePartition + + if (partLength < 1000000) { + filePartition = 1 + partitionSpan = 0 + } - } + j = 0 + while (j < filePartition) { + partOffset = j * partLength.toLong + if (j + 1 == filePartition) partLength = fileDetails(i)._2.toLong - j * partLength + + partList += new Tuple7(partPath, partOffset, partLength, j + 1, filePartition, + partitionSpan, recordSeparator) + j += 1 + k += 1 + } + } + + i += 1 + + } + + if (k < 1) { + throw new Exception("Zero File Content") + } + + var finalPartList = new Array[Tuple7[String, Long, Long, Int, Int, Int, String]](k) + + partList.copyToArray(finalPartList) + + finalPartList + } + + /** + * This function returns the list of files in a folder with file details as RDD + */ + def listFromWebHdfs(sc: SparkContext, + path: String, + trustStoreCred: String, + userCred: String, + connProp: String): RDD[String] = { + + val conns = connProp.split(":") + + val conn = Array(conns(0).toInt, conns(1).toInt) + + val usrCred = userCred.split(":") + + val trustCred = createTrustStoreCredForExecutors(trustStoreCred, path) + + val fileDetails = getFilesDetails(path, trustCred, usrCred, conn) + + def g(v: (String, Long, Long, Int)) = { + v._1.split("/").last + "," + v._2.toString + "," + v._3.toString + "," + v._4.toString + "\n" + } + + val fds = fileDetails.map(x => g(x)) + + val flRdd = sc.parallelize(fds) + + flRdd + } + + /** + * This function returns a Tuple for credential store which contains flag for validating + * Certificate, the Certificate File object and Certificate File Object password. + */ + def createTrustStoreCredForExecutors(cred: String, path: String): Tuple3[String, File, String] = { + + val trustStoreMap = if (cred != "") { + if (cred == "N") { + new Tuple3("N", null, "") + } + else { + if (cred == "Y") { + val tsd = SSLTrustStore.getCertDetails(path) + new Tuple3("Y", tsd._1, tsd._2) + } + else { + throw new Exception("Invalid Certificate Validation Option") + } + } + } else { + new Tuple3("", null, "") + } + trustStoreMap + } + + /** + * This function returns the details of the the files in a folder if the path passed is a folder. + * In case a File path is passed it returns the details of the files. + * Returns an Array of Tuple where each Tuple represents one file with details of full file path, + * size of the file, block size of the file and number of partitions based on size of the file and + * block size. + */ + def getFilesDetails(path: String, + trustCred: (String, File, String), + usrCred: Array[String], + connProp: Array[Int]): Array[(String, Long, Long, Int)] = { + + val listStatusOpr = s"op=LISTSTATUS" + + val listStatus = callWebHdfsAPI(path, "", "GET", "BODY", trustCred, usrCred, connProp, + listStatusOpr) + + if (listStatus.contains("RemoteException")) { + throw new Exception(listStatus) + } + + val flist = JSON + .parseFull(listStatus) + .toList(0) + .asInstanceOf[Map[String, Map[String, Any]]] + .get("FileStatuses") + .get("FileStatus") + .asInstanceOf[List[Map[String, Any]]] + + val fileCount = flist.length + + var i = 0 + var j = 0L + val fileDetails = new Array[(String, Long, Long, Int)](fileCount) + var fSuffix = "" + var fLength = 0L + var fBlocksize = 0L + var fPart = 0 + var fullFilePath = "" + + while (i < fileCount) { + fSuffix = flist(i).get("pathSuffix").getOrElse(path).asInstanceOf[String].toString + + fullFilePath = if (fSuffix == "") path else (path + "/" + fSuffix) + + fLength = flist(i).get("length").getOrElse(0).asInstanceOf[Double].toLong + + fBlocksize = flist(i).get("blockSize").getOrElse(0).asInstanceOf[Double].toLong + if (fLength > 0) fPart = (floor((fLength / fBlocksize)).toInt + 1) else fPart = 0 + fileDetails(i) = new Tuple4(fullFilePath, fLength, fBlocksize, fPart) + + i += 1 + } + fileDetails + } + + /** + * This function calls the WebHDFS API after creating all necessary parameters from different + * configurations + */ + def callWebHdfsAPI(path: String, + data: String, + method: String, + respType: String, + trustStoreCred: (String, File, String), + userCred: Array[String], + connProp: Array[Int], + opr: String): String = { + + val pathComp = path.split(":") + + val uri = (if (trustStoreCred._1 != "") "https:" else "http:") + .concat(pathComp(1) + ":" + pathComp(2) + "?" + opr) + + var httpc = Http(uri).auth(userCred(0), userCred(1)).timeout(connTimeoutMs = connProp(0), + readTimeoutMs = connProp(1)) + + httpc = (method: @switch) match { + case "GET" => httpc + case "PUT" => httpc.put(data).header("content-type", "application/csv") + case "POST" => httpc.postData(data).header("content-type", "application/csv") + } + + httpc = (trustStoreCred._1: @switch) match { + case "" => httpc + case "N" => httpc.option(HttpOptions.allowUnsafeSSL) + case "Y" => httpc.option(HttpOptions.sslSocketFactory(biocSslSocketFactory(trustStoreCred._2, + trustStoreCred._3))) + } + + val resp = (respType: @switch) match { + case "BODY" => httpc.asString.body + case "CODE" => httpc.asString.code + case "HEADERS" => httpc.asString.headers + case "LOCATION" => httpc.asString.location.mkString(" ") + } + + resp.toString() + } + + /** + * This function returns a SSLSocketFactory which needs to be used in HTTP connection library in + * case Certificate to be validated + */ + def biocSslSocketFactory(fl: File, pswrd: String): SSLSocketFactory = { + + val tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()) + val ks = KeyStore.getInstance("JKS") + val fis = new java.io.FileInputStream(fl) + ks.load(fis, pswrd.toCharArray()); + tmf.init(ks); + + val sslc = SSLContext.getInstance("SSL") + + sslc.init(null, tmf.getTrustManagers(), null) + + sslc.getSocketFactory() + } + + /** + * This function returns data of a file (or data of all files in a folder with same structure) as + * RDD + */ + def loadFromWebHdfs(sc: SparkContext, + path: String, + charset: String, + trustStoreCred: String, + userCred: String, + connProp: String, + partitionDetails: String, + recordSeparator: String): RDD[String] = { + + val conns = connProp.split(":") + + val conn = Array(conns(0).toInt, conns(1).toInt) + + val usrCrd = userCred.split(":") + + val trustCred = createTrustStoreCredForExecutors(trustStoreCred, path) + + val fileDetails = getFilesDetails(path, trustCred, usrCrd, conn) + + val parts = preparePartitions(fileDetails, path, partitionDetails, recordSeparator) + + val input = sc.parallelize(parts, parts.length) + + input.collect() + + val fRdd = input.flatMap(x => WebHdfsConnector.getAllFiles(x, usrCrd, trustCred, conn)) + + fRdd + } + + /** + * This function is passed to each executor through flatMap function to spawn one http connection + * from each executor for get a part of the file + */ + def getAllFiles(partInfo: (String, Long, Long, Int, Int, Int, String), + usrCred: Array[String], + trustCred: (String, File, String), + connProp: Array[Int]): Iterator[String] = { + + val foffset = partInfo._2.toLong + val flength = partInfo._3.toLong + val ffilePath = partInfo._1 + val fpartNum = partInfo._4 + val ftotalPart = partInfo._5 + val fspan = partInfo._6 + val frecordSeparator = partInfo._7 + + val fileGetOpr = if (fpartNum < ftotalPart) { + val effLength = flength + fspan + s"op=OPEN&offset=$foffset&length=$effLength&bufferSize=$effLength" + } + else { + s"op=OPEN&offset=$foffset&length=$flength&bufferSize=$flength" + } + + val getUrl = callWebHdfsAPI(ffilePath, "", "GET", "LOCATION", trustCred, usrCred, connProp, + fileGetOpr) + val partContent = callWebHdfsAPI(getUrl, "", "GET", "BODY", trustCred, usrCred, connProp, + fileGetOpr) + + val records = getTillEndOfRecord(partContent, flength, fpartNum, ftotalPart, frecordSeparator) + + records.split("\n").iterator + } + + /** + * This function resolves record boundaries. + * Right now this only supports "\n" as record boundary . This function has to be refined to + * support json or xml formats for different type of record separators + */ + def getTillEndOfRecord(content: String, + partLength: Long, + partNum: Int, + totalPart: Int, + recordSeparator: String): String = { + + val contentBytes = content.getBytes("UTF-8") +// val recordSeparatorBytes = recordSeparator.getBytes("UTF-8") + + val contentBytesLength = contentBytes.length + + var bytePosition = 0 + + var startbyte = 0 + + startbyte = if (partNum == 1) 0 + else { + // TODO: This part of the code has to be rewritten later on to make it more generic for + // supporting other formats apart from csv. Right now it supports only csv + while (contentBytes(bytePosition) != '\n') bytePosition += 1 + bytePosition + } + + val length = if (partNum == totalPart) (contentBytesLength.toInt - startbyte).toInt + else { + bytePosition = partLength.toInt + // TODO: This part of the code has to be rewritten later on to make it more generic for + // supporting other formats apart from csv. Right now it supports only csv. + while (contentBytes(bytePosition) != '\n') bytePosition += 1 + (bytePosition - startbyte) + } + + new String(contentBytes, startbyte, length, "UTF-8") + } + + /** + * This function writes data back to hdfs using WebHDFS using multiple parallel connections. + * Right now file overwrite is not supported + */ + def writeToWebHdfs(dataToWrite: RDD[String], + path: String, + trustStoreCredStr: String, + connStr: String, + userCredStr: String, + partitionStr: String): Unit = { + + val trustCred = createTrustStoreCredForExecutors(trustStoreCredStr, path) + + val conns = connStr.split(":") + + val conn = Array(conns(0).toInt, conns(1).toInt) + + val usr = userCredStr.split(":") + + val webHdfsChkDirOpr = "op=GETFILESTATUS" + val returnChkDir = callWebHdfsAPI(path, "", "GET", "CODE", trustCred, usr, conn, + webHdfsChkDirOpr) + + if (returnChkDir == "200") { + throw new Exception("The File Already Exists : " + path + "\n") + } + + val dPartitions = partitionStr.toInt + + val textRdd = dataToWrite.repartition(dPartitions) + + val webHdfsMakeDirOpr = "op=MKDIRS" + val returnCreateDir = callWebHdfsAPI(path, "", "PUT", "CODE", trustCred, usr, conn, + webHdfsMakeDirOpr) + + textRdd.mapPartitionsWithIndex((idx, iter) => WebHdfsConnector.saveAllFiles(idx, iter, usr, + path, trustCred, conn)).collect() + } + + /** + * This function is passed to mapPartitionsWithIndex so that each executor task can save part of + * the data using separate connection + */ + def saveAllFiles(idx: Int, + data: Iterator[String], + usrCred: Array[String], + path: String, + trustCred: (String, File, String), + connProp: Array[Int]): Iterator[String] = { + + var dataP = data.next() + while (data.hasNext) { + dataP = dataP + "\n" + data.next() + } + + val fnameArray = path.split("/") + val fnameIdx = fnameArray.length - 1 + val fname = fnameArray(fnameIdx) + val filePath = s"$path/part-000$idx-$fname" + + val createOpr = "op=CREATE" + + val createUrl = callWebHdfsAPI(filePath, "", "PUT", "LOCATION", trustCred, usrCred, connProp, + createOpr) + val created = callWebHdfsAPI(createUrl, dataP, "PUT", "CODE", trustCred, usrCred, connProp, + createOpr) + + val ret = Array(created.toString) + ret.iterator + } } diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsInferSchema.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsInferSchema.scala index 4b4935e9..a012a3fb 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsInferSchema.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsInferSchema.scala @@ -25,13 +25,45 @@ import scala.util.control.Exception._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.types._ + /** * Utility functions for infering schema - * Copied from com.databricks.spark.csv.util package as there was now way to reuse this object -*/ + * TODO: copied from com.databricks.spark.csv.util package as there was now way to reuse this object + */ +private[webhdfs] object WebHdfsInferSchema { + /** + * TODO: Copied from internal Spark API + * [[org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion]] + */ + val findTightestCommonType: (DataType, DataType) => Option[DataType] = { + case (t1, t2) if t1 == t2 => Some(t1) + case (NullType, t1) => Some(t1) + case (t1, NullType) => Some(t1) + case (StringType, t2) => Some(StringType) + case (t1, StringType) => Some(StringType) -private[webhdfs] object WebHdfsInferSchema { + // Promote numeric types to the highest of the two and all numeric types to unlimited decimal + case (t1, t2) if Seq(t1, t2).forall(numericPrecedence.contains) => + val index = numericPrecedence.lastIndexWhere(t => t == t1 || t == t2) + Some(numericPrecedence(index)) + + case _ => None + } + /** + * TODO: Copied from internal Spark api + * [[org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion]] + */ + private val numericPrecedence: IndexedSeq[DataType] = + IndexedSeq[DataType]( + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + TimestampType, + DecimalType.USER_DEFAULT) /** * Similar to the JSON schema inference. @@ -40,11 +72,10 @@ private[webhdfs] object WebHdfsInferSchema { * 2. Merge row types to find common type * 3. Replace any null types with string type */ - def apply( - tokenRdd: RDD[Array[String]], - header: Array[String], - nullValue: String = "", - dateFormatter: SimpleDateFormat = null): StructType = { + def apply( tokenRdd: RDD[Array[String]], + header: Array[String], + nullValue: String = "", + dateFormatter: SimpleDateFormat = null): StructType = { val startType: Array[DataType] = Array.fill[DataType](header.length)(NullType) val rootTypes: Array[DataType] = tokenRdd.aggregate(startType)( inferRowType(nullValue, dateFormatter), @@ -62,31 +93,24 @@ private[webhdfs] object WebHdfsInferSchema { } private def inferRowType(nullValue: String, dateFormatter: SimpleDateFormat) - (rowSoFar: Array[DataType], next: Array[String]): Array[DataType] = { + (rowSoFar: Array[DataType], next: Array[String]): Array[DataType] = { var i = 0 - while (i < math.min(rowSoFar.length, next.length)) { // May have columns on right missing. + while (i < math.min(rowSoFar.length, next.length)) { + // May have columns on right missing. rowSoFar(i) = inferField(rowSoFar(i), next(i), nullValue, dateFormatter) - i+=1 + i += 1 } rowSoFar } - private[webhdfs] def mergeRowTypes( - first: Array[DataType], - second: Array[DataType]): Array[DataType] = { - first.zipAll(second, NullType, NullType).map { case ((a, b)) => - findTightestCommonType(a, b).getOrElse(NullType) - } - } - /** * Infer type of string field. Given known type Double, and a string "1", there is no * point checking if it is an Int, as the final type must be Double or higher. */ private[webhdfs] def inferField(typeSoFar: DataType, - field: String, - nullValue: String = "", - dateFormatter: SimpleDateFormat = null): DataType = { + field: String, + nullValue: String = "", + dateFormatter: SimpleDateFormat = null): DataType = { def tryParseInteger(field: String): DataType = if ((allCatch opt field.toInt).isDefined) { IntegerType } else { @@ -110,7 +134,7 @@ private[webhdfs] object WebHdfsInferSchema { def tryParseTimestamp(field: String): DataType = { if (dateFormatter != null) { // This case infers a custom `dataFormat` is set. - if ((allCatch opt dateFormatter.parse(field)).isDefined){ + if ((allCatch opt dateFormatter.parse(field)).isDefined) { TimestampType } else { tryParseBoolean(field) @@ -157,38 +181,10 @@ private[webhdfs] object WebHdfsInferSchema { } } - /** - * Copied from internal Spark api - * [[org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion]] - */ - private val numericPrecedence: IndexedSeq[DataType] = - IndexedSeq[DataType]( - ByteType, - ShortType, - IntegerType, - LongType, - FloatType, - DoubleType, - TimestampType, - DecimalType.USER_DEFAULT) - - - /** - * Copied from internal Spark api - * [[org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion]] - */ - val findTightestCommonType: (DataType, DataType) => Option[DataType] = { - case (t1, t2) if t1 == t2 => Some(t1) - case (NullType, t1) => Some(t1) - case (t1, NullType) => Some(t1) - case (StringType, t2) => Some(StringType) - case (t1, StringType) => Some(StringType) - - // Promote numeric types to the highest of the two and all numeric types to unlimited decimal - case (t1, t2) if Seq(t1, t2).forall(numericPrecedence.contains) => - val index = numericPrecedence.lastIndexWhere(t => t == t1 || t == t2) - Some(numericPrecedence(index)) - - case _ => None + private[webhdfs] def mergeRowTypes(first: Array[DataType], + second: Array[DataType]): Array[DataType] = { + first.zipAll(second, NullType, NullType).map { case ((a, b)) => + findTightestCommonType(a, b).getOrElse(NullType) + } } } diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsParseModes.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsParseModes.scala index 332a6e38..52b7162a 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsParseModes.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsParseModes.scala @@ -15,34 +15,33 @@ * limitations under the License. */ - package org.apache.bahir.datasource.webhdfs.util /** * Utility functions for mode of Parsing * Copied from com.databricks.spark.csv.util package as there was now way to reuse this object -*/ - + */ private[webhdfs] object WebHdfsParseModes { val PERMISSIVE_MODE = "PERMISSIVE" val DROP_MALFORMED_MODE = "DROPMALFORMED" val FAIL_FAST_MODE = "FAILFAST" - val DEFAULT = PERMISSIVE_MODE - def isValidMode(mode: String): Boolean = { - mode.toUpperCase match { - case PERMISSIVE_MODE | DROP_MALFORMED_MODE | FAIL_FAST_MODE => true - case _ => false - } - } - def isDropMalformedMode(mode: String): Boolean = mode.toUpperCase == DROP_MALFORMED_MODE + def isFailFastMode(mode: String): Boolean = mode.toUpperCase == FAIL_FAST_MODE - def isPermissiveMode(mode: String): Boolean = if (isValidMode(mode)) { + + def isPermissiveMode(mode: String): Boolean = if (isValidMode(mode)) { mode.toUpperCase == PERMISSIVE_MODE } else { true // We default to permissive is the mode string is not valid } + + def isValidMode(mode: String): Boolean = { + mode.toUpperCase match { + case PERMISSIVE_MODE | DROP_MALFORMED_MODE | FAIL_FAST_MODE => true + case _ => false + } + } } diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsTypeCast.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsTypeCast.scala index 26acd6bd..d3a93663 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsTypeCast.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsTypeCast.scala @@ -20,18 +20,17 @@ package org.apache.bahir.datasource.webhdfs.util import java.math.BigDecimal import java.sql.{Date, Timestamp} -import java.text.{SimpleDateFormat, NumberFormat} +import java.text.{NumberFormat, SimpleDateFormat} import java.util.Locale -import org.apache.spark.sql.types._ - import scala.util.Try +import org.apache.spark.sql.types._ + /** * Utility functions for type casting * Copied from com.databricks.spark.csv.util package as there was now way to reuse this object -*/ - + */ object WebHdfsTypeCast { /** @@ -41,23 +40,23 @@ object WebHdfsTypeCast { * For string types, this is simply the datum. For other types. * For other nullable types, this is null if the string datum is empty. * - * @param datum string value + * @param datum string value * @param castType SparkSQL type */ - private[webhdfs] def castTo( - datum: String, - castType: DataType, - nullable: Boolean = true, - treatEmptyValuesAsNulls: Boolean = false, - nullValue: String = "", - dateFormatter: SimpleDateFormat = null): Any = { + private[webhdfs] def castTo(datum: String, + castType: DataType, + nullable: Boolean = true, + treatEmptyValuesAsNulls: Boolean = false, + nullValue: String = "", + dateFormatter: SimpleDateFormat = null): Any = { + // if nullValue is not an empty string, don't require treatEmptyValuesAsNulls // to be set to true val nullValueIsNotEmpty = nullValue != "" if (datum == nullValue && nullable && (!castType.isInstanceOf[StringType] || treatEmptyValuesAsNulls || nullValueIsNotEmpty) - ){ + ) { null } else { castType match { From d7b3bf7ed849ef5f39ffb935127fa049b3537603 Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Wed, 16 Nov 2016 15:32:16 -0800 Subject: [PATCH 08/24] [BAHIR-75] - use "${scala.binary.version}"" instead of "2.11" --- datasource-webhdfs/pom.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datasource-webhdfs/pom.xml b/datasource-webhdfs/pom.xml index cad5a04f..80a5f958 100755 --- a/datasource-webhdfs/pom.xml +++ b/datasource-webhdfs/pom.xml @@ -25,7 +25,6 @@ ../pom.xml - org.apache.bahir datasource-webhdfs_2.11 datasource-webhdfs @@ -37,7 +36,7 @@ org.scalaj - scalaj-http_2.11 + scalaj-http_${scala.binary.version} 2.3.0 From a77e3725693c9d4034ffa4dd6a4b9521824370eb Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Wed, 16 Nov 2016 19:20:56 -0800 Subject: [PATCH 09/24] [BAHIR-75] - add "spark-" prefix to artifactId consistent with other Bahir packages --- datasource-webhdfs/README.md | 8 +++++--- datasource-webhdfs/pom.xml | 3 ++- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/datasource-webhdfs/README.md b/datasource-webhdfs/README.md index f8d19937..7158c34a 100644 --- a/datasource-webhdfs/README.md +++ b/datasource-webhdfs/README.md @@ -8,7 +8,7 @@ Using SBT: libraryDependencies += "org.apache.bahir" %% "spark-datasource-webhdfs" % "2.1.0-SNAPSHOT" ``` -Using Maven: +Using Maven (Scala version 2.11): ```xml @@ -21,12 +21,14 @@ Using Maven: This library can also be added to Spark jobs launched through `spark-shell` or `spark-submit` by using the `--packages` command line option. For example, to include it when starting the spark shell: - $ bin/spark-shell --packages org.apache.bahir:spark-datasource-webhdfs_2.11:2.1.0-SNAPSHOT +```Shell +$ bin/bin/spark-shell --packages org.apache.bahir:spark-datasource-webhdfs_2.11:2.1.0-SNAPSHOT +``` Unlike using `--jars`, using `--packages` ensures that this library and its dependencies will be added to the classpath. The `--packages` argument can also be used with `bin/spark-submit`. -This library is compiled for Scala 2.11 only, and intends to support Spark 2.0 onwards. +This library is compiled for Scala 2.10 and 2.11, and intended to support Spark 2.0 onwards. ## Examples diff --git a/datasource-webhdfs/pom.xml b/datasource-webhdfs/pom.xml index 80a5f958..795a123c 100755 --- a/datasource-webhdfs/pom.xml +++ b/datasource-webhdfs/pom.xml @@ -25,7 +25,8 @@ ../pom.xml - datasource-webhdfs_2.11 + org.apache.bahir + spark-datasource-webhdfs_2.11 datasource-webhdfs From 6936bd8a47919db8093a596f30761c7b46c98547 Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Wed, 30 Nov 2016 16:38:53 -0800 Subject: [PATCH 10/24] [BAHIR-75][WIP] - rudimentary extension of WebHdfsFileSystem --- .../web/bahir/BahirWebHdfsFileSystem.scala | 50 +++++++++++++++++++ 1 file changed, 50 insertions(+) create mode 100644 datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/bahir/BahirWebHdfsFileSystem.scala diff --git a/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/bahir/BahirWebHdfsFileSystem.scala b/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/bahir/BahirWebHdfsFileSystem.scala new file mode 100644 index 00000000..05183465 --- /dev/null +++ b/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/bahir/BahirWebHdfsFileSystem.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdfs.web.bahir + +import java.net.URL + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hdfs.web.WebHdfsFileSystem +import org.apache.hadoop.hdfs.web.resources.HttpOpParam.Op +import org.apache.hadoop.hdfs.web.resources.Param +import org.apache.hadoop.io.Text + +/** + * A FileSystem for HDFS over the web, extending [[org.apache.hadoop.hdfs.web.WebHdfsFileSystem]] + * to allow secure authentication and configurable gateway path segments. + * + * TODO: upgrade Hadoop version (Maven dependencies) to override SWebHdfsFileSystem + */ +class BahirWebHdfsFileSystem extends WebHdfsFileSystem { + + // TODO: figure out how/where to authenticate + + // TODO: gateway path should be configurable + val gatewayPath = "/gateway/default" + + + override def toUrl(op: Op, fspath: Path, parameters: Param[_, _]*): URL = { + val url = super.toUrl(op, fspath, parameters: _*) + + new URL(url.getProtocol, url.getHost, url.getPort, + url.getFile.replaceFirst(WebHdfsFileSystem.PATH_PREFIX, + gatewayPath + WebHdfsFileSystem.PATH_PREFIX)) + } + +} From a9ef90745d498a5243ca96e4615d859733fcba60 Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Wed, 30 Nov 2016 18:10:55 -0800 Subject: [PATCH 11/24] [BAHIR-75][WIP] - rudimentary extension of WebHdfsFileSystem (use original hadoop namespace for package private field/method access) --- .../hadoop/hdfs/web/{bahir => }/BahirWebHdfsFileSystem.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) rename datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/{bahir => }/BahirWebHdfsFileSystem.scala (93%) diff --git a/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/bahir/BahirWebHdfsFileSystem.scala b/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfsFileSystem.scala similarity index 93% rename from datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/bahir/BahirWebHdfsFileSystem.scala rename to datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfsFileSystem.scala index 05183465..2b64dd9b 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/bahir/BahirWebHdfsFileSystem.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfsFileSystem.scala @@ -15,15 +15,13 @@ * limitations under the License. */ -package org.apache.hadoop.hdfs.web.bahir +package org.apache.hadoop.hdfs.web import java.net.URL import org.apache.hadoop.fs.Path -import org.apache.hadoop.hdfs.web.WebHdfsFileSystem import org.apache.hadoop.hdfs.web.resources.HttpOpParam.Op import org.apache.hadoop.hdfs.web.resources.Param -import org.apache.hadoop.io.Text /** * A FileSystem for HDFS over the web, extending [[org.apache.hadoop.hdfs.web.WebHdfsFileSystem]] From f791f1cb330518e56d968ac04605cb91b9fbfefa Mon Sep 17 00:00:00 2001 From: Sourav Mazumder Date: Wed, 7 Dec 2016 13:21:59 -0800 Subject: [PATCH 12/24] WebHdfsConnector prototype --- ...pache.spark.sql.sources.DataSourceRegister | 1 - .../webhdfs/BahirWebHdfsDataSetWrapper.scala | 365 ++++++++++ .../datasource/webhdfs/DefaultSource.scala | 248 ------- .../webhdfs/util/WebHdfsConnector.scala | 689 +++++++----------- .../apache/hadoop/hdfs/web/BahirWebHdfs.java | 75 ++ .../hdfs/web/BahirWebHdfsFileSystem.scala | 33 +- 6 files changed, 711 insertions(+), 700 deletions(-) create mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala delete mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/DefaultSource.scala create mode 100644 datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfs.java diff --git a/datasource-webhdfs/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/datasource-webhdfs/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index 2108bc67..e69de29b 100644 --- a/datasource-webhdfs/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/datasource-webhdfs/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -1 +0,0 @@ -org.apache.bahir.datasource.webhdfs.DefaultSource diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala new file mode 100644 index 00000000..dc755b85 --- /dev/null +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala @@ -0,0 +1,365 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.bahir.datasource.webhdfs + +import java.net.URI +import java.net.URL +import java.io._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.permission.FsPermission +import org.apache.hadoop.util.Progressable + +import scala.collection.mutable.HashMap +import scala.math._ + +import org.apache.bahir.datasource.webhdfs.util._ +import org.apache.bahir.datasource.webhdfs.csv._ + + +/** + * This class contains functions for reading/writing data from/to remote webhdfs server in Spark DataSource +*/ + + + +class BahirWebHdfsFileSystem + extends FileSystem { + + var uri:URI = null + var rHdfsUri:URI = null + var conf:Configuration = null + var workingDir = null + + var readFullFile = false + var usrCred = "" + var connections = 0 + var certValidation = "Y" + + var fileStatusMap : HashMap[String, FileStatus] = HashMap() + var listStatusMap : HashMap[String, Array[FileStatus]] = HashMap() + + override def getUri() : URI = uri + + + /** + * This method does necessary initialization of the configuration parameters + */ + + override def initialize( + uriOrg: URI, + confOrg: Configuration): Unit = { + + super.initialize(uriOrg, confOrg) + + setConf(confOrg) + + rHdfsUri = uriOrg + conf = confOrg + + val rfFlg = conf.get("readFullFile") + readFullFile = if(rfFlg == null) false else rfFlg.toBoolean + + val usrCredStr = conf.get("usrCredStr") + usrCred = if(usrCredStr == null) throw new Exception ("User Credential Has To Be Specified For The Remote HDFS") else usrCredStr.toString + + val conns = conf.get("connections") + connections = if(conns == null) 0 else conns.toInt + + val certFlg = conf.get("certValidationFlg") + certValidation = if(certFlg == null) "Y" else certFlg.toString + + uri = URI.create(uriOrg.getScheme() + "://" + uriOrg.getAuthority()) + //print("uri : ", uri + " , connections : " + connections + " , user cred : " + usrCred + "\n") + + } + + override def getWorkingDirectory() : Path = { + + val path = new Path(rHdfsUri) + //print("Working Directory : " + path + "\n") + path + } + + override def setWorkingDirectory(dir : Path) : Unit = {} + + override def rename(srcPath : Path, destPath : Path) : Boolean = { + throw new Exception("File Rename Not Supported") + } + + override def delete(srcPath : Path, recursive : Boolean) : Boolean = { + throw new Exception("File Delete Not Supported") + } + + override def mkdirs(srcPath : Path, permission : FsPermission) : Boolean = { + throw new Exception("Make Directory Not Supported") + } + + override def append(srcPath : Path, bufferSize : Int, progress : Progressable) : FSDataOutputStream = { + throw new Exception("File Append Not Supported") + } + + override def getFileStatus(f : Path) : FileStatus = { + + val file = stripQryFromFilePath(f).toString + + var fStatus : FileStatus = fileStatusMap.getOrElse(file, null) + + val fileStatus = if (fStatus == null) { + val fStatusMap = WebHdfsConnector.getFileStatus(file, certValidation, "1000:5000", usrCred) + fStatus = createFileStatus(f, fStatusMap) + fileStatusMap.put(f.toString, fStatus) + fStatus + } + else + fStatus + + fileStatus + } + + override def listStatus(f : Path) : Array[FileStatus] = { + + val file = stripQryFromFilePath(f).toString + + var lStatus : Array[FileStatus] = listStatusMap.getOrElse(file, null) + + //print("file in listStatus: " + file + "\n") + + val listStatus = if(lStatus == null) { + + val fStatusMapList = WebHdfsConnector.getListStatus(file, certValidation, "1000:5000", usrCred) + val fileCount = fStatusMapList.length + + lStatus = new Array[FileStatus](fileCount) + + var i = 0 + + while(i < fileCount) + { + lStatus(i) = createFileStatus(f, fStatusMapList(i)) + i+=1 + } + + listStatusMap.put(f.toString, lStatus) + lStatus + } + else + lStatus + + + //print(" listStatus: " + listStatus + "\n") + listStatus + } + + override def open(f: Path, bs: Int) : FSDataInputStream = { + + val fileStatus = getFileStatus(f) + val blockSize = fileStatus.getBlockSize + val fileLength = fileStatus.getLen + + val file = stripQryFromFilePath(f) + + print("file uri in open : " + file + "\n") + + val qMap = getQryMapFromFilePath(f) + + val fConnections = if(qMap == null) connections + else + qMap.getOrElse("connections", connections).asInstanceOf[String].toInt + + new FSDataInputStream(new BahirWebHdfsInputStream(file, bs, blockSize, fileLength, readFullFile, usrCred, fConnections, certValidation)) + + } + + override def create(srcPath : Path, permission : FsPermission, flag : Boolean, bufferSize : Int, replication : Short, blockSize : Long, progress : Progressable) : FSDataOutputStream = { + throw new Exception("File Create Not Yet Supported") + } + + + private def createFileStatus(fPath : Path, statusMap: Map[String, Any]) : FileStatus = { + + val lng = conf.get("length") + val partlng = if(lng == null) 1 else lng.toInt + + val blk = conf.get("block") + val partblk = if(blk == null) 1 else blk.toInt + + + val isDirFlg = if(statusMap.getOrElse("type", "") == "DIRECTORY") true else false + val pathSuffix = statusMap.getOrElse("pathSuffix", "") + val targetPath = if(pathSuffix == "") fPath else new Path(fPath.toString + "/" + pathSuffix) + val fStatus = new FileStatus( + statusMap.getOrElse("length", 0).asInstanceOf[Double].toLong*partlng, + isDirFlg, + statusMap.getOrElse("replication",1).asInstanceOf[Double].toInt, + (statusMap.getOrElse("blockSize", 128000000).asInstanceOf[Double].toLong)/partblk, + statusMap.getOrElse("modificationTime",0).asInstanceOf[Double].toLong, + statusMap.getOrElse("accessTime",0).asInstanceOf[Double].toLong, + null, + statusMap.getOrElse("owner", "default").asInstanceOf[String], + statusMap.getOrElse("group", "default").asInstanceOf[String], + null, targetPath) + fStatus + } + + private def stripQryFromFilePath(f : Path) : Path = { + + //print("file uri : " + f.toUri + "\n") + + val pathStrWithoutQry = f.toString.replace(getQryStrFromFilePath(f), "") + new Path(pathStrWithoutQry) + + } + + private def getQryMapFromFilePath(f : Path) : HashMap[String, String] = { + + val qryStr = getQryStrFromFilePath(f) + if(qryStr == "") null + else { + + val params = qryStr.replace(";", "").substring(1).split("&") + + val paramCount = params.length + + //print("params : " + params + " , lenth : " + paramCount + "\n") + var paramMap : HashMap[String, String] = new HashMap() + + var i = 0 + + while(i < paramCount) + { + val paramKV = params(i).split("=") + paramMap.put(paramKV(0), paramKV(1)) + i+=1 + } + + //print("param map : " + paramMap + "\n") + paramMap + } + + } + + private def getQryStrFromFilePath(f : Path) : String = { + + val fileStr = f.toString + + val start = fileStr.indexOf("&") + val end = fileStr.indexOf(";") + + //print("start and end index " + start +"\n") + + val qryStr = if (start > 0) fileStr.substring(start, end) else "" + + //print("query : " + qryStr + "\n") + qryStr + } + +} + +class BahirWebHdfsInputStream (fPath : Path, bufferSz : Int, blockSz : Long, fileSz : Long, readFull : Boolean, usrCrd : String, conns : Int, certValidation : String) + extends FSInputStream { + + val filePath: Path = fPath + val bufferSize: Int = bufferSz + val blockSize: Long = blockSz + val fileSize: Long = fileSz + val readFullFlg: Boolean = readFull + val usrCred: String = usrCrd + val connections: Int = conns + val certValidationFlg: String = certValidation + + var pos = 0L + + var in : ByteArrayInputStream = null + + var callCount = 0 + + + override def read(b: Array[Byte], offset: Int, length: Int) : Int = { + + if (in == null) createWebHdfsInputStream(pos) + + callCount+=1 + + var bCount = in.read(b, offset, length) + + //print("In read - call count : " + callCount + " , pos : " + pos + ", offset : " + offset + " , length : " + length + " , byte count total : " + bCount + "\n") + + if (bCount < 0 && pos < fileSize) + { + //print("In read - bCount less than 0 , call count : " + callCount + " , file size : " + fileSize + " , pos : " + pos + ", offset : " + offset + " , length : " + length + " , byte count total : " + bCount + "\n") + //createWebHdfsInputStream(pos) + seek(pos) + bCount = in.read(b, offset, length) + } + + pos+=bCount + + bCount + + } + + /** + * This is a dummy implementation as Spark does not use it. We need it here just to satisy interface contract + */ + + override def read() = { + + read(new Array[Byte](4056), 0, 100) + } + + /** + * This is a dummy implementation as Spark does not use it. We need it here just to satisy interface contract + */ + + override def seekToNewSource(targetPos: Long) = false + + override def getPos() = { + pos + } + + + override def seek(newPos : Long) = { + //print("In seek - newpos : " + newPos + " , old pos : " + pos + "\n") + if (pos != newPos) { + pos = newPos + if (in != null) in.close + } + createWebHdfsInputStream(pos) + } + + private def createWebHdfsInputStream(pos : Long) = { + + val poe = if(connections == 0) + if(blockSize > fileSize || readFullFlg == true) fileSize else (floor(pos/blockSize).toLong + 1)*blockSize + 10000 + else + floor(fileSize/(connections - 1)).toInt + 10000 + + + //print("In read - input stream null , block size : " + blockSize + " , file size : " + fileSize + " , red full flg : " + readFullFlg + " , pos : " + pos + " , poe : " + poe +"\n") + + in = WebHdfsConnector.getFileInputStream(filePath.toString(), pos, poe, certValidationFlg, "1000:50000", usrCred) + } + +} diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/DefaultSource.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/DefaultSource.scala deleted file mode 100644 index d6caa269..00000000 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/DefaultSource.scala +++ /dev/null @@ -1,248 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.bahir.datasource.webhdfs - -import scala.annotation.switch - -import org.apache.bahir.datasource.webhdfs.csv._ -import org.apache.bahir.datasource.webhdfs.util._ - -import org.apache.spark.sql.{DataFrame, SaveMode, SQLContext} -import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructField, StructType} - - -/** - * A Spark data source to read/write data from/to remote WebHDFS servers. - * This function is written in line with the DataSource function in com.databricks.spark.csv - */ -class DefaultSource - extends RelationProvider - with SchemaRelationProvider - with CreatableRelationProvider - with DataSourceRegister { - - override def shortName() : String = "webhdfs" - - private def checkPath(parameters: Map[String, String]): String = { - parameters.getOrElse("path", sys.error("'path' must be specified ")) - } - - /** - * Creates a new relation for data store in CSV given parameters. - * Parameters have to include 'path' and optionally 'delimiter', 'quote', and 'header' - */ - override def createRelation(sqlContext: SQLContext, - parameters: Map[String, String]): BaseRelation = { - createRelation(sqlContext, parameters, null) - } - - /** - * Creates a new relation for data store in CSV given parameters and user supported schema. - * Parameters have to include 'path' and optionally 'delimiter', 'quote', and 'header' - */ - override def createRelation(sqlContext: SQLContext, - parameters: Map[String, String], - schema: StructType): WebHdfsCsvRelation = { - // print("In Create Relation of DefaultSource" + "\n") - val path = checkPath(parameters) - val delimiter = WebHdfsTypeCast.toChar(parameters.getOrElse("delimiter", ",")) - - val quote = parameters.getOrElse("quote", "\"") - val quoteChar: Character = if (quote == null) { - null - } else if (quote.length == 1) { - quote.charAt(0) - } else { - throw new Exception("Quotation cannot be more than one character.") - } - - val escape = parameters.getOrElse("escape", null) - val escapeChar: Character = if (escape == null) { - null - } else if (escape.length == 1) { - escape.charAt(0) - } else { - throw new Exception("Escape character cannot be more than one character.") - } - - val comment = parameters.getOrElse("comment", "#") - val commentChar: Character = if (comment == null) { - null - } else if (comment.length == 1) { - comment.charAt(0) - } else { - throw new Exception("Comment marker cannot be more than one character.") - } - - val parseMode = parameters.getOrElse("mode", "PERMISSIVE") - - val useHeader = parameters.getOrElse("header", "false") - val headerFlag = if (useHeader == "true") { - true - } else if (useHeader == "false") { - false - } else { - throw new Exception("Header flag must be true or false") - } - - // val parserLib = parameters.getOrElse("parserLib", ParserLibs.DEFAULT) - val parserLib = parameters.getOrElse("parserLib", "COMMONS") - val ignoreLeadingWhiteSpace = parameters.getOrElse("ignoreLeadingWhiteSpace", "false") - val ignoreLeadingWhiteSpaceFlag = if (ignoreLeadingWhiteSpace == "false") { - false - } else if (ignoreLeadingWhiteSpace == "true") { - // if (!ParserLibs.isUnivocityLib(parserLib)) { - // throw new Exception("Ignore white space supported for Univocity parser only") - // } - true - } else { - throw new Exception("Ignore white space flag must be true or false") - } - - val ignoreTrailingWhiteSpace = parameters.getOrElse("ignoreTrailingWhiteSpace", "false") - val ignoreTrailingWhiteSpaceFlag = if (ignoreTrailingWhiteSpace == "false") { - false - } else if (ignoreTrailingWhiteSpace == "true") { - // if (!ParserLibs.isUnivocityLib(parserLib)) { - // throw new Exception("Ignore white space supported for the Univocity parser only") - // } - true - } else { - throw new Exception("Ignore white space flag can be true or false") - } - val treatEmptyValuesAsNulls = parameters.getOrElse("treatEmptyValuesAsNulls", "false") - val treatEmptyValuesAsNullsFlag = if (treatEmptyValuesAsNulls == "false") { - false - } else if (treatEmptyValuesAsNulls == "true") { - true - } else { - throw new Exception("Treat empty values as null flag can be true or false") - } - - val charset = parameters.getOrElse("charset", WebHdfsConnector.DEFAULT_CHARSET.name()) - // TODO validate charset? - - val inferSchema = parameters.getOrElse("inferSchema", "false") - val inferSchemaFlag = if (inferSchema == "false") { - false - } else if (inferSchema == "true") { - true - } else { - throw new Exception("Infer schema flag can be true or false") - } - val nullValue = parameters.getOrElse("nullValue", "") - val dateFormat = parameters.getOrElse("dateFormat", null) - val codec = parameters.getOrElse("codec", null) - val maxCharsPerColStr = parameters.getOrElse("maxCharsPerCol", "100000") - val maxCharsPerCol = try { - maxCharsPerColStr.toInt - } catch { - case e: Exception => throw new Exception("maxCharsPerCol must be a valid integer") - } - - val trustCredStr = parameters.getOrElse("certValidation", "") - val userCredStr = parameters.getOrElse("userCred", "") - if (userCredStr == "") { - throw new Exception("User Credential has to be set") - } - - val connPropStr = parameters.getOrElse("connProp", "10000:60000") - var partitionDetailsStr = parameters.getOrElse("partitions", "4:10000") - - if (!partitionDetailsStr.contains(":")) partitionDetailsStr = partitionDetailsStr + ":10000" - - val formatDetailsStr = parameters.getOrElse("format", "csv:\n") - val formatDetailsArr = formatDetailsStr.split(":") - var formatRecordSeparator = formatDetailsArr(1) - val outputTypeStr = parameters.getOrElse("output", "Data") - val formatType = formatDetailsArr(0) - if (formatType != "csv" && formatRecordSeparator == "") { - throw new Exception("Record Separator cannot be inferred for Format other than csv") - } - if (formatType == "csv" && formatRecordSeparator == "") formatRecordSeparator = "\n" - - val outRdd = (outputTypeStr : @switch) match { - case "LIST" => WebHdfsConnector.listFromWebHdfs(sqlContext.sparkContext, path, trustCredStr, - userCredStr, connPropStr) - case default => WebHdfsConnector.loadFromWebHdfs(sqlContext.sparkContext, path, charset, - trustCredStr, userCredStr, connPropStr, partitionDetailsStr, formatRecordSeparator) - } - - val targetSchema = (outputTypeStr : @switch) match { - case "LIST" => StructType(Array( - StructField("Name", StringType, true), - StructField("File Size", LongType, true), - StructField("Block Size", LongType, true), - StructField("# of Blocks", IntegerType, true))) - case default => schema - } - - val relation = (formatType : @switch) match { - case "csv" => - WebHdfsCsvRelation( - () => outRdd, - Some(path), - headerFlag, - delimiter, - quoteChar, - escapeChar, - commentChar, - parseMode, - parserLib, - ignoreLeadingWhiteSpaceFlag, - ignoreTrailingWhiteSpaceFlag, - treatEmptyValuesAsNullsFlag, - targetSchema, - inferSchemaFlag, - codec, - nullValue, - dateFormat, - maxCharsPerCol)(sqlContext) - case default => throw new Exception("Format Not Supported") - } - - relation - } - - override def createRelation(sqlContext: SQLContext, - mode: SaveMode, - parameters: Map[String, String], - data: DataFrame): BaseRelation = { - val path = checkPath(parameters) - val trustStoreCredStr = parameters.getOrElse("certValidation", "") - val connStr = parameters.getOrElse("connProp", "1000:10000") - val partitionStr = parameters.getOrElse("partitions", "4") - val formatStr = parameters.getOrElse("format", "csv") - val userCredStr = parameters.getOrElse("userCred", "") - if (userCredStr == "") { - throw new Exception("User Credentials cannot be null") - } - - // As of now only CSV format is supported - val rddToWrite = (formatStr : @switch) match { - case "csv" => WebHdfsCsvFormatter.convToCsvFormat(data, parameters) - case default => throw new Exception("Format Not Supported") - } - - WebHdfsConnector - .writeToWebHdfs(rddToWrite, path, trustStoreCredStr, connStr, userCredStr, partitionStr) - - createRelation(sqlContext, parameters, data.schema) - } -} diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala index 3cb62ae6..515b6578 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala @@ -17,472 +17,271 @@ package org.apache.bahir.datasource.webhdfs.util -import java.io._ import java.nio.charset.Charset -import java.security._ -import javax.net.ssl.{SSLContext, SSLSocketFactory, TrustManagerFactory} +import java.text.SimpleDateFormat +import java.sql.{Timestamp, Date} +import java.util.Date -import scala.annotation.switch -import scala.collection.mutable.ArrayBuffer -import scala.math._ -import scala.util.parsing.json.JSON - -import scalaj.http._ +import org.apache.spark.sql.types.{DateType, TimestampType} import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD +import scala.annotation.switch +import scalaj.http._ +import java.security._ +import javax.net.ssl.TrustManagerFactory +import java.io._ +import javax.net.ssl.SSLSocketFactory +import javax.net.ssl.SSLContext +import org.apache.http.conn.scheme.Scheme +import java.net.HttpURLConnection +import scala.collection.mutable.HashMap +import scala.math._ +import org.apache.spark.sql.types.{DateType, TimestampType} +import org.apache.spark.sql.{DataFrame, SQLContext} +import scala.collection.mutable.ArrayBuffer +import scala.util.control.Breaks._ + + /** - * This object contains all utility functions for reading/writing data from/to remote WebHDFS - * server. The abstraction maintained in this layer is at the level of RDD. - */ -private[webhdfs] object WebHdfsConnector { + * This object contains all utility functions for reading/writing data from/to remote webhdfs server. The abstraction maintained in this layer is at the level of RDD +*/ + +object WebHdfsConnector { + + + /** + * This function returns a Tuple for credential store which contains flag for validating Certificate, the Certificate File object and Certificate File Object password + + */ + + def createTrustStoreCredForExecutors(cred: String, path: String) : Tuple3[String, File, String] = { + + val trustStoreMap = if (cred != "") { + if (cred == "N") + new Tuple3("N", null, "") + else if(cred == "Y") + { + val tsd = SSLTrustStore.getCertDetails(path) + + new Tuple3("Y", tsd._1, tsd._2) + } + else + throw new Exception("Invalid Certificate Validation Option") + + } else { + new Tuple3("", null, "") + } + trustStoreMap + + + } + + + /** + * This function returns a SSLSocketFactory which needs to be used in HTTP connection library in case Certificate to be validated + */ + + def biocSslSocketFactory(fl: File, pswrd: String): SSLSocketFactory = { + + + val tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()) + val ks = KeyStore.getInstance("JKS") + val fis = new java.io.FileInputStream(fl) + ks.load(fis, pswrd.toCharArray()); + tmf.init(ks); + + + val sslc = SSLContext.getInstance("SSL") + + sslc.init(null, tmf.getTrustManagers(),null) + + sslc.getSocketFactory() + } + + /** + * This function returns the list of files in a folder with file details as RDD + + def listFromWebHdfs(sc: SparkContext, path: String, trustStoreCred: String, userCred: String, connProp: String): RDD[String] = { + + + + val conns = connProp.split(":") + + val conn = Array(conns(0).toInt, conns(1).toInt) + + val usrCred = userCred.split(":") + + val trustCred = createTrustStoreCredForExecutors(trustStoreCred, path) + + val fileDetails = getFilesDetails(path, trustCred, usrCred, conn) + + def g(v:Tuple4[String, Long, Long, Int]) = v._1.split("/").last + "," + v._2.toString + "," + v._3.toString + "," + v._4.toString + "\n" + + val fds = fileDetails.map(x => g(x)) + + val flRdd = sc.parallelize(fds) + + flRdd + + } + + * This function writes data back to hdfs using WebHDFS using multiple parallel connections. Right now file overwrite is not supported + + def writeToWebHdfs(dataToWrite: RDD[String] ,path: String, trustStoreCredStr: String, connStr : String, userCredStr : String, partitionStr : String): Unit = { + + val trustCred = createTrustStoreCredForExecutors(trustStoreCredStr, path) + + val conns = connStr.split(":") + + val conn = Array(conns(0).toInt, conns(1).toInt) + + val usr = userCredStr.split(":") + + val webHdfsChkDirOpr = "op=GETFILESTATUS" + val returnChkDir = callWebHdfsAPI(path, "", "GET", "CODE", trustCred, usr, conn, webHdfsChkDirOpr) + + if (returnChkDir == "200") + throw new Exception("The File Already Exists : " + path + "\n") + + val dPartitions = partitionStr.toInt + + val textRdd = dataToWrite.repartition(dPartitions) + + val webHdfsMakeDirOpr = "op=MKDIRS" + val returnCreateDir = callWebHdfsAPI(path, "", "PUT", "CODE", trustCred, usr, conn, webHdfsMakeDirOpr) + + textRdd.mapPartitionsWithIndex((idx, iter) => WebHdfsConnector.saveAllFiles(idx, iter, usr, path, trustCred, conn)).collect() + + } + + * This function is passed to mapPartitionsWithIndex so that each executor task can save part of the data using separate connection + + def saveAllFiles (idx: Int, data : Iterator[String], usrCred: Array[String], path: String, trustCred: Tuple3[String, File , String], connProp: Array[Int]): Iterator[String] = { + + + var dataP = data.next() + while(data.hasNext) { + dataP = dataP + "\n" + data.next() + } + + val fnameArray = path.split("/") + val fnameIdx = fnameArray.length - 1 + val fname = fnameArray(fnameIdx) + val filePath = s"$path/part-000$idx-$fname" + + val createOpr = "op=CREATE" + + val createUrl = callWebHdfsAPI(filePath, "", "PUT", "LOCATION", trustCred, usrCred, connProp, createOpr) + val created = callWebHdfsAPI(createUrl, dataP, "PUT", "CODE", trustCred, usrCred, connProp, createOpr) + + val ret = Array(created.toString) + ret.iterator + + } + */ + + + def callWebHdfsAPI(path: String, data: String, method: String, respType: String, trustStoreCredStr: String, usrCredStr: String, connStr: String, opr: String, outputType : String): Any = { + + //print("path in callWebHdfs : " + path + "\n") + + val pathComp = path.split(":") + + val trustCred = createTrustStoreCredForExecutors(trustStoreCredStr, path) + + val conns = connStr.split(":") + + val connProp = Array(conns(0).toInt, conns(1).toInt) + + val usrCred = usrCredStr.split(":") + + val uri = (if(trustCred._1 != "") "https:" else "http:") + pathComp(1) + ":" + pathComp(2) + "?" + opr + + var httpc = Http(uri).auth(usrCred(0), usrCred(1)).timeout(connTimeoutMs = connProp(0), readTimeoutMs = connProp(1)) + + httpc = (method : @switch) match { + case "GET" => httpc + case "PUT" => httpc.put(data).header("content-type", "application/csv") + case "POST" => httpc.postData(data).header("content-type", "application/csv") + } + + httpc = (trustCred._1 : @switch) match { + case "" => httpc + case "N" => httpc.option(HttpOptions.allowUnsafeSSL) + case "Y" => httpc.option(HttpOptions.sslSocketFactory(biocSslSocketFactory(trustCred._2, trustCred._3))) + } + + val out = (outputType : @switch) match { + case "" => httpc.asString + case "String" => httpc.asString + case "Bytes" => httpc.asBytes + } + + val resp = (respType : @switch) match { + case "BODY" => out.body + case "CODE" => out.code + case "HEADERS" => out.headers + case "LOCATION" => out.location.mkString(" ") + } + + /* + val resp = (respType : @switch) match { + case "BODY" => httpc.asBytes.body + case "CODE" => httpc.asString.code + case "HEADERS" => httpc.asString.headers + case "LOCATION" => httpc.asString.location.mkString(" ") + } + */ + + resp + + } + + def getFileInputStream (filePath : String, offset : Long, length : Long, trustStoreCredStr: String, connStr: String, usrCredStr : String): ByteArrayInputStream = { + + //print("path in getFileInputStream : " + filePath + "\n") + + val fileGetOpr = if(length > 0) s"op=OPEN&offset=$offset&length=$length&bufferSize=$length" else s"op=OPEN&offset=$offset" + + val getUrl = callWebHdfsAPI(filePath, "", "GET", "LOCATION", trustStoreCredStr, usrCredStr, connStr, fileGetOpr, "String").asInstanceOf[String] - /** - * Currently only files transferred using UTF-8 are supported - */ - val DEFAULT_CHARSET = Charset.forName("UTF-8") + val content = callWebHdfsAPI(getUrl, "", "GET", "BODY", trustStoreCredStr, usrCredStr, connStr, fileGetOpr, "Bytes").asInstanceOf[Array[Byte]] - /** - * This function prepares the partition details for each file based on the details populated by - * getFilesDetails. This partition details is further used to spawn multiple connections to get - * data of a file using multiple connections. - */ - def preparePartitions( - fileDetails: Array[(String, Long, Long, Int)], - baseFile: String, - partitionDetails: String, - recordSeparator: String): Array[(String, Long, Long, Int, Int, Int, String)] = { + new ByteArrayInputStream(content) + + } + + def getFileStatus(filePath: String, trustStoreCredStr : String, connStr: String, usrCredStr: String): Map[String, Any] = { - val totalFileCount = fileDetails.length - var i = 0 + //print("path in getFileStatus : " + filePath + "\n") + val fileStatusOpr = s"op=GETFILESTATUS" - val partitionDet = partitionDetails.split(":") + val fileStatus = callWebHdfsAPI(filePath, "", "GET", "BODY", trustStoreCredStr, usrCredStr, connStr, fileStatusOpr, "String").asInstanceOf[String] - // if number of partitions used for opening connections is passed as 0 or less, partition is - // defaulted to 4 - var filePartition = if (partitionDet(0).toInt < 1) 4 else partitionDet(0).toInt + if (fileStatus.contains("RemoteException")) + throw new Exception(fileStatus) - // if partition span (used to resolve record boundary) is sent as less than 10 KB, it is - // defaulted to 10 KB. Otherwise it is kept between 10KB to 100 KB - var partitionSpan = if (partitionDet(1).toInt < 10000) { - 10000 - } else { - math.min(partitionDet(1).toInt, 100000) - } + val responseMap = scala.util.parsing.json.JSON.parseFull(fileStatus).toList(0).asInstanceOf[Map[String, Map[String, Any]]] - i = 0 + responseMap.getOrElse("FileStatus", throw new Exception ("File Status Null")) - var partList = new ArrayBuffer[(String, Long, Long, Int, Int, Int, String)]() + } - var j = 0 - var k = 0 -// var filePart = 0 - var partPath = "" - var partLength = 0L - var fileLength = 0L - var partOffset = 0L -// var fileSpan = 0 + def getListStatus(filePath: String, trustStoreCredStr : String, connStr: String, usrCredStr: String): List[Map[String, Any]] = { -// val maxSpan = 1000000 -// val minSpan = 1000 - while (i < totalFileCount) { + //print("path in getListStatus : " + filePath + "\n") + val listStatusOpr = s"op=LISTSTATUS" - fileLength = fileDetails(i)._2.toLong + val listStatus = callWebHdfsAPI(filePath, "", "GET", "BODY", trustStoreCredStr, usrCredStr, connStr, listStatusOpr, "String").asInstanceOf[String] - if (fileLength > 0) { - - partPath = fileDetails(i)._1 - - fileLength = fileDetails(i)._2.toLong - partLength = fileLength / filePartition - - if (partLength < 1000000) { - filePartition = 1 - partitionSpan = 0 - } + if (listStatus.contains("RemoteException")) + throw new Exception(listStatus) - j = 0 - while (j < filePartition) { - partOffset = j * partLength.toLong + scala.util.parsing.json.JSON.parseFull(listStatus).toList(0).asInstanceOf[Map[String, Map[String, Any]]].get("FileStatuses").get("FileStatus").asInstanceOf[List[Map[String, Any]]] + } - if (j + 1 == filePartition) partLength = fileDetails(i)._2.toLong - j * partLength - - partList += new Tuple7(partPath, partOffset, partLength, j + 1, filePartition, - partitionSpan, recordSeparator) - j += 1 - k += 1 - } - } - - i += 1 - - } - - if (k < 1) { - throw new Exception("Zero File Content") - } - - var finalPartList = new Array[Tuple7[String, Long, Long, Int, Int, Int, String]](k) - - partList.copyToArray(finalPartList) - - finalPartList - } - - /** - * This function returns the list of files in a folder with file details as RDD - */ - def listFromWebHdfs(sc: SparkContext, - path: String, - trustStoreCred: String, - userCred: String, - connProp: String): RDD[String] = { - - val conns = connProp.split(":") - - val conn = Array(conns(0).toInt, conns(1).toInt) - - val usrCred = userCred.split(":") - - val trustCred = createTrustStoreCredForExecutors(trustStoreCred, path) - - val fileDetails = getFilesDetails(path, trustCred, usrCred, conn) - - def g(v: (String, Long, Long, Int)) = { - v._1.split("/").last + "," + v._2.toString + "," + v._3.toString + "," + v._4.toString + "\n" - } - - val fds = fileDetails.map(x => g(x)) - - val flRdd = sc.parallelize(fds) - - flRdd - } - - /** - * This function returns a Tuple for credential store which contains flag for validating - * Certificate, the Certificate File object and Certificate File Object password. - */ - def createTrustStoreCredForExecutors(cred: String, path: String): Tuple3[String, File, String] = { - - val trustStoreMap = if (cred != "") { - if (cred == "N") { - new Tuple3("N", null, "") - } - else { - if (cred == "Y") { - val tsd = SSLTrustStore.getCertDetails(path) - new Tuple3("Y", tsd._1, tsd._2) - } - else { - throw new Exception("Invalid Certificate Validation Option") - } - } - } else { - new Tuple3("", null, "") - } - trustStoreMap - } - - /** - * This function returns the details of the the files in a folder if the path passed is a folder. - * In case a File path is passed it returns the details of the files. - * Returns an Array of Tuple where each Tuple represents one file with details of full file path, - * size of the file, block size of the file and number of partitions based on size of the file and - * block size. - */ - def getFilesDetails(path: String, - trustCred: (String, File, String), - usrCred: Array[String], - connProp: Array[Int]): Array[(String, Long, Long, Int)] = { - - val listStatusOpr = s"op=LISTSTATUS" - - val listStatus = callWebHdfsAPI(path, "", "GET", "BODY", trustCred, usrCred, connProp, - listStatusOpr) - - if (listStatus.contains("RemoteException")) { - throw new Exception(listStatus) - } - - val flist = JSON - .parseFull(listStatus) - .toList(0) - .asInstanceOf[Map[String, Map[String, Any]]] - .get("FileStatuses") - .get("FileStatus") - .asInstanceOf[List[Map[String, Any]]] - - val fileCount = flist.length - - var i = 0 - var j = 0L - val fileDetails = new Array[(String, Long, Long, Int)](fileCount) - var fSuffix = "" - var fLength = 0L - var fBlocksize = 0L - var fPart = 0 - var fullFilePath = "" - - while (i < fileCount) { - fSuffix = flist(i).get("pathSuffix").getOrElse(path).asInstanceOf[String].toString - - fullFilePath = if (fSuffix == "") path else (path + "/" + fSuffix) - - fLength = flist(i).get("length").getOrElse(0).asInstanceOf[Double].toLong - - fBlocksize = flist(i).get("blockSize").getOrElse(0).asInstanceOf[Double].toLong - if (fLength > 0) fPart = (floor((fLength / fBlocksize)).toInt + 1) else fPart = 0 - fileDetails(i) = new Tuple4(fullFilePath, fLength, fBlocksize, fPart) - - i += 1 - } - fileDetails - } - - /** - * This function calls the WebHDFS API after creating all necessary parameters from different - * configurations - */ - def callWebHdfsAPI(path: String, - data: String, - method: String, - respType: String, - trustStoreCred: (String, File, String), - userCred: Array[String], - connProp: Array[Int], - opr: String): String = { - - val pathComp = path.split(":") - - val uri = (if (trustStoreCred._1 != "") "https:" else "http:") - .concat(pathComp(1) + ":" + pathComp(2) + "?" + opr) - - var httpc = Http(uri).auth(userCred(0), userCred(1)).timeout(connTimeoutMs = connProp(0), - readTimeoutMs = connProp(1)) - - httpc = (method: @switch) match { - case "GET" => httpc - case "PUT" => httpc.put(data).header("content-type", "application/csv") - case "POST" => httpc.postData(data).header("content-type", "application/csv") - } - - httpc = (trustStoreCred._1: @switch) match { - case "" => httpc - case "N" => httpc.option(HttpOptions.allowUnsafeSSL) - case "Y" => httpc.option(HttpOptions.sslSocketFactory(biocSslSocketFactory(trustStoreCred._2, - trustStoreCred._3))) - } - - val resp = (respType: @switch) match { - case "BODY" => httpc.asString.body - case "CODE" => httpc.asString.code - case "HEADERS" => httpc.asString.headers - case "LOCATION" => httpc.asString.location.mkString(" ") - } - - resp.toString() - } - - /** - * This function returns a SSLSocketFactory which needs to be used in HTTP connection library in - * case Certificate to be validated - */ - def biocSslSocketFactory(fl: File, pswrd: String): SSLSocketFactory = { - - val tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()) - val ks = KeyStore.getInstance("JKS") - val fis = new java.io.FileInputStream(fl) - ks.load(fis, pswrd.toCharArray()); - tmf.init(ks); - - val sslc = SSLContext.getInstance("SSL") - - sslc.init(null, tmf.getTrustManagers(), null) - - sslc.getSocketFactory() - } - - /** - * This function returns data of a file (or data of all files in a folder with same structure) as - * RDD - */ - def loadFromWebHdfs(sc: SparkContext, - path: String, - charset: String, - trustStoreCred: String, - userCred: String, - connProp: String, - partitionDetails: String, - recordSeparator: String): RDD[String] = { - - val conns = connProp.split(":") - - val conn = Array(conns(0).toInt, conns(1).toInt) - - val usrCrd = userCred.split(":") - - val trustCred = createTrustStoreCredForExecutors(trustStoreCred, path) - - val fileDetails = getFilesDetails(path, trustCred, usrCrd, conn) - - val parts = preparePartitions(fileDetails, path, partitionDetails, recordSeparator) - - val input = sc.parallelize(parts, parts.length) - - input.collect() - - val fRdd = input.flatMap(x => WebHdfsConnector.getAllFiles(x, usrCrd, trustCred, conn)) - - fRdd - } - - /** - * This function is passed to each executor through flatMap function to spawn one http connection - * from each executor for get a part of the file - */ - def getAllFiles(partInfo: (String, Long, Long, Int, Int, Int, String), - usrCred: Array[String], - trustCred: (String, File, String), - connProp: Array[Int]): Iterator[String] = { - - val foffset = partInfo._2.toLong - val flength = partInfo._3.toLong - val ffilePath = partInfo._1 - val fpartNum = partInfo._4 - val ftotalPart = partInfo._5 - val fspan = partInfo._6 - val frecordSeparator = partInfo._7 - - val fileGetOpr = if (fpartNum < ftotalPart) { - val effLength = flength + fspan - s"op=OPEN&offset=$foffset&length=$effLength&bufferSize=$effLength" - } - else { - s"op=OPEN&offset=$foffset&length=$flength&bufferSize=$flength" - } - - val getUrl = callWebHdfsAPI(ffilePath, "", "GET", "LOCATION", trustCred, usrCred, connProp, - fileGetOpr) - val partContent = callWebHdfsAPI(getUrl, "", "GET", "BODY", trustCred, usrCred, connProp, - fileGetOpr) - - val records = getTillEndOfRecord(partContent, flength, fpartNum, ftotalPart, frecordSeparator) - - records.split("\n").iterator - } - - /** - * This function resolves record boundaries. - * Right now this only supports "\n" as record boundary . This function has to be refined to - * support json or xml formats for different type of record separators - */ - def getTillEndOfRecord(content: String, - partLength: Long, - partNum: Int, - totalPart: Int, - recordSeparator: String): String = { - - val contentBytes = content.getBytes("UTF-8") -// val recordSeparatorBytes = recordSeparator.getBytes("UTF-8") - - val contentBytesLength = contentBytes.length - - var bytePosition = 0 - - var startbyte = 0 - - startbyte = if (partNum == 1) 0 - else { - // TODO: This part of the code has to be rewritten later on to make it more generic for - // supporting other formats apart from csv. Right now it supports only csv - while (contentBytes(bytePosition) != '\n') bytePosition += 1 - bytePosition - } - - val length = if (partNum == totalPart) (contentBytesLength.toInt - startbyte).toInt - else { - bytePosition = partLength.toInt - // TODO: This part of the code has to be rewritten later on to make it more generic for - // supporting other formats apart from csv. Right now it supports only csv. - while (contentBytes(bytePosition) != '\n') bytePosition += 1 - (bytePosition - startbyte) - } - - new String(contentBytes, startbyte, length, "UTF-8") - } - - /** - * This function writes data back to hdfs using WebHDFS using multiple parallel connections. - * Right now file overwrite is not supported - */ - def writeToWebHdfs(dataToWrite: RDD[String], - path: String, - trustStoreCredStr: String, - connStr: String, - userCredStr: String, - partitionStr: String): Unit = { - - val trustCred = createTrustStoreCredForExecutors(trustStoreCredStr, path) - - val conns = connStr.split(":") - - val conn = Array(conns(0).toInt, conns(1).toInt) - - val usr = userCredStr.split(":") - - val webHdfsChkDirOpr = "op=GETFILESTATUS" - val returnChkDir = callWebHdfsAPI(path, "", "GET", "CODE", trustCred, usr, conn, - webHdfsChkDirOpr) - - if (returnChkDir == "200") { - throw new Exception("The File Already Exists : " + path + "\n") - } - - val dPartitions = partitionStr.toInt - - val textRdd = dataToWrite.repartition(dPartitions) - - val webHdfsMakeDirOpr = "op=MKDIRS" - val returnCreateDir = callWebHdfsAPI(path, "", "PUT", "CODE", trustCred, usr, conn, - webHdfsMakeDirOpr) - - textRdd.mapPartitionsWithIndex((idx, iter) => WebHdfsConnector.saveAllFiles(idx, iter, usr, - path, trustCred, conn)).collect() - } - - /** - * This function is passed to mapPartitionsWithIndex so that each executor task can save part of - * the data using separate connection - */ - def saveAllFiles(idx: Int, - data: Iterator[String], - usrCred: Array[String], - path: String, - trustCred: (String, File, String), - connProp: Array[Int]): Iterator[String] = { - - var dataP = data.next() - while (data.hasNext) { - dataP = dataP + "\n" + data.next() - } - - val fnameArray = path.split("/") - val fnameIdx = fnameArray.length - 1 - val fname = fnameArray(fnameIdx) - val filePath = s"$path/part-000$idx-$fname" - - val createOpr = "op=CREATE" - - val createUrl = callWebHdfsAPI(filePath, "", "PUT", "LOCATION", trustCred, usrCred, connProp, - createOpr) - val created = callWebHdfsAPI(createUrl, dataP, "PUT", "CODE", trustCred, usrCred, connProp, - createOpr) - - val ret = Array(created.toString) - ret.iterator - } } diff --git a/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfs.java b/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfs.java new file mode 100644 index 00000000..4daa11ca --- /dev/null +++ b/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfs.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdfs.web; + +import java.net.URL; +import java.net.URI; + +import java.io.IOException; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.web.resources.Param; +import org.apache.hadoop.hdfs.web.resources.HttpOpParam.Op; + +import org.apache.hadoop.conf.Configuration; + +/** + * A FileSystem for HDFS over the web, extending [[org.apache.hadoop.hdfs.web.WebHdfsFileSystem]] + * to allow secure authentication and configurable gateway path segments. + * + * TODO: upgrade Hadoop version (Maven dependencies) to override SWebHdfsFileSystem + */ +class BahirWebHdfs extends WebHdfsFileSystem { + + // TODO: figure out how/where to authenticate + + // TODO: gateway path should be configurable + private String gatewayPath = "/gateway/default"; + + private URI uri = null; + private URI rHdfsUri = null; + private Configuration conf = null; + + /* + @Override + public void initialize( + URI uriOrg, + Configuration confOrg) throws IOException { + + super.initialize(uriOrg, confOrg); + + uri = URI.create(uriOrg.getScheme() + "://" + uriOrg.getAuthority()) ; + System.out.println("uri in initialize : "+ uri + "\n"); + + } + */ + + @Override + URL toUrl(final Op op, final Path fspath, final Param... parameters) throws IOException { + URL url = super.toUrl(op, fspath, parameters); + url = new URL("http://www.google.com"); + + System.out.print("url in overridden toUrl: " + url + "\n"); + return url; + + //Url newUrl = new URL("https", url.getHost, url.getPort, + //url.getFile.replaceFirst(WebHdfsFileSystem.PATH_PREFIX, + // gatewayPath + WebHdfsFileSystem.PATH_PREFIX)) + } + +} diff --git a/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfsFileSystem.scala b/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfsFileSystem.scala index 2b64dd9b..f817b2ee 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfsFileSystem.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfsFileSystem.scala @@ -18,11 +18,14 @@ package org.apache.hadoop.hdfs.web import java.net.URL +import java.net.URI import org.apache.hadoop.fs.Path import org.apache.hadoop.hdfs.web.resources.HttpOpParam.Op import org.apache.hadoop.hdfs.web.resources.Param +import org.apache.hadoop.conf.Configuration + /** * A FileSystem for HDFS over the web, extending [[org.apache.hadoop.hdfs.web.WebHdfsFileSystem]] * to allow secure authentication and configurable gateway path segments. @@ -36,13 +39,31 @@ class BahirWebHdfsFileSystem extends WebHdfsFileSystem { // TODO: gateway path should be configurable val gatewayPath = "/gateway/default" + var uri:URI = null + var rHdfsUri:URI = null + var conf:Configuration = null + + override def initialize( + uriOrg: URI, + confOrg: Configuration): Unit = { + + super.initialize(uriOrg, confOrg) + + uri = URI.create(uriOrg.getScheme() + "://" + uriOrg.getAuthority()+"/gateway/default") + print("uri in initialize : " + uri + "\n") + + } + + //override def toUrl(op: HttpOpParam.Op, fspath: Path, parameters: Param[_, _]*): URL = { + override def toUrl(op: Op, fspath: Path, parameters: Param[_, _]*): URL = { + var url = super.toUrl(op, fspath, parameters: _*) + url = new URL("http://www.google.com") - override def toUrl(op: Op, fspath: Path, parameters: Param[_, _]*): URL = { - val url = super.toUrl(op, fspath, parameters: _*) + print("url in overriden toUrl: " + url + "\n") - new URL(url.getProtocol, url.getHost, url.getPort, - url.getFile.replaceFirst(WebHdfsFileSystem.PATH_PREFIX, - gatewayPath + WebHdfsFileSystem.PATH_PREFIX)) - } + new URL("https", url.getHost, url.getPort, + url.getFile.replaceFirst(WebHdfsFileSystem.PATH_PREFIX, + gatewayPath + WebHdfsFileSystem.PATH_PREFIX)) + } } From 2932f99244b23732ff1a6306f7e88e42c72a3360 Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Wed, 7 Dec 2016 14:01:50 -0800 Subject: [PATCH 13/24] [BAHIR-75][WIP] - override WebHdfsFileSystem - code style fixes, remove unused files --- .../webhdfs/BahirWebHdfsDataSetWrapper.scala | 584 +++++++++--------- .../webhdfs/csv/WebHdfsCsvFormatter.scala | 142 ----- .../webhdfs/csv/WebHdfsCsvParserLibs.scala | 50 -- .../webhdfs/csv/WebHdfsCsvRelation.scala | 292 --------- .../apache/hadoop/hdfs/web/BahirWebHdfs.java | 75 --- .../hdfs/web/BahirWebHdfsFileSystem.scala | 69 --- 6 files changed, 290 insertions(+), 922 deletions(-) delete mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvFormatter.scala delete mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvParserLibs.scala delete mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvRelation.scala delete mode 100644 datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfs.java delete mode 100644 datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfsFileSystem.scala diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala index dc755b85..6353f620 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala @@ -17,349 +17,345 @@ package org.apache.bahir.datasource.webhdfs -import java.net.URI -import java.net.URL import java.io._ - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FSInputStream; -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.fs.FileStatus -import org.apache.hadoop.fs.permission.FsPermission -import org.apache.hadoop.util.Progressable +import java.net.URI import scala.collection.mutable.HashMap import scala.math._ -import org.apache.bahir.datasource.webhdfs.util._ -import org.apache.bahir.datasource.webhdfs.csv._ - - -/** - * This class contains functions for reading/writing data from/to remote webhdfs server in Spark DataSource -*/ - - - -class BahirWebHdfsFileSystem - extends FileSystem { - - var uri:URI = null - var rHdfsUri:URI = null - var conf:Configuration = null - var workingDir = null - - var readFullFile = false - var usrCred = "" - var connections = 0 - var certValidation = "Y" - - var fileStatusMap : HashMap[String, FileStatus] = HashMap() - var listStatusMap : HashMap[String, Array[FileStatus]] = HashMap() - - override def getUri() : URI = uri - - - /** - * This method does necessary initialization of the configuration parameters - */ - - override def initialize( - uriOrg: URI, - confOrg: Configuration): Unit = { - - super.initialize(uriOrg, confOrg) - - setConf(confOrg) - - rHdfsUri = uriOrg - conf = confOrg - - val rfFlg = conf.get("readFullFile") - readFullFile = if(rfFlg == null) false else rfFlg.toBoolean - - val usrCredStr = conf.get("usrCredStr") - usrCred = if(usrCredStr == null) throw new Exception ("User Credential Has To Be Specified For The Remote HDFS") else usrCredStr.toString - - val conns = conf.get("connections") - connections = if(conns == null) 0 else conns.toInt - - val certFlg = conf.get("certValidationFlg") - certValidation = if(certFlg == null) "Y" else certFlg.toString - - uri = URI.create(uriOrg.getScheme() + "://" + uriOrg.getAuthority()) - //print("uri : ", uri + " , connections : " + connections + " , user cred : " + usrCred + "\n") - - } - - override def getWorkingDirectory() : Path = { - - val path = new Path(rHdfsUri) - //print("Working Directory : " + path + "\n") - path - } - - override def setWorkingDirectory(dir : Path) : Unit = {} - - override def rename(srcPath : Path, destPath : Path) : Boolean = { - throw new Exception("File Rename Not Supported") - } - - override def delete(srcPath : Path, recursive : Boolean) : Boolean = { - throw new Exception("File Delete Not Supported") - } - - override def mkdirs(srcPath : Path, permission : FsPermission) : Boolean = { - throw new Exception("Make Directory Not Supported") - } - - override def append(srcPath : Path, bufferSize : Int, progress : Progressable) : FSDataOutputStream = { - throw new Exception("File Append Not Supported") - } - - override def getFileStatus(f : Path) : FileStatus = { - - val file = stripQryFromFilePath(f).toString - - var fStatus : FileStatus = fileStatusMap.getOrElse(file, null) - - val fileStatus = if (fStatus == null) { - val fStatusMap = WebHdfsConnector.getFileStatus(file, certValidation, "1000:5000", usrCred) - fStatus = createFileStatus(f, fStatusMap) - fileStatusMap.put(f.toString, fStatus) - fStatus - } - else - fStatus - - fileStatus - } - - override def listStatus(f : Path) : Array[FileStatus] = { - - val file = stripQryFromFilePath(f).toString - - var lStatus : Array[FileStatus] = listStatusMap.getOrElse(file, null) - - //print("file in listStatus: " + file + "\n") - - val listStatus = if(lStatus == null) { +import org.apache.bahir.datasource.webhdfs.util.WebHdfsConnector +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ +import org.apache.hadoop.fs.permission.FsPermission +import org.apache.hadoop.util.Progressable - val fStatusMapList = WebHdfsConnector.getListStatus(file, certValidation, "1000:5000", usrCred) - val fileCount = fStatusMapList.length - lStatus = new Array[FileStatus](fileCount) +/* + * This class contains functions for reading/writing data from/to remote webhdfs server in Spark + * DataSource + */ +class BahirWebHdfsFileSystem extends FileSystem { - var i = 0 + var uri: URI = null + var rHdfsUri: URI = null + var conf: Configuration = null + var workingDir = null - while(i < fileCount) - { - lStatus(i) = createFileStatus(f, fStatusMapList(i)) - i+=1 - } + var readFullFile = false + var usrCred = "" + var connections = 0 + var certValidation = "Y" - listStatusMap.put(f.toString, lStatus) - lStatus - } - else - lStatus + var fileStatusMap: HashMap[String, FileStatus] = HashMap() + var listStatusMap: HashMap[String, Array[FileStatus]] = HashMap() + override def getUri(): URI = uri - //print(" listStatus: " + listStatus + "\n") - listStatus - } - override def open(f: Path, bs: Int) : FSDataInputStream = { + /** + * This method does necessary initialization of the configuration parameters + */ + override def initialize(uriOrg: URI, + confOrg: Configuration): Unit = { - val fileStatus = getFileStatus(f) - val blockSize = fileStatus.getBlockSize - val fileLength = fileStatus.getLen + super.initialize(uriOrg, confOrg) - val file = stripQryFromFilePath(f) + setConf(confOrg) - print("file uri in open : " + file + "\n") + rHdfsUri = uriOrg + conf = confOrg - val qMap = getQryMapFromFilePath(f) + val rfFlg = conf.get("readFullFile") + readFullFile = if (rfFlg == null) false else rfFlg.toBoolean - val fConnections = if(qMap == null) connections - else - qMap.getOrElse("connections", connections).asInstanceOf[String].toInt + val usrCredStr = conf.get("usrCredStr") + usrCred = if (usrCredStr == null) { + throw new Exception("User Credential Has To Be Specified For The Remote HDFS") + } else usrCredStr.toString - new FSDataInputStream(new BahirWebHdfsInputStream(file, bs, blockSize, fileLength, readFullFile, usrCred, fConnections, certValidation)) + val conns = conf.get("connections") + connections = if (conns == null) 0 else conns.toInt - } + val certFlg = conf.get("certValidationFlg") + certValidation = if (certFlg == null) "Y" else certFlg.toString - override def create(srcPath : Path, permission : FsPermission, flag : Boolean, bufferSize : Int, replication : Short, blockSize : Long, progress : Progressable) : FSDataOutputStream = { - throw new Exception("File Create Not Yet Supported") - } + uri = URI.create(uriOrg.getScheme() + "://" + uriOrg.getAuthority()) + // println("uri: ", uri + " , connections: " + connections + " , user cred: " + usrCred) + + } + override def getWorkingDirectory(): Path = { + + val path = new Path(rHdfsUri) + // println("Working Directory: " + path) + path + } + + override def setWorkingDirectory(dir: Path): Unit = {} + + override def rename(srcPath: Path, destPath: Path): Boolean = { + throw new Exception("File Rename Not Supported") + } - private def createFileStatus(fPath : Path, statusMap: Map[String, Any]) : FileStatus = { - - val lng = conf.get("length") - val partlng = if(lng == null) 1 else lng.toInt + override def delete(srcPath: Path, recursive: Boolean): Boolean = { + throw new Exception("File Delete Not Supported") + } - val blk = conf.get("block") - val partblk = if(blk == null) 1 else blk.toInt + override def mkdirs(srcPath: Path, permission: FsPermission): Boolean = { + throw new Exception("Make Directory Not Supported") + } + override def append(srcPath: Path, + bufferSize: Int, + progress: Progressable): FSDataOutputStream = { + throw new Exception("File Append Not Supported") + } - val isDirFlg = if(statusMap.getOrElse("type", "") == "DIRECTORY") true else false - val pathSuffix = statusMap.getOrElse("pathSuffix", "") - val targetPath = if(pathSuffix == "") fPath else new Path(fPath.toString + "/" + pathSuffix) - val fStatus = new FileStatus( - statusMap.getOrElse("length", 0).asInstanceOf[Double].toLong*partlng, - isDirFlg, - statusMap.getOrElse("replication",1).asInstanceOf[Double].toInt, - (statusMap.getOrElse("blockSize", 128000000).asInstanceOf[Double].toLong)/partblk, - statusMap.getOrElse("modificationTime",0).asInstanceOf[Double].toLong, - statusMap.getOrElse("accessTime",0).asInstanceOf[Double].toLong, - null, - statusMap.getOrElse("owner", "default").asInstanceOf[String], - statusMap.getOrElse("group", "default").asInstanceOf[String], - null, targetPath) - fStatus - } + override def getFileStatus(f: Path): FileStatus = { + val file = stripQryFromFilePath(f).toString + var fStatus: FileStatus = fileStatusMap.getOrElse(file, null) - private def stripQryFromFilePath(f : Path) : Path = { + val fileStatus = if (fStatus == null) { + val fStatusMap = WebHdfsConnector.getFileStatus(file, certValidation, "1000:5000", usrCred) + fStatus = createFileStatus(f, fStatusMap) + fileStatusMap.put(f.toString, fStatus) + fStatus + } + else { + fStatus + } - //print("file uri : " + f.toUri + "\n") + fileStatus + } - val pathStrWithoutQry = f.toString.replace(getQryStrFromFilePath(f), "") - new Path(pathStrWithoutQry) + override def listStatus(f: Path): Array[FileStatus] = { - } + val file = stripQryFromFilePath(f).toString - private def getQryMapFromFilePath(f : Path) : HashMap[String, String] = { + var lStatus: Array[FileStatus] = listStatusMap.getOrElse(file, null) + + // println("file in listStatus: " + file) - val qryStr = getQryStrFromFilePath(f) - if(qryStr == "") null - else { + val listStatus = if (lStatus == null) { + val fStatusMapList = WebHdfsConnector + .getListStatus(file, certValidation, "1000:5000", usrCred) + val fileCount = fStatusMapList.length + lStatus = new Array[FileStatus](fileCount) + var i = 0 + while (i < fileCount) { + lStatus(i) = createFileStatus(f, fStatusMapList(i)) + i += 1 + } + listStatusMap.put(f.toString, lStatus) + lStatus + } + else { + lStatus + } - val params = qryStr.replace(";", "").substring(1).split("&") + // println(" listStatus: " + listStatus) + listStatus + } - val paramCount = params.length + private def createFileStatus(fPath: Path, statusMap: Map[String, Any]): FileStatus = { - //print("params : " + params + " , lenth : " + paramCount + "\n") - var paramMap : HashMap[String, String] = new HashMap() + val lng = conf.get("length") + val partlng = if (lng == null) 1 else lng.toInt - var i = 0 + val blk = conf.get("block") + val partblk = if (blk == null) 1 else blk.toInt - while(i < paramCount) - { - val paramKV = params(i).split("=") - paramMap.put(paramKV(0), paramKV(1)) - i+=1 - } - //print("param map : " + paramMap + "\n") - paramMap - } + val isDirFlg = if (statusMap.getOrElse("type", "") == "DIRECTORY") true else false + val pathSuffix = statusMap.getOrElse("pathSuffix", "") + val targetPath = if (pathSuffix == "") fPath else new Path(fPath.toString + "/" + pathSuffix) + val fStatus = new FileStatus( + statusMap.getOrElse("length", 0).asInstanceOf[Double].toLong * partlng, + isDirFlg, + statusMap.getOrElse("replication", 1).asInstanceOf[Double].toInt, + (statusMap.getOrElse("blockSize", 128000000).asInstanceOf[Double].toLong) / partblk, + statusMap.getOrElse("modificationTime", 0).asInstanceOf[Double].toLong, + statusMap.getOrElse("accessTime", 0).asInstanceOf[Double].toLong, + null, + statusMap.getOrElse("owner", "default").asInstanceOf[String], + statusMap.getOrElse("group", "default").asInstanceOf[String], + null, targetPath) + fStatus + } + + private def stripQryFromFilePath(f: Path): Path = { + // println("file uri : " + f.toUri) + val pathStrWithoutQry = f.toString.replace(getQryStrFromFilePath(f), "") + new Path(pathStrWithoutQry) + } + + private def getQryStrFromFilePath(f: Path): String = { + val fileStr = f.toString + val start = fileStr.indexOf("&") + val end = fileStr.indexOf(";") + + // print("start and end index " + start +"\n") + + val qryStr = if (start > 0) fileStr.substring(start, end) else "" + + // print("query : " + qryStr + "\n") + qryStr + } + + override def open(f: Path, bs: Int): FSDataInputStream = { + + val fileStatus = getFileStatus(f) + val blockSize = fileStatus.getBlockSize + val fileLength = fileStatus.getLen + + val file = stripQryFromFilePath(f) + + print("file uri in open : " + file + "\n") + + val qMap = getQryMapFromFilePath(f) + + val fConnections = if (qMap == null) { + connections + } + else { + qMap.getOrElse("connections", connections).asInstanceOf[String].toInt + } - } + new FSDataInputStream(new BahirWebHdfsInputStream(file, bs, blockSize, fileLength, + readFullFile, usrCred, fConnections, certValidation)) + } - private def getQryStrFromFilePath(f : Path) : String = { + override def create(srcPath: Path, + permission: FsPermission, + flag: Boolean, + bufferSize: Int, + replication: Short, + blockSize: Long, + progress: Progressable): FSDataOutputStream = { + throw new Exception("File Create Not Yet Supported") + } + + private def getQryMapFromFilePath(f: Path): HashMap[String, String] = { + + val qryStr = getQryStrFromFilePath(f) + if (qryStr == "") null + else { + + val params = qryStr.replace(";", "").substring(1).split("&") + + val paramCount = params.length - val fileStr = f.toString + // print("params : " + params + " , lenth : " + paramCount + "\n") + var paramMap: HashMap[String, String] = new HashMap() - val start = fileStr.indexOf("&") - val end = fileStr.indexOf(";") + var i = 0 - //print("start and end index " + start +"\n") + while (i < paramCount) { + val paramKV = params(i).split("=") + paramMap.put(paramKV(0), paramKV(1)) + i += 1 + } - val qryStr = if (start > 0) fileStr.substring(start, end) else "" + // print("param map : " + paramMap + "\n") + paramMap + } - //print("query : " + qryStr + "\n") - qryStr - } + } } -class BahirWebHdfsInputStream (fPath : Path, bufferSz : Int, blockSz : Long, fileSz : Long, readFull : Boolean, usrCrd : String, conns : Int, certValidation : String) +class BahirWebHdfsInputStream(fPath: Path, + bufferSz: Int, + blockSz: Long, + fileSz: Long, + readFull: Boolean, + usrCrd: String, + conns: Int, + certValidation: String) extends FSInputStream { - val filePath: Path = fPath - val bufferSize: Int = bufferSz - val blockSize: Long = blockSz - val fileSize: Long = fileSz - val readFullFlg: Boolean = readFull - val usrCred: String = usrCrd - val connections: Int = conns - val certValidationFlg: String = certValidation - - var pos = 0L - - var in : ByteArrayInputStream = null - - var callCount = 0 - - - override def read(b: Array[Byte], offset: Int, length: Int) : Int = { - - if (in == null) createWebHdfsInputStream(pos) - - callCount+=1 - - var bCount = in.read(b, offset, length) - - //print("In read - call count : " + callCount + " , pos : " + pos + ", offset : " + offset + " , length : " + length + " , byte count total : " + bCount + "\n") - - if (bCount < 0 && pos < fileSize) - { - //print("In read - bCount less than 0 , call count : " + callCount + " , file size : " + fileSize + " , pos : " + pos + ", offset : " + offset + " , length : " + length + " , byte count total : " + bCount + "\n") - //createWebHdfsInputStream(pos) - seek(pos) - bCount = in.read(b, offset, length) - } - - pos+=bCount - - bCount - - } - - /** - * This is a dummy implementation as Spark does not use it. We need it here just to satisy interface contract - */ - - override def read() = { - - read(new Array[Byte](4056), 0, 100) - } - - /** - * This is a dummy implementation as Spark does not use it. We need it here just to satisy interface contract - */ - - override def seekToNewSource(targetPos: Long) = false - - override def getPos() = { - pos - } - - - override def seek(newPos : Long) = { - //print("In seek - newpos : " + newPos + " , old pos : " + pos + "\n") - if (pos != newPos) { - pos = newPos - if (in != null) in.close - } - createWebHdfsInputStream(pos) - } - - private def createWebHdfsInputStream(pos : Long) = { - - val poe = if(connections == 0) - if(blockSize > fileSize || readFullFlg == true) fileSize else (floor(pos/blockSize).toLong + 1)*blockSize + 10000 - else - floor(fileSize/(connections - 1)).toInt + 10000 - - - //print("In read - input stream null , block size : " + blockSize + " , file size : " + fileSize + " , red full flg : " + readFullFlg + " , pos : " + pos + " , poe : " + poe +"\n") - - in = WebHdfsConnector.getFileInputStream(filePath.toString(), pos, poe, certValidationFlg, "1000:50000", usrCred) - } + val filePath: Path = fPath + val bufferSize: Int = bufferSz + val blockSize: Long = blockSz + val fileSize: Long = fileSz + val readFullFlg: Boolean = readFull + val usrCred: String = usrCrd + val connections: Int = conns + val certValidationFlg: String = certValidation + + var pos = 0L + + var in: ByteArrayInputStream = null + + var callCount = 0 + + /* + * This is a dummy implementation as Spark does not use it. We need it here just to satisy + * interface contract + */ + override def read(): Int = { + read(new Array[Byte](4056), 0, 100) + } + + override def read(b: Array[Byte], offset: Int, length: Int): Int = { + if (in == null) createWebHdfsInputStream(pos) + callCount += 1 + var bCount = in.read(b, offset, length) + +// println("In read - call count: " + callCount + " , pos: " + pos + ", offset: " + offset + +// ", length: " + length + ", byte count total: " + bCount) + + if (bCount < 0 && pos < fileSize) { +// println("In read - bCount less than 0, call count: " + callCount + ", file size : " + +// fileSize + " , pos : " + pos + ", offset : " + offset + " , length : " + length + +// " , byte count total : " + bCount) + // createWebHdfsInputStream(pos) + seek(pos) + bCount = in.read(b, offset, length) + } + + pos += bCount + + bCount + + } + + override def seek(newPos: Long): Unit = { + // print("In seek - newpos : " + newPos + " , old pos : " + pos + "\n") + if (pos != newPos) { + pos = newPos + if (in != null) in.close + } + createWebHdfsInputStream(pos) + } + + private def createWebHdfsInputStream(pos: Long) = { + + val poe = if (connections == 0) { + if (blockSize > fileSize || readFullFlg == true) { + fileSize + } else { + (floor(pos / blockSize).toLong + 1) * blockSize + 10000 + } + } + else { + floor(fileSize / (connections - 1)).toInt + 10000 + } + + + // println("In read - input stream null , block size : " + blockSize + " , file size : " + + // fileSize + " , red full flg : " + readFullFlg + " , pos : " + pos + " , poe : " + poe +"\n") + + in = WebHdfsConnector + .getFileInputStream(filePath.toString(), pos, poe, certValidationFlg, "1000:50000", usrCred) + } + + /* + * This is a dummy implementation as Spark does not use it. We need it here just to satisy + * interface contract + */ + override def seekToNewSource(targetPos: Long): Boolean = false + + override def getPos(): Long = pos } diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvFormatter.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvFormatter.scala deleted file mode 100644 index 60fad882..00000000 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvFormatter.scala +++ /dev/null @@ -1,142 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.bahir.datasource.webhdfs.csv - -import java.sql.Timestamp -import java.text.SimpleDateFormat - -import org.apache.commons.csv.{CSVFormat, QuoteMode} - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql._ -import org.apache.spark.sql.types._ - - -/** - * This object contains all utility functions for converting data to CSV format - * This is copied from com.databricks.spark.csv as the required object could not be reused as it is - * declared as private - */ -private[webhdfs] object WebHdfsCsvFormatter { - - def convToCsvFormat(dataFrame: DataFrame, - parameters: Map[String, String] = Map()) : RDD[String] = { - val delimiter = parameters.getOrElse("delimiter", ",") - val dateFormat = parameters.getOrElse("dateFormat", "yyyy-MM-dd HH:mm:ss.S") - val dateFormatter: SimpleDateFormat = new SimpleDateFormat(dateFormat) - - val delimiterChar = if (delimiter.length == 1) { - delimiter.charAt(0) - } else { - throw new Exception("Delimiter cannot be more than one character.") - } - - val escape = parameters.getOrElse("escape", null) - val escapeChar: Character = if (escape == null) { - null - } else if (escape.length == 1) { - escape.charAt(0) - } else { - throw new Exception("Escape character cannot be more than one character.") - } - - val quote = parameters.getOrElse("quote", "\"") - val quoteChar: Character = if (quote == null) { - null - } else if (quote.length == 1) { - quote.charAt(0) - } else { - throw new Exception("Quotation cannot be more than one character.") - } - - val quoteModeString = parameters.getOrElse("quoteMode", "MINIMAL") - val quoteMode: QuoteMode = if (quoteModeString == null) { - null - } else { - QuoteMode.valueOf(quoteModeString.toUpperCase) - } - - val nullValue = parameters.getOrElse("nullValue", "null") - - val csvFormat = CSVFormat.DEFAULT - .withDelimiter(delimiterChar) - .withQuote(quoteChar) - .withEscape(escapeChar) - .withQuoteMode(quoteMode) - .withSkipHeaderRecord(false) - .withNullString(nullValue) - - val generateHeader = parameters.getOrElse("header", "false").toBoolean - val header = if (generateHeader) { - csvFormat.format(dataFrame.columns.map(_.asInstanceOf[AnyRef]): _*) - } else { - "" // There is no need to generate header in this case - } - - val schema = dataFrame.schema - val formatForIdx = schema.fieldNames.map(fname => schema(fname).dataType match { - case TimestampType => (timestamp: Any) => { - if (timestamp == null) { - nullValue - } else { - dateFormatter.format(new java.sql.Date(timestamp.asInstanceOf[Timestamp].getTime)) - } - } - case DateType => (date: Any) => { - if (date == null) nullValue else dateFormatter.format(date) - } - case _ => (fieldValue: Any) => fieldValue.asInstanceOf[AnyRef] - }) - - val strRDD = dataFrame.rdd.mapPartitionsWithIndex { case (index, iter) => - val csvFormat = CSVFormat.DEFAULT - .withDelimiter(delimiterChar) - .withQuote(quoteChar) - .withEscape(escapeChar) - .withQuoteMode(quoteMode) - .withSkipHeaderRecord(false) - .withNullString(nullValue) - - new Iterator[String] { - var firstRow: Boolean = generateHeader - - override def hasNext: Boolean = iter.hasNext || firstRow - - override def next: String = { - if (iter.nonEmpty) { - // try .zipWithIndex.foreach - val values: Seq[AnyRef] = iter.next().toSeq.zipWithIndex.map { - case (fieldVal, i) => formatForIdx(i)(fieldVal) - } - val row = csvFormat.format(values: _*) - if (firstRow) { - firstRow = false - header + "\n" + row - } else { - row - } - } else { - firstRow = false - header - } - } - } - } - strRDD - } -} diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvParserLibs.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvParserLibs.scala deleted file mode 100644 index 5c873404..00000000 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvParserLibs.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.bahir.datasource.webhdfs.csv - - -/** - * This object contains all utility functions needed for determining CSV parser lib - * This is copied from com.databricks.spark.csv.util as the required object could not be reused as - * it is declared as private - */ -private[webhdfs] object WebHdfsCsvParserLibs { - val OLD = "COMMONS" - val NEW = "UNIVOCITY" - val DEFAULT = OLD - - def isValidLib(lib: String): Boolean = { - lib.toUpperCase match { - case OLD | NEW => true - case _ => false - } - } - - def isCommonsLib(lib: String): Boolean = if (isValidLib(lib)) { - lib.toUpperCase == OLD - } else { - true // default - } - - def isUnivocityLib(lib: String): Boolean = if (isValidLib(lib)) { - lib.toUpperCase == NEW - } else { - false // not the default - } - -} diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvRelation.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvRelation.scala deleted file mode 100644 index 07e80d0a..00000000 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/csv/WebHdfsCsvRelation.scala +++ /dev/null @@ -1,292 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.bahir.datasource.webhdfs.csv - - -import java.text.SimpleDateFormat - -// TODO: use scala.collection.JavaConverters instead of implicit JavaConversions -// scalastyle:off javaconversions -import scala.collection.JavaConversions._ -// scalastyle:on -import scala.util.control.NonFatal - -import org.apache.bahir.datasource.webhdfs.util._ -import org.apache.commons.csv.{CSVFormat, CSVParser} -import org.slf4j.LoggerFactory - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} -import org.apache.spark.sql.sources.{BaseRelation, PrunedScan, TableScan} -import org.apache.spark.sql.types.{StringType, StructField, StructType} - -/** - * This class contains functions for converting RDD to csv data source - * This is copied from com.databricks.spark.csv as the required object could not be reused as it is - * declared as private - */ -case class WebHdfsCsvRelation protected[webhdfs] ( - baseRDD: () => RDD[String], - location: Option[String], - useHeader: Boolean, - delimiter: Char, - quote: Character, - escape: Character, - comment: Character, - parseMode: String, - parserLib: String, - ignoreLeadingWhiteSpace: Boolean, - ignoreTrailingWhiteSpace: Boolean, - treatEmptyValuesAsNulls: Boolean, - userSchema: StructType = null, - inferCsvSchema: Boolean, - codec: String = null, - nullValue: String = "", - dateFormat: String = null, - maxCharsPerCol: Int = 100000)(@transient val sqlContext: SQLContext) - extends BaseRelation with TableScan with PrunedScan { - - // Share date format object as it is expensive to parse date pattern. - private val dateFormatter = if (dateFormat != null) new SimpleDateFormat(dateFormat) else null - - private val logger = LoggerFactory.getLogger(WebHdfsCsvRelation.getClass) - - // Parse mode flags - if (!WebHdfsParseModes.isValidMode(parseMode)) { - logger.warn(s"$parseMode is not a valid parse mode. Using ${WebHdfsParseModes.DEFAULT}.") - } - - if ((ignoreLeadingWhiteSpace || ignoreLeadingWhiteSpace) - && WebHdfsCsvParserLibs.isCommonsLib(parserLib)) { - logger.warn(s"Ignore white space options may not work with Commons parserLib option") - } - - private val failFast = WebHdfsParseModes.isFailFastMode(parseMode) - private val dropMalformed = WebHdfsParseModes.isDropMalformedMode(parseMode) - private val permissive = WebHdfsParseModes.isPermissiveMode(parseMode) - - override val schema: StructType = inferSchema() - - private def tokenRdd(header: Array[String]): RDD[Array[String]] = { - - val csvFormat = CSVFormat.DEFAULT - .withDelimiter(delimiter) - .withQuote(quote) - .withEscape(escape) - .withSkipHeaderRecord(false) - .withHeader(header: _*) - .withCommentMarker(comment) - - // If header is set, make sure firstLine is materialized before sending to executors. - val filterLine = if (useHeader) firstLine else null - - baseRDD().mapPartitions { iter => - // When using header, any input line that equals firstLine is assumed to be header - val csvIter = if (useHeader) { - iter.filter(_ != filterLine) - } else { - iter - } - parseCSV(csvIter, csvFormat) - } - } - - override def buildScan: RDD[Row] = { - val simpleDateFormatter = dateFormatter - val schemaFields = schema.fields - val rowArray = new Array[Any](schemaFields.length) - tokenRdd(schemaFields.map(_.name)).flatMap { tokens => - - if (dropMalformed && schemaFields.length != tokens.length) { - logger.warn(s"Dropping malformed line: ${tokens.mkString(",")}") - None - } else if (failFast && schemaFields.length != tokens.length) { - throw new RuntimeException(s"Malformed line in FAILFAST mode: ${tokens.mkString(",")}") - } else { - var index: Int = 0 - try { - index = 0 - while (index < schemaFields.length) { - val field = schemaFields(index) - rowArray(index) = WebHdfsTypeCast.castTo(tokens(index), field.dataType, field.nullable, - treatEmptyValuesAsNulls, nullValue, simpleDateFormatter) - index = index + 1 - } - Some(Row.fromSeq(rowArray)) - } catch { - case aiob: ArrayIndexOutOfBoundsException if permissive => - (index until schemaFields.length).foreach(ind => rowArray(ind) = null) - Some(Row.fromSeq(rowArray)) - case _: java.lang.NumberFormatException | - _: IllegalArgumentException if dropMalformed => - logger.warn("Number format exception. " + - s"Dropping malformed line: ${tokens.mkString(delimiter.toString)}") - None - case pe: java.text.ParseException if dropMalformed => - logger.warn("Parse exception. " + - s"Dropping malformed line: ${tokens.mkString(delimiter.toString)}") - None - } - } - } - } - - - /** - * This supports to eliminate unneeded columns before producing an RDD - * containing all of its tuples as Row objects. This reads all the tokens of each line - * and then drop unneeded tokens without casting and type-checking by mapping - * both the indices produced by `requiredColumns` and the ones of tokens. - */ - override def buildScan(requiredColumns: Array[String]): RDD[Row] = { - val simpleDateFormatter = dateFormatter - val schemaFields = schema.fields - val requiredFields = StructType(requiredColumns.map(schema(_))).fields - val shouldTableScan = schemaFields.deep == requiredFields.deep - val safeRequiredFields = if (dropMalformed) { - // If `dropMalformed` is enabled, then it needs to parse all the values - // so that we can decide which row is malformed. - requiredFields ++ schemaFields.filterNot(requiredFields.contains(_)) - } else { - requiredFields - } - val rowArray = new Array[Any](safeRequiredFields.length) - if (shouldTableScan) { - buildScan() - } else { - val safeRequiredIndices = new Array[Int](safeRequiredFields.length) - schemaFields.zipWithIndex.filter { - case (field, _) => safeRequiredFields.contains(field) - }.foreach { - case (field, index) => safeRequiredIndices(safeRequiredFields.indexOf(field)) = index - } - val requiredSize = requiredFields.length - tokenRdd(schemaFields.map(_.name)).flatMap { tokens => - - if (dropMalformed && schemaFields.length != tokens.length) { - logger.warn(s"Dropping malformed line: ${tokens.mkString(delimiter.toString)}") - None - } else if (failFast && schemaFields.length != tokens.length) { - throw new RuntimeException(s"Malformed line in FAILFAST mode: " + - s"${tokens.mkString(delimiter.toString)}") - } else { - val indexSafeTokens = if (permissive && schemaFields.length > tokens.length) { - tokens ++ new Array[String](schemaFields.length - tokens.length) - } else if (permissive && schemaFields.length < tokens.length) { - tokens.take(schemaFields.length) - } else { - tokens - } - try { - var index: Int = 0 - var subIndex: Int = 0 - while (subIndex < safeRequiredIndices.length) { - index = safeRequiredIndices(subIndex) - val field = schemaFields(index) - rowArray(subIndex) = WebHdfsTypeCast.castTo( - indexSafeTokens(index), - field.dataType, - field.nullable, - treatEmptyValuesAsNulls, - nullValue, - simpleDateFormatter - ) - subIndex = subIndex + 1 - } - Some(Row.fromSeq(rowArray.take(requiredSize))) - } catch { - case _: java.lang.NumberFormatException | - _: IllegalArgumentException if dropMalformed => - logger.warn("Number format exception. " + - s"Dropping malformed line: ${tokens.mkString(delimiter.toString)}") - None - case pe: java.text.ParseException if dropMalformed => - logger.warn("Parse exception. " + - s"Dropping malformed line: ${tokens.mkString(delimiter.toString)}") - None - } - } - } - } - } - - private def inferSchema(): StructType = { - if (this.userSchema != null) { - userSchema - } else { - val csvFormat = CSVFormat.DEFAULT - .withDelimiter(delimiter) - .withQuote(quote) - .withEscape(escape) - .withSkipHeaderRecord(false) - val firstRow = CSVParser.parse(firstLine, csvFormat).getRecords.head.toArray - val header = if (useHeader) { - firstRow - } else { - firstRow.zipWithIndex.map { case (value, index) => s"C$index"} - } - if (this.inferCsvSchema) { - val simpleDateFormatter = dateFormatter - WebHdfsInferSchema(tokenRdd(header), header, nullValue, simpleDateFormatter) - } else { - // By default fields are assumed to be StringType - val schemaFields = header.map { fieldName => - StructField(fieldName.toString, StringType, nullable = true) - } - StructType(schemaFields) - } - } - } - - /** - * Returns the first line of the first non-empty file in path - */ - private lazy val firstLine = { - if (comment != null) { - baseRDD().filter { line => - line.trim.nonEmpty && !line.startsWith(comment.toString) - }.first() - } else { - baseRDD().filter { line => - line.trim.nonEmpty - }.first() - } - } - - - private def parseCSV( - iter: Iterator[String], - csvFormat: CSVFormat): Iterator[Array[String]] = { - iter.flatMap { line => - try { - val records = CSVParser.parse(line, csvFormat).getRecords - if (records.isEmpty) { - logger.warn(s"Ignoring empty line: $line") - None - } else { - Some(records.head.toArray) - } - } catch { - case NonFatal(e) if !failFast => - logger.error(s"Exception while parsing line: $line. ", e) - None - } - } - } - -} diff --git a/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfs.java b/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfs.java deleted file mode 100644 index 4daa11ca..00000000 --- a/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfs.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hdfs.web; - -import java.net.URL; -import java.net.URI; - -import java.io.IOException; - -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.web.resources.Param; -import org.apache.hadoop.hdfs.web.resources.HttpOpParam.Op; - -import org.apache.hadoop.conf.Configuration; - -/** - * A FileSystem for HDFS over the web, extending [[org.apache.hadoop.hdfs.web.WebHdfsFileSystem]] - * to allow secure authentication and configurable gateway path segments. - * - * TODO: upgrade Hadoop version (Maven dependencies) to override SWebHdfsFileSystem - */ -class BahirWebHdfs extends WebHdfsFileSystem { - - // TODO: figure out how/where to authenticate - - // TODO: gateway path should be configurable - private String gatewayPath = "/gateway/default"; - - private URI uri = null; - private URI rHdfsUri = null; - private Configuration conf = null; - - /* - @Override - public void initialize( - URI uriOrg, - Configuration confOrg) throws IOException { - - super.initialize(uriOrg, confOrg); - - uri = URI.create(uriOrg.getScheme() + "://" + uriOrg.getAuthority()) ; - System.out.println("uri in initialize : "+ uri + "\n"); - - } - */ - - @Override - URL toUrl(final Op op, final Path fspath, final Param... parameters) throws IOException { - URL url = super.toUrl(op, fspath, parameters); - url = new URL("http://www.google.com"); - - System.out.print("url in overridden toUrl: " + url + "\n"); - return url; - - //Url newUrl = new URL("https", url.getHost, url.getPort, - //url.getFile.replaceFirst(WebHdfsFileSystem.PATH_PREFIX, - // gatewayPath + WebHdfsFileSystem.PATH_PREFIX)) - } - -} diff --git a/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfsFileSystem.scala b/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfsFileSystem.scala deleted file mode 100644 index f817b2ee..00000000 --- a/datasource-webhdfs/src/main/scala/org/apache/hadoop/hdfs/web/BahirWebHdfsFileSystem.scala +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.hdfs.web - -import java.net.URL -import java.net.URI - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hdfs.web.resources.HttpOpParam.Op -import org.apache.hadoop.hdfs.web.resources.Param - -import org.apache.hadoop.conf.Configuration - -/** - * A FileSystem for HDFS over the web, extending [[org.apache.hadoop.hdfs.web.WebHdfsFileSystem]] - * to allow secure authentication and configurable gateway path segments. - * - * TODO: upgrade Hadoop version (Maven dependencies) to override SWebHdfsFileSystem - */ -class BahirWebHdfsFileSystem extends WebHdfsFileSystem { - - // TODO: figure out how/where to authenticate - - // TODO: gateway path should be configurable - val gatewayPath = "/gateway/default" - - var uri:URI = null - var rHdfsUri:URI = null - var conf:Configuration = null - - override def initialize( - uriOrg: URI, - confOrg: Configuration): Unit = { - - super.initialize(uriOrg, confOrg) - - uri = URI.create(uriOrg.getScheme() + "://" + uriOrg.getAuthority()+"/gateway/default") - print("uri in initialize : " + uri + "\n") - - } - - //override def toUrl(op: HttpOpParam.Op, fspath: Path, parameters: Param[_, _]*): URL = { - override def toUrl(op: Op, fspath: Path, parameters: Param[_, _]*): URL = { - var url = super.toUrl(op, fspath, parameters: _*) - url = new URL("http://www.google.com") - - print("url in overriden toUrl: " + url + "\n") - - new URL("https", url.getHost, url.getPort, - url.getFile.replaceFirst(WebHdfsFileSystem.PATH_PREFIX, - gatewayPath + WebHdfsFileSystem.PATH_PREFIX)) - } - -} From 2497971264299fc8dbc5475b4787f15008b671e8 Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Wed, 7 Dec 2016 14:31:52 -0800 Subject: [PATCH 14/24] [BAHIR-75][WIP] - override WebHdfsFileSystem - more code style fixes, remove more unused files --- ...pache.spark.sql.sources.DataSourceRegister | 0 .../webhdfs/util/WebHdfsConnector.scala | 508 +++++++++--------- .../webhdfs/util/WebHdfsInferSchema.scala | 190 ------- .../webhdfs/util/WebHdfsParseModes.scala | 47 -- .../webhdfs/util/WebHdfsTypeCast.scala | 112 ---- 5 files changed, 256 insertions(+), 601 deletions(-) delete mode 100644 datasource-webhdfs/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister delete mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsInferSchema.scala delete mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsParseModes.scala delete mode 100644 datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsTypeCast.scala diff --git a/datasource-webhdfs/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/datasource-webhdfs/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister deleted file mode 100644 index e69de29b..00000000 diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala index 515b6578..3d6af59d 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala @@ -17,271 +17,275 @@ package org.apache.bahir.datasource.webhdfs.util -import java.nio.charset.Charset -import java.text.SimpleDateFormat -import java.sql.{Timestamp, Date} -import java.util.Date - -import org.apache.spark.sql.types.{DateType, TimestampType} - -import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD +import java.io._ +import java.security._ +import javax.net.ssl.{SSLContext, SSLSocketFactory, TrustManagerFactory} import scala.annotation.switch import scalaj.http._ -import java.security._ -import javax.net.ssl.TrustManagerFactory -import java.io._ -import javax.net.ssl.SSLSocketFactory -import javax.net.ssl.SSLContext -import org.apache.http.conn.scheme.Scheme -import java.net.HttpURLConnection -import scala.collection.mutable.HashMap -import scala.math._ -import org.apache.spark.sql.types.{DateType, TimestampType} -import org.apache.spark.sql.{DataFrame, SQLContext} -import scala.collection.mutable.ArrayBuffer -import scala.util.control.Breaks._ /** - * This object contains all utility functions for reading/writing data from/to remote webhdfs server. The abstraction maintained in this layer is at the level of RDD -*/ - + * This object contains all utility functions for reading/writing data from/to remote webhdfs + * server. The abstraction maintained in this layer is at the level of RDD + */ object WebHdfsConnector { - /** - * This function returns a Tuple for credential store which contains flag for validating Certificate, the Certificate File object and Certificate File Object password - - */ - - def createTrustStoreCredForExecutors(cred: String, path: String) : Tuple3[String, File, String] = { - - val trustStoreMap = if (cred != "") { - if (cred == "N") - new Tuple3("N", null, "") - else if(cred == "Y") - { - val tsd = SSLTrustStore.getCertDetails(path) - - new Tuple3("Y", tsd._1, tsd._2) - } - else - throw new Exception("Invalid Certificate Validation Option") - - } else { - new Tuple3("", null, "") - } - trustStoreMap - - - } - - - /** - * This function returns a SSLSocketFactory which needs to be used in HTTP connection library in case Certificate to be validated - */ - - def biocSslSocketFactory(fl: File, pswrd: String): SSLSocketFactory = { - - - val tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()) - val ks = KeyStore.getInstance("JKS") - val fis = new java.io.FileInputStream(fl) - ks.load(fis, pswrd.toCharArray()); - tmf.init(ks); - - - val sslc = SSLContext.getInstance("SSL") - - sslc.init(null, tmf.getTrustManagers(),null) - - sslc.getSocketFactory() - } - - /** - * This function returns the list of files in a folder with file details as RDD - - def listFromWebHdfs(sc: SparkContext, path: String, trustStoreCred: String, userCred: String, connProp: String): RDD[String] = { - - - - val conns = connProp.split(":") - - val conn = Array(conns(0).toInt, conns(1).toInt) - - val usrCred = userCred.split(":") - - val trustCred = createTrustStoreCredForExecutors(trustStoreCred, path) - - val fileDetails = getFilesDetails(path, trustCred, usrCred, conn) - - def g(v:Tuple4[String, Long, Long, Int]) = v._1.split("/").last + "," + v._2.toString + "," + v._3.toString + "," + v._4.toString + "\n" - - val fds = fileDetails.map(x => g(x)) - - val flRdd = sc.parallelize(fds) - - flRdd - - } - - * This function writes data back to hdfs using WebHDFS using multiple parallel connections. Right now file overwrite is not supported - - def writeToWebHdfs(dataToWrite: RDD[String] ,path: String, trustStoreCredStr: String, connStr : String, userCredStr : String, partitionStr : String): Unit = { - - val trustCred = createTrustStoreCredForExecutors(trustStoreCredStr, path) - - val conns = connStr.split(":") - - val conn = Array(conns(0).toInt, conns(1).toInt) - - val usr = userCredStr.split(":") - - val webHdfsChkDirOpr = "op=GETFILESTATUS" - val returnChkDir = callWebHdfsAPI(path, "", "GET", "CODE", trustCred, usr, conn, webHdfsChkDirOpr) - - if (returnChkDir == "200") - throw new Exception("The File Already Exists : " + path + "\n") - - val dPartitions = partitionStr.toInt - - val textRdd = dataToWrite.repartition(dPartitions) - - val webHdfsMakeDirOpr = "op=MKDIRS" - val returnCreateDir = callWebHdfsAPI(path, "", "PUT", "CODE", trustCred, usr, conn, webHdfsMakeDirOpr) - - textRdd.mapPartitionsWithIndex((idx, iter) => WebHdfsConnector.saveAllFiles(idx, iter, usr, path, trustCred, conn)).collect() - - } - - * This function is passed to mapPartitionsWithIndex so that each executor task can save part of the data using separate connection - - def saveAllFiles (idx: Int, data : Iterator[String], usrCred: Array[String], path: String, trustCred: Tuple3[String, File , String], connProp: Array[Int]): Iterator[String] = { - - - var dataP = data.next() - while(data.hasNext) { - dataP = dataP + "\n" + data.next() - } - - val fnameArray = path.split("/") - val fnameIdx = fnameArray.length - 1 - val fname = fnameArray(fnameIdx) - val filePath = s"$path/part-000$idx-$fname" - - val createOpr = "op=CREATE" - - val createUrl = callWebHdfsAPI(filePath, "", "PUT", "LOCATION", trustCred, usrCred, connProp, createOpr) - val created = callWebHdfsAPI(createUrl, dataP, "PUT", "CODE", trustCred, usrCred, connProp, createOpr) - - val ret = Array(created.toString) - ret.iterator - - } - */ - - - def callWebHdfsAPI(path: String, data: String, method: String, respType: String, trustStoreCredStr: String, usrCredStr: String, connStr: String, opr: String, outputType : String): Any = { - - //print("path in callWebHdfs : " + path + "\n") - - val pathComp = path.split(":") - - val trustCred = createTrustStoreCredForExecutors(trustStoreCredStr, path) - - val conns = connStr.split(":") - - val connProp = Array(conns(0).toInt, conns(1).toInt) - - val usrCred = usrCredStr.split(":") - - val uri = (if(trustCred._1 != "") "https:" else "http:") + pathComp(1) + ":" + pathComp(2) + "?" + opr - - var httpc = Http(uri).auth(usrCred(0), usrCred(1)).timeout(connTimeoutMs = connProp(0), readTimeoutMs = connProp(1)) - - httpc = (method : @switch) match { - case "GET" => httpc - case "PUT" => httpc.put(data).header("content-type", "application/csv") - case "POST" => httpc.postData(data).header("content-type", "application/csv") - } - - httpc = (trustCred._1 : @switch) match { - case "" => httpc - case "N" => httpc.option(HttpOptions.allowUnsafeSSL) - case "Y" => httpc.option(HttpOptions.sslSocketFactory(biocSslSocketFactory(trustCred._2, trustCred._3))) - } - - val out = (outputType : @switch) match { - case "" => httpc.asString - case "String" => httpc.asString - case "Bytes" => httpc.asBytes - } - - val resp = (respType : @switch) match { - case "BODY" => out.body - case "CODE" => out.code - case "HEADERS" => out.headers - case "LOCATION" => out.location.mkString(" ") - } - - /* - val resp = (respType : @switch) match { - case "BODY" => httpc.asBytes.body - case "CODE" => httpc.asString.code - case "HEADERS" => httpc.asString.headers - case "LOCATION" => httpc.asString.location.mkString(" ") - } - */ - - resp - - } - - def getFileInputStream (filePath : String, offset : Long, length : Long, trustStoreCredStr: String, connStr: String, usrCredStr : String): ByteArrayInputStream = { - - //print("path in getFileInputStream : " + filePath + "\n") - - val fileGetOpr = if(length > 0) s"op=OPEN&offset=$offset&length=$length&bufferSize=$length" else s"op=OPEN&offset=$offset" - - val getUrl = callWebHdfsAPI(filePath, "", "GET", "LOCATION", trustStoreCredStr, usrCredStr, connStr, fileGetOpr, "String").asInstanceOf[String] - - val content = callWebHdfsAPI(getUrl, "", "GET", "BODY", trustStoreCredStr, usrCredStr, connStr, fileGetOpr, "Bytes").asInstanceOf[Array[Byte]] - - new ByteArrayInputStream(content) - - } - - def getFileStatus(filePath: String, trustStoreCredStr : String, connStr: String, usrCredStr: String): Map[String, Any] = { - - //print("path in getFileStatus : " + filePath + "\n") - val fileStatusOpr = s"op=GETFILESTATUS" - - val fileStatus = callWebHdfsAPI(filePath, "", "GET", "BODY", trustStoreCredStr, usrCredStr, connStr, fileStatusOpr, "String").asInstanceOf[String] - - if (fileStatus.contains("RemoteException")) - throw new Exception(fileStatus) - - val responseMap = scala.util.parsing.json.JSON.parseFull(fileStatus).toList(0).asInstanceOf[Map[String, Map[String, Any]]] - - responseMap.getOrElse("FileStatus", throw new Exception ("File Status Null")) - - } - - def getListStatus(filePath: String, trustStoreCredStr : String, connStr: String, usrCredStr: String): List[Map[String, Any]] = { - + /* + * This function returns a Tuple for credential store which contains flag for validating + * Certificate, the Certificate File object and Certificate File Object password + */ + def createTrustStoreCredForExecutors(cred: String, path: String): Tuple3[String, File, String] = { + val trustStoreMap = { + if (cred != "") { + if (cred == "N") { + new Tuple3("N", null, "") + } else if (cred == "Y") { + val tsd = SSLTrustStore.getCertDetails(path) + new Tuple3("Y", tsd._1, tsd._2) + } else { + throw new Exception("Invalid Certificate Validation Option") + } + } else { + new Tuple3("", null, "") + } + } + trustStoreMap + } + + + /** + * This function returns a SSLSocketFactory which needs to be used in HTTP connection library in case Certificate to be validated + */ + def biocSslSocketFactory(fl: File, pswrd: String): SSLSocketFactory = { + val tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()) + val ks = KeyStore.getInstance("JKS") + val fis = new java.io.FileInputStream(fl) + ks.load(fis, pswrd.toCharArray()); + tmf.init(ks); + + val sslc = SSLContext.getInstance("SSL") + + sslc.init(null, tmf.getTrustManagers(), null) + + sslc.getSocketFactory() + } + + // /** + // * This function returns the list of files in a folder with file details as RDD + // */ + // def listFromWebHdfs(sc: SparkContext, path: String, trustStoreCred: String, userCred: String, connProp: String): RDD[String] = { + // + // + // + // val conns = connProp.split(":") + // + // val conn = Array(conns(0).toInt, conns(1).toInt) + // + // val usrCred = userCred.split(":") + // + // val trustCred = createTrustStoreCredForExecutors(trustStoreCred, path) + // + // val fileDetails = getFilesDetails(path, trustCred, usrCred, conn) + // + // def g(v:Tuple4[String, Long, Long, Int]) = v._1.split("/").last + "," + v._2.toString + "," + v._3.toString + "," + v._4.toString + "\n" + // + // val fds = fileDetails.map(x => g(x)) + // + // val flRdd = sc.parallelize(fds) + // + // flRdd + // + // } + // + // /** + // * This function writes data back to hdfs using WebHDFS using multiple parallel connections. Right now file overwrite is not supported + // */ + // def writeToWebHdfs(dataToWrite: RDD[String] ,path: String, trustStoreCredStr: String, connStr : String, userCredStr : String, partitionStr : String): Unit = { + // + // val trustCred = createTrustStoreCredForExecutors(trustStoreCredStr, path) + // + // val conns = connStr.split(":") + // + // val conn = Array(conns(0).toInt, conns(1).toInt) + // + // val usr = userCredStr.split(":") + // + // val webHdfsChkDirOpr = "op=GETFILESTATUS" + // val returnChkDir = callWebHdfsAPI(path, "", "GET", "CODE", trustCred, usr, conn, webHdfsChkDirOpr) + // + // if (returnChkDir == "200") + // throw new Exception("The File Already Exists : " + path + "\n") + // + // val dPartitions = partitionStr.toInt + // + // val textRdd = dataToWrite.repartition(dPartitions) + // + // val webHdfsMakeDirOpr = "op=MKDIRS" + // val returnCreateDir = callWebHdfsAPI(path, "", "PUT", "CODE", trustCred, usr, conn, webHdfsMakeDirOpr) + // + // textRdd.mapPartitionsWithIndex((idx, iter) => WebHdfsConnector.saveAllFiles(idx, iter, usr, path, trustCred, conn)).collect() + // + // } + // + // /** + // * This function is passed to mapPartitionsWithIndex so that each executor task can save part of the data using separate connection + // */ + // def saveAllFiles (idx: Int, data : Iterator[String], usrCred: Array[String], path: String, trustCred: Tuple3[String, File , String], connProp: Array[Int]): Iterator[String] = { + // + // + // var dataP = data.next() + // while(data.hasNext) { + // dataP = dataP + "\n" + data.next() + // } + // + // val fnameArray = path.split("/") + // val fnameIdx = fnameArray.length - 1 + // val fname = fnameArray(fnameIdx) + // val filePath = s"$path/part-000$idx-$fname" + // + // val createOpr = "op=CREATE" + // + // val createUrl = callWebHdfsAPI(filePath, "", "PUT", "LOCATION", trustCred, usrCred, connProp, createOpr) + // val created = callWebHdfsAPI(createUrl, dataP, "PUT", "CODE", trustCred, usrCred, connProp, createOpr) + // + // val ret = Array(created.toString) + // ret.iterator + // + // } + + + + def callWebHdfsAPI(path: String, + data: String, + method: String, + respType: String, + trustStoreCredStr: String, + usrCredStr: String, + connStr: String, + opr: String, + outputType : String): Any = { + + // print("path in callWebHdfs : " + path + "\n") + + val pathComp = path.split(":") + + val trustCred = createTrustStoreCredForExecutors(trustStoreCredStr, path) + + val conns = connStr.split(":") + + val connProp = Array(conns(0).toInt, conns(1).toInt) + + val usrCred = usrCredStr.split(":") + + val uri = (if (trustCred._1 != "") "https:" else "http:") + pathComp(1) + ":" + pathComp(2) + + "?" + opr + + var httpc = Http(uri).auth(usrCred(0), usrCred(1)).timeout(connTimeoutMs = connProp(0), + readTimeoutMs = connProp(1)) + + httpc = (method : @switch) match { + case "GET" => httpc + case "PUT" => httpc.put(data).header("content-type", "application/csv") + case "POST" => httpc.postData(data).header("content-type", "application/csv") + } + + httpc = (trustCred._1 : @switch) match { + case "" => httpc + case "N" => httpc.option(HttpOptions.allowUnsafeSSL) + case "Y" => httpc.option(HttpOptions.sslSocketFactory(biocSslSocketFactory(trustCred._2, + trustCred._3))) + } + + val out = (outputType : @switch) match { + case "" => httpc.asString + case "String" => httpc.asString + case "Bytes" => httpc.asBytes + } + + val resp = (respType : @switch) match { + case "BODY" => out.body + case "CODE" => out.code + case "HEADERS" => out.headers + case "LOCATION" => out.location.mkString(" ") + } + + /* + val resp = (respType : @switch) match { + case "BODY" => httpc.asBytes.body + case "CODE" => httpc.asString.code + case "HEADERS" => httpc.asString.headers + case "LOCATION" => httpc.asString.location.mkString(" ") + } + */ + + resp + } + + def getFileInputStream (filePath: String, + offset: Long, + length: Long, + trustStoreCredStr: String, + connStr: String, + usrCredStr: String): ByteArrayInputStream = { + + // print("path in getFileInputStream : " + filePath + "\n") + + val fileGetOpr = if (length > 0) { + s"op=OPEN&offset=$offset&length=$length&bufferSize=$length" + } else { + s"op=OPEN&offset=$offset" + } + + val getUrl = callWebHdfsAPI(filePath, "", "GET", "LOCATION", trustStoreCredStr, usrCredStr, + connStr, fileGetOpr, "String").asInstanceOf[String] + + val content = callWebHdfsAPI(getUrl, "", "GET", "BODY", trustStoreCredStr, usrCredStr, + connStr, fileGetOpr, "Bytes").asInstanceOf[Array[Byte]] + + new ByteArrayInputStream(content) + } + + def getFileStatus(filePath: String, + trustStoreCredStr: String, + connStr: String, + usrCredStr: String): Map[String, Any] = { - //print("path in getListStatus : " + filePath + "\n") - val listStatusOpr = s"op=LISTSTATUS" + // print("path in getFileStatus : " + filePath + "\n") + val fileStatusOpr = s"op=GETFILESTATUS" - val listStatus = callWebHdfsAPI(filePath, "", "GET", "BODY", trustStoreCredStr, usrCredStr, connStr, listStatusOpr, "String").asInstanceOf[String] + val fileStatus = callWebHdfsAPI(filePath, "", "GET", "BODY", trustStoreCredStr, usrCredStr, + connStr, fileStatusOpr, "String").asInstanceOf[String] + + if (fileStatus.contains("RemoteException")) { + throw new Exception(fileStatus) + } + + val responseMap = scala.util.parsing.json.JSON.parseFull(fileStatus).toList(0) + .asInstanceOf[Map[String, Map[String, Any]]] - if (listStatus.contains("RemoteException")) - throw new Exception(listStatus) + responseMap.getOrElse("FileStatus", throw new Exception ("File Status Null")) + } - scala.util.parsing.json.JSON.parseFull(listStatus).toList(0).asInstanceOf[Map[String, Map[String, Any]]].get("FileStatuses").get("FileStatus").asInstanceOf[List[Map[String, Any]]] - } + def getListStatus(filePath: String, + trustStoreCredStr: String, + connStr: String, + usrCredStr: String): List[Map[String, Any]] = { + // print("path in getListStatus : " + filePath + "\n") + val listStatusOpr = s"op=LISTSTATUS" + + val listStatus = callWebHdfsAPI(filePath, "", "GET", "BODY", trustStoreCredStr, usrCredStr, + connStr, listStatusOpr, "String").asInstanceOf[String] + + if (listStatus.contains("RemoteException")) { + throw new Exception(listStatus) + } + + scala.util.parsing.json.JSON.parseFull(listStatus).toList(0) + .asInstanceOf[Map[String, Map[String, Any]]].get("FileStatuses").get("FileStatus") + .asInstanceOf[List[Map[String, Any]]] + } } diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsInferSchema.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsInferSchema.scala deleted file mode 100644 index a012a3fb..00000000 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsInferSchema.scala +++ /dev/null @@ -1,190 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.bahir.datasource.webhdfs.util - -import java.sql.Timestamp -import java.text.SimpleDateFormat - -import scala.util.control.Exception._ - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.types._ - - -/** - * Utility functions for infering schema - * TODO: copied from com.databricks.spark.csv.util package as there was now way to reuse this object - */ -private[webhdfs] object WebHdfsInferSchema { - - /** - * TODO: Copied from internal Spark API - * [[org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion]] - */ - val findTightestCommonType: (DataType, DataType) => Option[DataType] = { - case (t1, t2) if t1 == t2 => Some(t1) - case (NullType, t1) => Some(t1) - case (t1, NullType) => Some(t1) - case (StringType, t2) => Some(StringType) - case (t1, StringType) => Some(StringType) - - // Promote numeric types to the highest of the two and all numeric types to unlimited decimal - case (t1, t2) if Seq(t1, t2).forall(numericPrecedence.contains) => - val index = numericPrecedence.lastIndexWhere(t => t == t1 || t == t2) - Some(numericPrecedence(index)) - - case _ => None - } - /** - * TODO: Copied from internal Spark api - * [[org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion]] - */ - private val numericPrecedence: IndexedSeq[DataType] = - IndexedSeq[DataType]( - ByteType, - ShortType, - IntegerType, - LongType, - FloatType, - DoubleType, - TimestampType, - DecimalType.USER_DEFAULT) - - /** - * Similar to the JSON schema inference. - * [[org.apache.spark.sql.execution.datasources.json.InferSchema]] - * 1. Infer type of each row - * 2. Merge row types to find common type - * 3. Replace any null types with string type - */ - def apply( tokenRdd: RDD[Array[String]], - header: Array[String], - nullValue: String = "", - dateFormatter: SimpleDateFormat = null): StructType = { - val startType: Array[DataType] = Array.fill[DataType](header.length)(NullType) - val rootTypes: Array[DataType] = tokenRdd.aggregate(startType)( - inferRowType(nullValue, dateFormatter), - mergeRowTypes) - - val structFields = header.zip(rootTypes).map { case (thisHeader, rootType) => - val dType = rootType match { - case z: NullType => StringType - case other => other - } - StructField(thisHeader, dType, nullable = true) - } - - StructType(structFields) - } - - private def inferRowType(nullValue: String, dateFormatter: SimpleDateFormat) - (rowSoFar: Array[DataType], next: Array[String]): Array[DataType] = { - var i = 0 - while (i < math.min(rowSoFar.length, next.length)) { - // May have columns on right missing. - rowSoFar(i) = inferField(rowSoFar(i), next(i), nullValue, dateFormatter) - i += 1 - } - rowSoFar - } - - /** - * Infer type of string field. Given known type Double, and a string "1", there is no - * point checking if it is an Int, as the final type must be Double or higher. - */ - private[webhdfs] def inferField(typeSoFar: DataType, - field: String, - nullValue: String = "", - dateFormatter: SimpleDateFormat = null): DataType = { - def tryParseInteger(field: String): DataType = if ((allCatch opt field.toInt).isDefined) { - IntegerType - } else { - tryParseLong(field) - } - - def tryParseLong(field: String): DataType = if ((allCatch opt field.toLong).isDefined) { - LongType - } else { - tryParseDouble(field) - } - - def tryParseDouble(field: String): DataType = { - if ((allCatch opt field.toDouble).isDefined) { - DoubleType - } else { - tryParseTimestamp(field) - } - } - - def tryParseTimestamp(field: String): DataType = { - if (dateFormatter != null) { - // This case infers a custom `dataFormat` is set. - if ((allCatch opt dateFormatter.parse(field)).isDefined) { - TimestampType - } else { - tryParseBoolean(field) - } - } else { - // We keep this for backwords competibility. - if ((allCatch opt Timestamp.valueOf(field)).isDefined) { - TimestampType - } else { - tryParseBoolean(field) - } - } - } - - def tryParseBoolean(field: String): DataType = { - if ((allCatch opt field.toBoolean).isDefined) { - BooleanType - } else { - stringType() - } - } - - // Defining a function to return the StringType constant is necessary in order to work around - // a Scala compiler issue which leads to runtime incompatibilities with certain Spark versions; - // see issue #128 for more details. - def stringType(): DataType = { - StringType - } - - if (field == null || field.isEmpty || field == nullValue) { - typeSoFar - } else { - typeSoFar match { - case NullType => tryParseInteger(field) - case IntegerType => tryParseInteger(field) - case LongType => tryParseLong(field) - case DoubleType => tryParseDouble(field) - case TimestampType => tryParseTimestamp(field) - case BooleanType => tryParseBoolean(field) - case StringType => StringType - case other: DataType => - throw new UnsupportedOperationException(s"Unexpected data type $other") - } - } - } - - private[webhdfs] def mergeRowTypes(first: Array[DataType], - second: Array[DataType]): Array[DataType] = { - first.zipAll(second, NullType, NullType).map { case ((a, b)) => - findTightestCommonType(a, b).getOrElse(NullType) - } - } -} diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsParseModes.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsParseModes.scala deleted file mode 100644 index 52b7162a..00000000 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsParseModes.scala +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.bahir.datasource.webhdfs.util - -/** - * Utility functions for mode of Parsing - * Copied from com.databricks.spark.csv.util package as there was now way to reuse this object - */ -private[webhdfs] object WebHdfsParseModes { - - val PERMISSIVE_MODE = "PERMISSIVE" - val DROP_MALFORMED_MODE = "DROPMALFORMED" - val FAIL_FAST_MODE = "FAILFAST" - val DEFAULT = PERMISSIVE_MODE - - def isDropMalformedMode(mode: String): Boolean = mode.toUpperCase == DROP_MALFORMED_MODE - - def isFailFastMode(mode: String): Boolean = mode.toUpperCase == FAIL_FAST_MODE - - def isPermissiveMode(mode: String): Boolean = if (isValidMode(mode)) { - mode.toUpperCase == PERMISSIVE_MODE - } else { - true // We default to permissive is the mode string is not valid - } - - def isValidMode(mode: String): Boolean = { - mode.toUpperCase match { - case PERMISSIVE_MODE | DROP_MALFORMED_MODE | FAIL_FAST_MODE => true - case _ => false - } - } -} diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsTypeCast.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsTypeCast.scala deleted file mode 100644 index d3a93663..00000000 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsTypeCast.scala +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.bahir.datasource.webhdfs.util - -import java.math.BigDecimal -import java.sql.{Date, Timestamp} -import java.text.{NumberFormat, SimpleDateFormat} -import java.util.Locale - -import scala.util.Try - -import org.apache.spark.sql.types._ - -/** - * Utility functions for type casting - * Copied from com.databricks.spark.csv.util package as there was now way to reuse this object - */ -object WebHdfsTypeCast { - - /** - * Casts given string datum to specified type. - * Currently we do not support complex types (ArrayType, MapType, StructType). - * - * For string types, this is simply the datum. For other types. - * For other nullable types, this is null if the string datum is empty. - * - * @param datum string value - * @param castType SparkSQL type - */ - private[webhdfs] def castTo(datum: String, - castType: DataType, - nullable: Boolean = true, - treatEmptyValuesAsNulls: Boolean = false, - nullValue: String = "", - dateFormatter: SimpleDateFormat = null): Any = { - - // if nullValue is not an empty string, don't require treatEmptyValuesAsNulls - // to be set to true - val nullValueIsNotEmpty = nullValue != "" - if (datum == nullValue && - nullable && - (!castType.isInstanceOf[StringType] || treatEmptyValuesAsNulls || nullValueIsNotEmpty) - ) { - null - } else { - castType match { - case _: ByteType => datum.toByte - case _: ShortType => datum.toShort - case _: IntegerType => datum.toInt - case _: LongType => datum.toLong - case _: FloatType => Try(datum.toFloat) - .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).floatValue()) - case _: DoubleType => Try(datum.toDouble) - .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).doubleValue()) - case _: BooleanType => datum.toBoolean - case _: DecimalType => new BigDecimal(datum.replaceAll(",", "")) - case _: TimestampType if dateFormatter != null => - new Timestamp(dateFormatter.parse(datum).getTime) - case _: TimestampType => Timestamp.valueOf(datum) - case _: DateType if dateFormatter != null => - new Date(dateFormatter.parse(datum).getTime) - case _: DateType => Date.valueOf(datum) - case _: StringType => datum - case _ => throw new RuntimeException(s"Unsupported type: ${castType.typeName}") - } - } - } - - /** - * Helper method that converts string representation of a character to actual character. - * It handles some Java escaped strings and throws exception if given string is longer than one - * character. - * - */ - @throws[IllegalArgumentException] - private[webhdfs] def toChar(str: String): Char = { - if (str.charAt(0) == '\\') { - str.charAt(1) - match { - case 't' => '\t' - case 'r' => '\r' - case 'b' => '\b' - case 'f' => '\f' - case '\"' => '\"' // In case user changes quote char and uses \" as delimiter in options - case '\'' => '\'' - case 'u' if str == """\u0000""" => '\u0000' - case _ => - throw new IllegalArgumentException(s"Unsupported special character for delimiter: $str") - } - } else if (str.length == 1) { - str.charAt(0) - } else { - throw new IllegalArgumentException(s"Delimiter cannot be more than one character: $str") - } - } -} From 29718806923a64d1b56490978516b8bd95479f8c Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Wed, 7 Dec 2016 14:39:34 -0800 Subject: [PATCH 15/24] [BAHIR-75][WIP] - override WebHdfsFileSystem - more and more code style fixes, remove more unused files --- .../webhdfs/BahirWebHdfsDataSetWrapper.scala | 4 +- .../webhdfs/util/WebHdfsConnector.scala | 39 ++++++++++++------- 2 files changed, 27 insertions(+), 16 deletions(-) diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala index 6353f620..ab9b82c0 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala @@ -53,8 +53,8 @@ class BahirWebHdfsFileSystem extends FileSystem { /** - * This method does necessary initialization of the configuration parameters - */ + * This method does necessary initialization of the configuration parameters + */ override def initialize(uriOrg: URI, confOrg: Configuration): Unit = { diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala index 3d6af59d..cd2d963d 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala @@ -22,8 +22,8 @@ import java.security._ import javax.net.ssl.{SSLContext, SSLSocketFactory, TrustManagerFactory} import scala.annotation.switch -import scalaj.http._ +import scalaj.http.{Http, HttpOptions} /** * This object contains all utility functions for reading/writing data from/to remote webhdfs @@ -31,7 +31,6 @@ import scalaj.http._ */ object WebHdfsConnector { - /* * This function returns a Tuple for credential store which contains flag for validating * Certificate, the Certificate File object and Certificate File Object password @@ -56,7 +55,8 @@ object WebHdfsConnector { /** - * This function returns a SSLSocketFactory which needs to be used in HTTP connection library in case Certificate to be validated + * This function returns a SSLSocketFactory which needs to be used in HTTP connection library in + * case Certificate to be validated */ def biocSslSocketFactory(fl: File, pswrd: String): SSLSocketFactory = { val tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()) @@ -75,7 +75,8 @@ object WebHdfsConnector { // /** // * This function returns the list of files in a folder with file details as RDD // */ - // def listFromWebHdfs(sc: SparkContext, path: String, trustStoreCred: String, userCred: String, connProp: String): RDD[String] = { + // def listFromWebHdfs(sc: SparkContext, path: String, trustStoreCred: String, userCred: String, + // connProp: String): RDD[String] = { // // // @@ -89,7 +90,8 @@ object WebHdfsConnector { // // val fileDetails = getFilesDetails(path, trustCred, usrCred, conn) // - // def g(v:Tuple4[String, Long, Long, Int]) = v._1.split("/").last + "," + v._2.toString + "," + v._3.toString + "," + v._4.toString + "\n" + // def g(v:Tuple4[String, Long, Long, Int]) = v._1.split("/").last + "," + + // v._2.toString + "," + v._3.toString + "," + v._4.toString + "\n" // // val fds = fileDetails.map(x => g(x)) // @@ -100,9 +102,11 @@ object WebHdfsConnector { // } // // /** - // * This function writes data back to hdfs using WebHDFS using multiple parallel connections. Right now file overwrite is not supported + // * This function writes data back to hdfs using WebHDFS using multiple parallel connections. + // Right now file overwrite is not supported // */ - // def writeToWebHdfs(dataToWrite: RDD[String] ,path: String, trustStoreCredStr: String, connStr : String, userCredStr : String, partitionStr : String): Unit = { + // def writeToWebHdfs(dataToWrite: RDD[String] ,path: String, trustStoreCredStr: String, + // connStr : String, userCredStr : String, partitionStr : String): Unit = { // // val trustCred = createTrustStoreCredForExecutors(trustStoreCredStr, path) // @@ -113,7 +117,8 @@ object WebHdfsConnector { // val usr = userCredStr.split(":") // // val webHdfsChkDirOpr = "op=GETFILESTATUS" - // val returnChkDir = callWebHdfsAPI(path, "", "GET", "CODE", trustCred, usr, conn, webHdfsChkDirOpr) + // val returnChkDir = callWebHdfsAPI(path, "", "GET", "CODE", trustCred, usr, conn, + // webHdfsChkDirOpr) // // if (returnChkDir == "200") // throw new Exception("The File Already Exists : " + path + "\n") @@ -123,16 +128,20 @@ object WebHdfsConnector { // val textRdd = dataToWrite.repartition(dPartitions) // // val webHdfsMakeDirOpr = "op=MKDIRS" - // val returnCreateDir = callWebHdfsAPI(path, "", "PUT", "CODE", trustCred, usr, conn, webHdfsMakeDirOpr) + // val returnCreateDir = callWebHdfsAPI(path, "", "PUT", "CODE", trustCred, usr, conn, + // webHdfsMakeDirOpr) // - // textRdd.mapPartitionsWithIndex((idx, iter) => WebHdfsConnector.saveAllFiles(idx, iter, usr, path, trustCred, conn)).collect() + // textRdd.mapPartitionsWithIndex((idx, iter) => WebHdfsConnector.saveAllFiles(idx, iter, usr, + // path, trustCred, conn)).collect() // // } // // /** - // * This function is passed to mapPartitionsWithIndex so that each executor task can save part of the data using separate connection + // * This function is passed to mapPartitionsWithIndex so that each executor task can save part + // of the data using separate connection // */ - // def saveAllFiles (idx: Int, data : Iterator[String], usrCred: Array[String], path: String, trustCred: Tuple3[String, File , String], connProp: Array[Int]): Iterator[String] = { + // def saveAllFiles (idx: Int, data : Iterator[String], usrCred: Array[String], path: String, + // trustCred: Tuple3[String, File , String], connProp: Array[Int]): Iterator[String] = { // // // var dataP = data.next() @@ -147,8 +156,10 @@ object WebHdfsConnector { // // val createOpr = "op=CREATE" // - // val createUrl = callWebHdfsAPI(filePath, "", "PUT", "LOCATION", trustCred, usrCred, connProp, createOpr) - // val created = callWebHdfsAPI(createUrl, dataP, "PUT", "CODE", trustCred, usrCred, connProp, createOpr) + // val createUrl = callWebHdfsAPI(filePath, "", "PUT", "LOCATION", trustCred, usrCred, + // connProp, createOpr) + // val created = callWebHdfsAPI(createUrl, dataP, "PUT", "CODE", trustCred, usrCred, connProp, + // createOpr) // // val ret = Array(created.toString) // ret.iterator From a9bbe31d17706d04bb44214b33479980a24a7d8e Mon Sep 17 00:00:00 2001 From: Sourav Mazumder Date: Wed, 7 Dec 2016 16:41:51 -0800 Subject: [PATCH 16/24] [BAHIR-75][WIP] - override WebHdfsFileSystem - add printouts for debugging --- .../webhdfs/BahirWebHdfsDataSetWrapper.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala index ab9b82c0..075cfec4 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala @@ -36,6 +36,10 @@ import org.apache.hadoop.util.Progressable */ class BahirWebHdfsFileSystem extends FileSystem { + // scalastyle:off println + println(s" - - - BahirWebHdfsFileSystem class loaded - - - ") + // scalastyle:on println + var uri: URI = null var rHdfsUri: URI = null var conf: Configuration = null @@ -80,7 +84,11 @@ class BahirWebHdfsFileSystem extends FileSystem { certValidation = if (certFlg == null) "Y" else certFlg.toString uri = URI.create(uriOrg.getScheme() + "://" + uriOrg.getAuthority()) - // println("uri: ", uri + " , connections: " + connections + " , user cred: " + usrCred) + + // scalastyle:off println + println("BahirWebHdfsFileSystem: uri=", uri + ", connections=" + connections + + ", usercred=" + usrCred) + // scalastyle:on println } From f6429c9be053f996fbfe6c7bf794843375aebf27 Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Wed, 7 Dec 2016 16:50:12 -0800 Subject: [PATCH 17/24] [BAHIR-75][WIP] - override WebHdfsFileSystem - add printouts for debugging --- .../bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala index 075cfec4..8e8092f4 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala @@ -86,8 +86,7 @@ class BahirWebHdfsFileSystem extends FileSystem { uri = URI.create(uriOrg.getScheme() + "://" + uriOrg.getAuthority()) // scalastyle:off println - println("BahirWebHdfsFileSystem: uri=", uri + ", connections=" + connections + - ", usercred=" + usrCred) + println(s"BahirWebHdfsFileSystem: uri=${uri}, connections=${connections}, usercred=${usrCred}") // scalastyle:on println } From 39f59851de1c462545866ea17ee547493fc71c5b Mon Sep 17 00:00:00 2001 From: Sourav Mazumder Date: Fri, 9 Dec 2016 10:56:01 -0800 Subject: [PATCH 18/24] [BAHIR-75][WIP] - write to remote via webhdfs --- .../webhdfs/BahirWebHdfsDataSetWrapper.scala | 69 ++++++- .../webhdfs/util/WebHdfsConnector.scala | 174 ++++++++++-------- 2 files changed, 164 insertions(+), 79 deletions(-) diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala index 8e8092f4..43c11cba 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala @@ -105,11 +105,14 @@ class BahirWebHdfsFileSystem extends FileSystem { } override def delete(srcPath: Path, recursive: Boolean): Boolean = { + println("In delete - Path : " + srcPath + " , recursive flg : " + recursive) + WebHdfsConnector.deleteFile(srcPath.toString(), recursive, certValidation, "1000:5000", usrCred) throw new Exception("File Delete Not Supported") } override def mkdirs(srcPath: Path, permission: FsPermission): Boolean = { - throw new Exception("Make Directory Not Supported") + println("In MkDirs - Path : " + srcPath) + WebHdfsConnector.makeDirectory(srcPath.toString(), certValidation, "1000:5000", usrCred) } override def append(srcPath: Path, @@ -120,9 +123,12 @@ class BahirWebHdfsFileSystem extends FileSystem { override def getFileStatus(f: Path): FileStatus = { val file = stripQryFromFilePath(f).toString + println("In bahir get filestatus - Path : " + f) var fStatus: FileStatus = fileStatusMap.getOrElse(file, null) + println("In bahir after checking filestatus map - : " + fStatus) val fileStatus = if (fStatus == null) { + println("In bahir before calling webhdfsconnector : ") val fStatusMap = WebHdfsConnector.getFileStatus(file, certValidation, "1000:5000", usrCred) fStatus = createFileStatus(f, fStatusMap) fileStatusMap.put(f.toString, fStatus) @@ -234,12 +240,14 @@ class BahirWebHdfsFileSystem extends FileSystem { override def create(srcPath: Path, permission: FsPermission, - flag: Boolean, + overwriteFlg: Boolean, bufferSize: Int, replication: Short, blockSize: Long, progress: Progressable): FSDataOutputStream = { - throw new Exception("File Create Not Yet Supported") + + new FSDataOutputStream(new BahirWebHdfsOutputStream(srcPath, bufferSize, blockSize, permission.toString, replication, + overwriteFlg, usrCred, certValidation), null) } private def getQryMapFromFilePath(f: Path): HashMap[String, String] = { @@ -305,7 +313,7 @@ class BahirWebHdfsInputStream(fPath: Path, } override def read(b: Array[Byte], offset: Int, length: Int): Int = { - if (in == null) createWebHdfsInputStream(pos) + //if (in == null) createWebHdfsInputStream(pos) callCount += 1 var bCount = in.read(b, offset, length) @@ -366,3 +374,56 @@ class BahirWebHdfsInputStream(fPath: Path, override def getPos(): Long = pos } + + +class BahirWebHdfsOutputStream(fPath: Path, + bufferSz: Int, + blockSz: Long, + permissionStr: String, + replicationCnt: Short, + overwriteFlg: Boolean, + usrCrd: String, + certValidation: String) + extends OutputStream { + + val filePath: Path = fPath + val bufferSize: Int = bufferSz + val blockSize: Long = blockSz + val permission: String = permissionStr + val replication: Short = replicationCnt + val overwrite: Boolean = overwriteFlg + val usrCred: String = usrCrd + val certValidationFlg: String = certValidation + + + override def write(b: Int): Unit = { + + println("in write single byte: " + b) + val singleByte : Array[Byte] = new Array(b)(1) + writeBytes(singleByte) + } + + override def write(b: Array[Byte]): Unit = { + + println("in write bytes ") + writeBytes(b) + + } + + override def write(b: Array[Byte], offset: Int, length: Int): Unit = { + + println("in write bytes with offset and length : " + offset + " , " + length) + writeBytes(b) + + } + + private def writeBytes(b: Array[Byte]): Unit = { + + println("in provate write bytes ") + WebHdfsConnector.writeFile(b, filePath.toString(), permissionStr, overwriteFlg, bufferSize, replication, blockSize, certValidation, "1000:5000", usrCred) + + } + +} + + diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala index cd2d963d..a51344a9 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala @@ -101,75 +101,87 @@ object WebHdfsConnector { // // } // - // /** - // * This function writes data back to hdfs using WebHDFS using multiple parallel connections. - // Right now file overwrite is not supported - // */ - // def writeToWebHdfs(dataToWrite: RDD[String] ,path: String, trustStoreCredStr: String, - // connStr : String, userCredStr : String, partitionStr : String): Unit = { - // - // val trustCred = createTrustStoreCredForExecutors(trustStoreCredStr, path) - // - // val conns = connStr.split(":") - // - // val conn = Array(conns(0).toInt, conns(1).toInt) - // - // val usr = userCredStr.split(":") - // - // val webHdfsChkDirOpr = "op=GETFILESTATUS" - // val returnChkDir = callWebHdfsAPI(path, "", "GET", "CODE", trustCred, usr, conn, - // webHdfsChkDirOpr) - // - // if (returnChkDir == "200") - // throw new Exception("The File Already Exists : " + path + "\n") - // + /** + * This function creates a directory in remote HDFS + */ + def makeDirectory(path : String, + trustStoreCredStr: String, + connStr : String, + userCredStr : String): Boolean = { + + println("In make directory - path : " + path) + val webHdfsChkDirOpr = "op=GETFILESTATUS" + val returnChkDir = callWebHdfsAPI(path, null, "GET", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsChkDirOpr, "STRING") + + if (returnChkDir == "200") + throw new Exception("The File Already Exists : " + path + "\n") + else { + // val dPartitions = partitionStr.toInt // // val textRdd = dataToWrite.repartition(dPartitions) + + val webHdfsMakeDirOpr = "op=MKDIRS" + val returnMakeDir = callWebHdfsAPI(path, null, "PUT", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsMakeDirOpr, "STRING") + println("In makeDirectory - return code : " + returnMakeDir) + true + } + } + + + /** + * This function deletes a file/directory recursively in remote HDFS + */ + def deleteFile(path : String, + recursiveFlg : Boolean, + trustStoreCredStr: String, + connStr : String, + userCredStr : String): Boolean = { + + println("In deleteFile - path : " + path + " , recusrsive flg : " + recursiveFlg) + val webHdfsChkDirOpr = "op=GETFILESTATUS" + val returnChkDir = callWebHdfsAPI(path, null, "GET", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsChkDirOpr, "STRING") + + if (returnChkDir != "200") + throw new Exception("The File/Directory Does Not Exist : " + path + "\n") + else { + + // val dPartitions = partitionStr.toInt // - // val webHdfsMakeDirOpr = "op=MKDIRS" - // val returnCreateDir = callWebHdfsAPI(path, "", "PUT", "CODE", trustCred, usr, conn, - // webHdfsMakeDirOpr) - // - // textRdd.mapPartitionsWithIndex((idx, iter) => WebHdfsConnector.saveAllFiles(idx, iter, usr, - // path, trustCred, conn)).collect() - // - // } - // - // /** - // * This function is passed to mapPartitionsWithIndex so that each executor task can save part - // of the data using separate connection - // */ - // def saveAllFiles (idx: Int, data : Iterator[String], usrCred: Array[String], path: String, - // trustCred: Tuple3[String, File , String], connProp: Array[Int]): Iterator[String] = { - // - // - // var dataP = data.next() - // while(data.hasNext) { - // dataP = dataP + "\n" + data.next() - // } - // - // val fnameArray = path.split("/") - // val fnameIdx = fnameArray.length - 1 - // val fname = fnameArray(fnameIdx) - // val filePath = s"$path/part-000$idx-$fname" - // - // val createOpr = "op=CREATE" - // - // val createUrl = callWebHdfsAPI(filePath, "", "PUT", "LOCATION", trustCred, usrCred, - // connProp, createOpr) - // val created = callWebHdfsAPI(createUrl, dataP, "PUT", "CODE", trustCred, usrCred, connProp, - // createOpr) - // - // val ret = Array(created.toString) - // ret.iterator - // - // } + // val textRdd = dataToWrite.repartition(dPartitions) + val webHdfsDeleteDirOpr = "op=DELETE&recursive=recursiveFlg" + val returnMakeDir = callWebHdfsAPI(path, null, "PUT", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsDeleteDirOpr, "STRING") + println("In deleteFile - return code : " + returnMakeDir) + true + } + } + + /** + * This function writes 1 file in remote HDFS + */ + def writeFile(data : Array[Byte], + path: String, + permission: String, + overwriteflag: Boolean, + bufferSize: Int, + replication: Short, + blockSize: Long, + trustStoreCredStr: String, + connStr : String, + userCredStr : String): Boolean = { + + val webHdfsCreateOpr = "op=CREATE&overwrite=overWriteFlg&blockSize=blockSize&replication=replication&permission=permission&bufferSize=bufferSize" + val createUrl = callWebHdfsAPI(path, null, "PUT", "LOCATION", trustStoreCredStr, userCredStr, connStr, webHdfsCreateOpr, "STRING").asInstanceOf[String] + val createdCode = callWebHdfsAPI(createUrl, data, "PUT", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsCreateOpr, "STRING") + println("In save file - return code : " + createdCode) + true + + } def callWebHdfsAPI(path: String, - data: String, + data: Array[Byte], method: String, respType: String, trustStoreCredStr: String, @@ -198,8 +210,8 @@ object WebHdfsConnector { httpc = (method : @switch) match { case "GET" => httpc - case "PUT" => httpc.put(data).header("content-type", "application/csv") - case "POST" => httpc.postData(data).header("content-type", "application/csv") + case "PUT" => httpc.put(data).header("content-type", "application/bahir-webhdfs") + case "POST" => httpc.postData(data).header("content-type", "application/bahir-webhdfs") } httpc = (trustCred._1 : @switch) match { @@ -249,10 +261,10 @@ object WebHdfsConnector { s"op=OPEN&offset=$offset" } - val getUrl = callWebHdfsAPI(filePath, "", "GET", "LOCATION", trustStoreCredStr, usrCredStr, + val getUrl = callWebHdfsAPI(filePath, null, "GET", "LOCATION", trustStoreCredStr, usrCredStr, connStr, fileGetOpr, "String").asInstanceOf[String] - val content = callWebHdfsAPI(getUrl, "", "GET", "BODY", trustStoreCredStr, usrCredStr, + val content = callWebHdfsAPI(getUrl, null, "GET", "BODY", trustStoreCredStr, usrCredStr, connStr, fileGetOpr, "Bytes").asInstanceOf[Array[Byte]] new ByteArrayInputStream(content) @@ -263,20 +275,32 @@ object WebHdfsConnector { connStr: String, usrCredStr: String): Map[String, Any] = { - // print("path in getFileStatus : " + filePath + "\n") + print("path in getFileStatus : " + filePath + "\n") val fileStatusOpr = s"op=GETFILESTATUS" + val returnChk = callWebHdfsAPI(filePath, null, "GET", "CODE", trustStoreCredStr, usrCredStr, connStr, fileStatusOpr, "STRING") + print("after file status check in getFileStatus : " + returnChk + "\n") - val fileStatus = callWebHdfsAPI(filePath, "", "GET", "BODY", trustStoreCredStr, usrCredStr, - connStr, fileStatusOpr, "String").asInstanceOf[String] + if (returnChk == "200") { - if (fileStatus.contains("RemoteException")) { - throw new Exception(fileStatus) - } + print("within return code 200 in getFileStatus : " + returnChk + "\n") + val fileStatus = callWebHdfsAPI(filePath, null, "GET", "BODY", trustStoreCredStr, usrCredStr, + connStr, fileStatusOpr, "String").asInstanceOf[String] + + if (fileStatus.contains("RemoteException")) { + print("within remote exception in getFileStatus : " + returnChk + "\n") + //throw new Exception(fileStatus) + } - val responseMap = scala.util.parsing.json.JSON.parseFull(fileStatus).toList(0) - .asInstanceOf[Map[String, Map[String, Any]]] + val responseMap = scala.util.parsing.json.JSON.parseFull(fileStatus).toList(0) + .asInstanceOf[Map[String, Map[String, Any]]] - responseMap.getOrElse("FileStatus", throw new Exception ("File Status Null")) + responseMap.getOrElse("FileStatus", null) + } + else { + + print("file does not exist : " + filePath + "\n") + null + } } def getListStatus(filePath: String, @@ -286,7 +310,7 @@ object WebHdfsConnector { // print("path in getListStatus : " + filePath + "\n") val listStatusOpr = s"op=LISTSTATUS" - val listStatus = callWebHdfsAPI(filePath, "", "GET", "BODY", trustStoreCredStr, usrCredStr, + val listStatus = callWebHdfsAPI(filePath, null, "GET", "BODY", trustStoreCredStr, usrCredStr, connStr, listStatusOpr, "String").asInstanceOf[String] if (listStatus.contains("RemoteException")) { From 183b1ec88b26c3de578bab4d2b8eb9ebb1c80e69 Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Fri, 9 Dec 2016 11:11:15 -0800 Subject: [PATCH 19/24] [BAHIR-75][WIP] - override WebHdfsFileSystem - fix code style errors, temporarily disable style checks for println --- .../webhdfs/BahirWebHdfsDataSetWrapper.scala | 43 ++--- .../webhdfs/util/WebHdfsConnector.scala | 172 ++++++++++-------- 2 files changed, 111 insertions(+), 104 deletions(-) diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala index 43c11cba..3f861584 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala @@ -34,11 +34,11 @@ import org.apache.hadoop.util.Progressable * This class contains functions for reading/writing data from/to remote webhdfs server in Spark * DataSource */ +// TODO: reenable scalastyle checks +// scalastyle:off println class BahirWebHdfsFileSystem extends FileSystem { - // scalastyle:off println println(s" - - - BahirWebHdfsFileSystem class loaded - - - ") - // scalastyle:on println var uri: URI = null var rHdfsUri: URI = null @@ -85,10 +85,7 @@ class BahirWebHdfsFileSystem extends FileSystem { uri = URI.create(uriOrg.getScheme() + "://" + uriOrg.getAuthority()) - // scalastyle:off println println(s"BahirWebHdfsFileSystem: uri=${uri}, connections=${connections}, usercred=${usrCred}") - // scalastyle:on println - } override def getWorkingDirectory(): Path = { @@ -128,7 +125,7 @@ class BahirWebHdfsFileSystem extends FileSystem { println("In bahir after checking filestatus map - : " + fStatus) val fileStatus = if (fStatus == null) { - println("In bahir before calling webhdfsconnector : ") + println("In bahir before calling webhdfsconnector : ") val fStatusMap = WebHdfsConnector.getFileStatus(file, certValidation, "1000:5000", usrCred) fStatus = createFileStatus(f, fStatusMap) fileStatusMap.put(f.toString, fStatus) @@ -246,8 +243,8 @@ class BahirWebHdfsFileSystem extends FileSystem { blockSize: Long, progress: Progressable): FSDataOutputStream = { - new FSDataOutputStream(new BahirWebHdfsOutputStream(srcPath, bufferSize, blockSize, permission.toString, replication, - overwriteFlg, usrCred, certValidation), null) + new FSDataOutputStream(new BahirWebHdfsOutputStream(srcPath, bufferSize, blockSize, + permission.toString, replication, overwriteFlg, usrCred, certValidation), null) } private def getQryMapFromFilePath(f: Path): HashMap[String, String] = { @@ -313,7 +310,7 @@ class BahirWebHdfsInputStream(fPath: Path, } override def read(b: Array[Byte], offset: Int, length: Int): Int = { - //if (in == null) createWebHdfsInputStream(pos) +// if (in == null) createWebHdfsInputStream(pos) callCount += 1 var bCount = in.read(b, offset, length) @@ -377,13 +374,13 @@ class BahirWebHdfsInputStream(fPath: Path, class BahirWebHdfsOutputStream(fPath: Path, - bufferSz: Int, - blockSz: Long, - permissionStr: String, - replicationCnt: Short, - overwriteFlg: Boolean, - usrCrd: String, - certValidation: String) + bufferSz: Int, + blockSz: Long, + permissionStr: String, + replicationCnt: Short, + overwriteFlg: Boolean, + usrCrd: String, + certValidation: String) extends OutputStream { val filePath: Path = fPath @@ -399,31 +396,27 @@ class BahirWebHdfsOutputStream(fPath: Path, override def write(b: Int): Unit = { println("in write single byte: " + b) - val singleByte : Array[Byte] = new Array(b)(1) + val singleByte : Array[Byte] = new Array(b)(1) writeBytes(singleByte) } override def write(b: Array[Byte]): Unit = { - println("in write bytes ") writeBytes(b) - } override def write(b: Array[Byte], offset: Int, length: Int): Unit = { - println("in write bytes with offset and length : " + offset + " , " + length) writeBytes(b) - } private def writeBytes(b: Array[Byte]): Unit = { - println("in provate write bytes ") - WebHdfsConnector.writeFile(b, filePath.toString(), permissionStr, overwriteFlg, bufferSize, replication, blockSize, certValidation, "1000:5000", usrCred) - + WebHdfsConnector.writeFile(b, filePath.toString(), permissionStr, overwriteFlg, bufferSize, + replication, blockSize, certValidation, "1000:5000", usrCred) } } - +// TODO: reenable scalastyle checks +// scalastyle:on println diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala index a51344a9..3848853b 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala @@ -29,7 +29,9 @@ import scalaj.http.{Http, HttpOptions} * This object contains all utility functions for reading/writing data from/to remote webhdfs * server. The abstraction maintained in this layer is at the level of RDD */ -object WebHdfsConnector { +// TODO: reenable scalastyle checks +// scalastyle:off println +object WebHdfsConnector { /* * This function returns a Tuple for credential store which contains flag for validating @@ -102,78 +104,87 @@ object WebHdfsConnector { // } // /** - * This function creates a directory in remote HDFS - */ - def makeDirectory(path : String, - trustStoreCredStr: String, - connStr : String, - userCredStr : String): Boolean = { - - println("In make directory - path : " + path) + * This function creates a directory in remote HDFS + */ + def makeDirectory(path: String, + trustStoreCredStr: String, + connStr: String, + userCredStr: String): Boolean = { + + println("In make directory - path : " + path) val webHdfsChkDirOpr = "op=GETFILESTATUS" - val returnChkDir = callWebHdfsAPI(path, null, "GET", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsChkDirOpr, "STRING") - - if (returnChkDir == "200") - throw new Exception("The File Already Exists : " + path + "\n") + val returnChkDir = callWebHdfsAPI(path, null, "GET", "CODE", trustStoreCredStr, userCredStr, + connStr, webHdfsChkDirOpr, "STRING") + + if (returnChkDir == "200") { + throw new Exception("The File Already Exists : " + path + "\n") + } else { - - // val dPartitions = partitionStr.toInt - // - // val textRdd = dataToWrite.repartition(dPartitions) - val webHdfsMakeDirOpr = "op=MKDIRS" - val returnMakeDir = callWebHdfsAPI(path, null, "PUT", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsMakeDirOpr, "STRING") - println("In makeDirectory - return code : " + returnMakeDir) - true + // val dPartitions = partitionStr.toInt + // + // val textRdd = dataToWrite.repartition(dPartitions) + + val webHdfsMakeDirOpr = "op=MKDIRS" + val returnMakeDir = callWebHdfsAPI(path, null, "PUT", "CODE", trustStoreCredStr, userCredStr, + connStr, webHdfsMakeDirOpr, "STRING") + println("In makeDirectory - return code : " + returnMakeDir) + true } } - + /** - * This function deletes a file/directory recursively in remote HDFS - */ - def deleteFile(path : String, - recursiveFlg : Boolean, - trustStoreCredStr: String, - connStr : String, - userCredStr : String): Boolean = { - - println("In deleteFile - path : " + path + " , recusrsive flg : " + recursiveFlg) + * This function deletes a file/directory recursively in remote HDFS + */ + def deleteFile(path: String, + recursiveFlg: Boolean, + trustStoreCredStr: String, + connStr: String, + userCredStr: String): Boolean = { + + println("In deleteFile - path : " + path + " , recusrsive flg : " + recursiveFlg) val webHdfsChkDirOpr = "op=GETFILESTATUS" - val returnChkDir = callWebHdfsAPI(path, null, "GET", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsChkDirOpr, "STRING") - - if (returnChkDir != "200") - throw new Exception("The File/Directory Does Not Exist : " + path + "\n") + val returnChkDir = callWebHdfsAPI(path, null, "GET", "CODE", trustStoreCredStr, userCredStr, + connStr, webHdfsChkDirOpr, "STRING") + + if (returnChkDir != "200") { + throw new Exception("The File/Directory Does Not Exist : " + path + "\n") + } else { - - // val dPartitions = partitionStr.toInt - // - // val textRdd = dataToWrite.repartition(dPartitions) - val webHdfsDeleteDirOpr = "op=DELETE&recursive=recursiveFlg" - val returnMakeDir = callWebHdfsAPI(path, null, "PUT", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsDeleteDirOpr, "STRING") - println("In deleteFile - return code : " + returnMakeDir) - true + // val dPartitions = partitionStr.toInt + // + // val textRdd = dataToWrite.repartition(dPartitions) + + val webHdfsDeleteDirOpr = "op=DELETE&recursive=recursiveFlg" + val returnMakeDir = callWebHdfsAPI(path, null, "PUT", "CODE", trustStoreCredStr, userCredStr, + connStr, webHdfsDeleteDirOpr, "STRING") + println("In deleteFile - return code : " + returnMakeDir) + true } } /** - * This function writes 1 file in remote HDFS - */ - def writeFile(data : Array[Byte], + * This function writes 1 file in remote HDFS + */ + def writeFile(data: Array[Byte], path: String, permission: String, overwriteflag: Boolean, bufferSize: Int, replication: Short, blockSize: Long, - trustStoreCredStr: String, - connStr : String, - userCredStr : String): Boolean = { - - val webHdfsCreateOpr = "op=CREATE&overwrite=overWriteFlg&blockSize=blockSize&replication=replication&permission=permission&bufferSize=bufferSize" - val createUrl = callWebHdfsAPI(path, null, "PUT", "LOCATION", trustStoreCredStr, userCredStr, connStr, webHdfsCreateOpr, "STRING").asInstanceOf[String] - val createdCode = callWebHdfsAPI(createUrl, data, "PUT", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsCreateOpr, "STRING") + trustStoreCredStr: String, + connStr: String, + userCredStr: String): Boolean = { + + val webHdfsCreateOpr = "op=CREATE&overwrite=overWriteFlg&blockSize=blockSize" + + "&replication=replication&permission=permission&bufferSize=bufferSize" + val createUrl = callWebHdfsAPI(path, null, "PUT", "LOCATION", trustStoreCredStr, userCredStr, + connStr, webHdfsCreateOpr, "STRING").asInstanceOf[String] + val createdCode = callWebHdfsAPI(createUrl, data, "PUT", "CODE", trustStoreCredStr, userCredStr, + connStr, webHdfsCreateOpr, "STRING") println("In save file - return code : " + createdCode) true @@ -188,7 +199,7 @@ object WebHdfsConnector { usrCredStr: String, connStr: String, opr: String, - outputType : String): Any = { + outputType: String): Any = { // print("path in callWebHdfs : " + path + "\n") @@ -208,26 +219,26 @@ object WebHdfsConnector { var httpc = Http(uri).auth(usrCred(0), usrCred(1)).timeout(connTimeoutMs = connProp(0), readTimeoutMs = connProp(1)) - httpc = (method : @switch) match { + httpc = (method: @switch) match { case "GET" => httpc case "PUT" => httpc.put(data).header("content-type", "application/bahir-webhdfs") case "POST" => httpc.postData(data).header("content-type", "application/bahir-webhdfs") } - httpc = (trustCred._1 : @switch) match { + httpc = (trustCred._1: @switch) match { case "" => httpc case "N" => httpc.option(HttpOptions.allowUnsafeSSL) case "Y" => httpc.option(HttpOptions.sslSocketFactory(biocSslSocketFactory(trustCred._2, trustCred._3))) } - val out = (outputType : @switch) match { + val out = (outputType: @switch) match { case "" => httpc.asString case "String" => httpc.asString case "Bytes" => httpc.asBytes } - val resp = (respType : @switch) match { + val resp = (respType: @switch) match { case "BODY" => out.body case "CODE" => out.code case "HEADERS" => out.headers @@ -246,12 +257,12 @@ object WebHdfsConnector { resp } - def getFileInputStream (filePath: String, - offset: Long, - length: Long, - trustStoreCredStr: String, - connStr: String, - usrCredStr: String): ByteArrayInputStream = { + def getFileInputStream(filePath: String, + offset: Long, + length: Long, + trustStoreCredStr: String, + connStr: String, + usrCredStr: String): ByteArrayInputStream = { // print("path in getFileInputStream : " + filePath + "\n") @@ -277,29 +288,30 @@ object WebHdfsConnector { print("path in getFileStatus : " + filePath + "\n") val fileStatusOpr = s"op=GETFILESTATUS" - val returnChk = callWebHdfsAPI(filePath, null, "GET", "CODE", trustStoreCredStr, usrCredStr, connStr, fileStatusOpr, "STRING") + val returnChk = callWebHdfsAPI(filePath, null, "GET", "CODE", trustStoreCredStr, usrCredStr, + connStr, fileStatusOpr, "STRING") print("after file status check in getFileStatus : " + returnChk + "\n") - if (returnChk == "200") { + if (returnChk == "200") { - print("within return code 200 in getFileStatus : " + returnChk + "\n") - val fileStatus = callWebHdfsAPI(filePath, null, "GET", "BODY", trustStoreCredStr, usrCredStr, - connStr, fileStatusOpr, "String").asInstanceOf[String] + print("within return code 200 in getFileStatus : " + returnChk + "\n") + val fileStatus = callWebHdfsAPI(filePath, null, "GET", "BODY", trustStoreCredStr, usrCredStr, + connStr, fileStatusOpr, "String").asInstanceOf[String] - if (fileStatus.contains("RemoteException")) { - print("within remote exception in getFileStatus : " + returnChk + "\n") - //throw new Exception(fileStatus) - } + if (fileStatus.contains("RemoteException")) { + print("within remote exception in getFileStatus : " + returnChk + "\n") +// throw new Exception(fileStatus) + } - val responseMap = scala.util.parsing.json.JSON.parseFull(fileStatus).toList(0) - .asInstanceOf[Map[String, Map[String, Any]]] + val responseMap = scala.util.parsing.json.JSON.parseFull(fileStatus).toList(0) + .asInstanceOf[Map[String, Map[String, Any]]] - responseMap.getOrElse("FileStatus", null) + responseMap.getOrElse("FileStatus", null) } else { - - print("file does not exist : " + filePath + "\n") - null + + print("file does not exist : " + filePath + "\n") + null } } @@ -324,3 +336,5 @@ object WebHdfsConnector { } +// TODO: reenable scalastyle checks +// scalastyle:on println From 59cad8ea9fbbcc4951b2a8a121e736e4121a31a7 Mon Sep 17 00:00:00 2001 From: Sourav Mazumder Date: Mon, 12 Dec 2016 14:38:49 -0800 Subject: [PATCH 20/24] [BAHIR-75][WIP] - write files via webhdfs --- .../webhdfs/BahirWebHdfsDataSetWrapper.scala | 62 +++++---- .../webhdfs/util/SSLTrustStoreUtil.scala | 2 + .../webhdfs/util/WebHdfsConnector.scala | 124 ++++++++++++------ 3 files changed, 126 insertions(+), 62 deletions(-) diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala index 3f861584..f9a0493d 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala @@ -98,18 +98,26 @@ class BahirWebHdfsFileSystem extends FileSystem { override def setWorkingDirectory(dir: Path): Unit = {} override def rename(srcPath: Path, destPath: Path): Boolean = { - throw new Exception("File Rename Not Supported") + + //val destPathModStr = destPath.toString.replace("default/gateway/webhdfs/v1/", "")(1) + val destPathModStr = Path.getPathWithoutSchemeAndAuthority(destPath).toString.replace("/gateway/default/webhdfs/v1","") + println("In rename - Path : " + srcPath + " , dest path : " + destPath + " , mod dest path : "+ destPathModStr) + //println("In rename after mod - Path : " + srcPathModStr + " , dest path : " + destPathModStr) + WebHdfsConnector.renameFile(srcPath.toString, destPathModStr, certValidation, "1000:5000", usrCred) } override def delete(srcPath: Path, recursive: Boolean): Boolean = { println("In delete - Path : " + srcPath + " , recursive flg : " + recursive) - WebHdfsConnector.deleteFile(srcPath.toString(), recursive, certValidation, "1000:5000", usrCred) - throw new Exception("File Delete Not Supported") + val srcPathModStr = modifyFilePath(srcPath).toString + println("In delete after mod - Path : " + srcPathModStr + " , recursive flg : " + recursive) + WebHdfsConnector.deleteFile(srcPathModStr, recursive, certValidation, "1000:5000", usrCred) } override def mkdirs(srcPath: Path, permission: FsPermission): Boolean = { println("In MkDirs - Path : " + srcPath) - WebHdfsConnector.makeDirectory(srcPath.toString(), certValidation, "1000:5000", usrCred) + val srcPathModStr = modifyFilePath(srcPath).toString + println("In MkDirs after mod - Path : " + srcPathModStr) + WebHdfsConnector.makeDirectory(srcPathModStr, permission.toShort, certValidation, "1000:5000", usrCred) } override def append(srcPath: Path, @@ -119,30 +127,33 @@ class BahirWebHdfsFileSystem extends FileSystem { } override def getFileStatus(f: Path): FileStatus = { - val file = stripQryFromFilePath(f).toString - println("In bahir get filestatus - Path : " + f) - var fStatus: FileStatus = fileStatusMap.getOrElse(file, null) + val file = modifyFilePath(f).toString + println("In bahir get filestatus - Path , and mod Path: " + f + file + "\n") + var fStatus: FileStatus = fileStatusMap.getOrElse(f.toString, null) println("In bahir after checking filestatus map - : " + fStatus) val fileStatus = if (fStatus == null) { println("In bahir before calling webhdfsconnector : ") val fStatusMap = WebHdfsConnector.getFileStatus(file, certValidation, "1000:5000", usrCred) - fStatus = createFileStatus(f, fStatusMap) - fileStatusMap.put(f.toString, fStatus) - fStatus + if (fStatusMap != null) { + fStatus = createFileStatus(f, fStatusMap) + fileStatusMap.put(f.toString, fStatus) + } + fStatus } else { fStatus } + println("In bahir before returning from getFileStatis fileStatus : " + fileStatus) fileStatus } override def listStatus(f: Path): Array[FileStatus] = { - val file = stripQryFromFilePath(f).toString + val file = modifyFilePath(f).toString - var lStatus: Array[FileStatus] = listStatusMap.getOrElse(file, null) + var lStatus: Array[FileStatus] = listStatusMap.getOrElse(f.toString, null) // println("file in listStatus: " + file) @@ -193,10 +204,11 @@ class BahirWebHdfsFileSystem extends FileSystem { fStatus } - private def stripQryFromFilePath(f: Path): Path = { + private def modifyFilePath(f: Path): Path = { // println("file uri : " + f.toUri) - val pathStrWithoutQry = f.toString.replace(getQryStrFromFilePath(f), "") - new Path(pathStrWithoutQry) + val wQryStr = f.toString.replace(getQryStrFromFilePath(f), "") + //val fStr = wQryStr.replace("_temporary", "bahir_tmp") + new Path(wQryStr) } private def getQryStrFromFilePath(f: Path): String = { @@ -214,13 +226,15 @@ class BahirWebHdfsFileSystem extends FileSystem { override def open(f: Path, bs: Int): FSDataInputStream = { + println("In bahir open file path - : " + f) + val fileStatus = getFileStatus(f) val blockSize = fileStatus.getBlockSize val fileLength = fileStatus.getLen - val file = stripQryFromFilePath(f) + val file = modifyFilePath(f) - print("file uri in open : " + file + "\n") + print("file uri in open after modification : " + file + "\n") val qMap = getQryMapFromFilePath(f) @@ -243,8 +257,12 @@ class BahirWebHdfsFileSystem extends FileSystem { blockSize: Long, progress: Progressable): FSDataOutputStream = { - new FSDataOutputStream(new BahirWebHdfsOutputStream(srcPath, bufferSize, blockSize, - permission.toString, replication, overwriteFlg, usrCred, certValidation), null) + val file = modifyFilePath(srcPath) + + print("file uri in create after modification : " + file + "\n") + + new FSDataOutputStream(new BahirWebHdfsOutputStream(file, bufferSize, blockSize, + permission.toShort, replication, overwriteFlg, usrCred, certValidation), null) } private def getQryMapFromFilePath(f: Path): HashMap[String, String] = { @@ -376,7 +394,7 @@ class BahirWebHdfsInputStream(fPath: Path, class BahirWebHdfsOutputStream(fPath: Path, bufferSz: Int, blockSz: Long, - permissionStr: String, + perms: Short, replicationCnt: Short, overwriteFlg: Boolean, usrCrd: String, @@ -386,7 +404,7 @@ class BahirWebHdfsOutputStream(fPath: Path, val filePath: Path = fPath val bufferSize: Int = bufferSz val blockSize: Long = blockSz - val permission: String = permissionStr + val permission: Short = perms val replication: Short = replicationCnt val overwrite: Boolean = overwriteFlg val usrCred: String = usrCrd @@ -412,7 +430,7 @@ class BahirWebHdfsOutputStream(fPath: Path, private def writeBytes(b: Array[Byte]): Unit = { println("in provate write bytes ") - WebHdfsConnector.writeFile(b, filePath.toString(), permissionStr, overwriteFlg, bufferSize, + WebHdfsConnector.writeFile(b, filePath.toString, permission, overwriteFlg, bufferSize, replication, blockSize, certValidation, "1000:5000", usrCred) } diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala index 1ab14176..2e48f7c1 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala @@ -37,6 +37,8 @@ object SSLTrustStore { */ def getCertDetails(path: String): Tuple2[File, String] = { + print("path in ssltrust store getCertDetails : " + path + "\n") + val pathComp = path.split("/") val srvr = pathComp(2) diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala index 3848853b..38ed32de 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala @@ -107,6 +107,7 @@ object WebHdfsConnector { * This function creates a directory in remote HDFS */ def makeDirectory(path: String, + permission: Short, trustStoreCredStr: String, connStr: String, userCredStr: String): Boolean = { @@ -114,10 +115,10 @@ object WebHdfsConnector { println("In make directory - path : " + path) val webHdfsChkDirOpr = "op=GETFILESTATUS" val returnChkDir = callWebHdfsAPI(path, null, "GET", "CODE", trustStoreCredStr, userCredStr, - connStr, webHdfsChkDirOpr, "STRING") + connStr, webHdfsChkDirOpr, "String").asInstanceOf[Integer] - if (returnChkDir == "200") { - throw new Exception("The File Already Exists : " + path + "\n") + if (returnChkDir == 200) { + throw new Exception("The Directory Already Exists : " + path + "\n") } else { @@ -125,11 +126,15 @@ object WebHdfsConnector { // // val textRdd = dataToWrite.repartition(dPartitions) - val webHdfsMakeDirOpr = "op=MKDIRS" - val returnMakeDir = callWebHdfsAPI(path, null, "PUT", "CODE", trustStoreCredStr, userCredStr, - connStr, webHdfsMakeDirOpr, "STRING") - println("In makeDirectory - return code : " + returnMakeDir) - true + val webHdfsMakeDirOpr = s"op=MKDIRS&permission=$permission" + val returnMakeDir = callWebHdfsAPI(path, "".getBytes(), "PUT", "CODE", trustStoreCredStr, userCredStr, + connStr, webHdfsMakeDirOpr, "String").asInstanceOf[Integer] + println("In makeDirectory - return code : " + returnMakeDir) + if(returnMakeDir != 200) + throw new Exception("The Directory could not be created , Src path, and code: " + path + " , " + returnMakeDir + "\n") + //false + else + true } } @@ -146,22 +151,22 @@ object WebHdfsConnector { println("In deleteFile - path : " + path + " , recusrsive flg : " + recursiveFlg) val webHdfsChkDirOpr = "op=GETFILESTATUS" val returnChkDir = callWebHdfsAPI(path, null, "GET", "CODE", trustStoreCredStr, userCredStr, - connStr, webHdfsChkDirOpr, "STRING") + connStr, webHdfsChkDirOpr, "String").asInstanceOf[Integer] - if (returnChkDir != "200") { + if (returnChkDir != 200) { throw new Exception("The File/Directory Does Not Exist : " + path + "\n") } else { - // val dPartitions = partitionStr.toInt - // - // val textRdd = dataToWrite.repartition(dPartitions) - - val webHdfsDeleteDirOpr = "op=DELETE&recursive=recursiveFlg" - val returnMakeDir = callWebHdfsAPI(path, null, "PUT", "CODE", trustStoreCredStr, userCredStr, - connStr, webHdfsDeleteDirOpr, "STRING") - println("In deleteFile - return code : " + returnMakeDir) - true + val webHdfsDeleteDirOpr = s"op=DELETE&recursive=$recursiveFlg" + val returnDelDir = callWebHdfsAPI(path, null, "DELETE", "CODE", trustStoreCredStr, userCredStr, + connStr, webHdfsDeleteDirOpr, "String").asInstanceOf[Integer] + println("In deleteFile - return code : " + returnDelDir) + if(returnDelDir != 200) + throw new Exception("The File/Directory could not be renamed , Src path, Dest path and code: " + path + " , " + returnDelDir + "\n") + //false + else + true } } @@ -170,8 +175,8 @@ object WebHdfsConnector { */ def writeFile(data: Array[Byte], path: String, - permission: String, - overwriteflag: Boolean, + permission: Short, + overwriteFlg: Boolean, bufferSize: Int, replication: Short, blockSize: Long, @@ -179,17 +184,49 @@ object WebHdfsConnector { connStr: String, userCredStr: String): Boolean = { - val webHdfsCreateOpr = "op=CREATE&overwrite=overWriteFlg&blockSize=blockSize" + - "&replication=replication&permission=permission&bufferSize=bufferSize" - val createUrl = callWebHdfsAPI(path, null, "PUT", "LOCATION", trustStoreCredStr, userCredStr, - connStr, webHdfsCreateOpr, "STRING").asInstanceOf[String] + val webHdfsCreateOpr = s"op=CREATE&overwrite=$overwriteFlg&blockSize=$blockSize" + + s"&replication=$replication&bufferSize=$bufferSize&permission=$permission" + val createUrl = callWebHdfsAPI(path, "".getBytes(), "PUT", "LOCATION", trustStoreCredStr, userCredStr, + connStr, webHdfsCreateOpr, "String").asInstanceOf[String] val createdCode = callWebHdfsAPI(createUrl, data, "PUT", "CODE", trustStoreCredStr, userCredStr, - connStr, webHdfsCreateOpr, "STRING") + connStr, webHdfsCreateOpr, "String").asInstanceOf[Integer] println("In save file - return code : " + createdCode) true } + /** + * This function renames 1 file in remote HDFS + */ + def renameFile(path: String, + destPath: String, + trustStoreCredStr: String, + connStr: String, + userCredStr: String): Boolean = { + + println("In rename file - path : " + path + " , dest path : " + destPath + "\n") + val webHdfsChkFileOpr = "op=GETFILESTATUS" + val returnChkFile = callWebHdfsAPI(path, null, "GET", "CODE", trustStoreCredStr, userCredStr, + connStr, webHdfsChkFileOpr, "String").asInstanceOf[Integer] + + if (returnChkFile != 200) { + throw new Exception("The File/Directory Does Not Exist , path and code: " + path + " , " + returnChkFile + "\n") + } + else { + val webHdfsRenameOpr = s"op=RENAME&destination=$destPath" + val returnRename = callWebHdfsAPI(path, "".getBytes(), "PUT", "CODE", trustStoreCredStr, userCredStr, + connStr, webHdfsRenameOpr, "String").asInstanceOf[Integer] + println("In Rename - return code : " + returnRename) + if(returnRename != 200) + throw new Exception("The File/Directory could not be renamed , Src path, Dest path and code: " + path + " , " + destPath + " , " + returnRename + "\n") + //false + else + true + } + + } + + def callWebHdfsAPI(path: String, data: Array[Byte], @@ -201,7 +238,7 @@ object WebHdfsConnector { opr: String, outputType: String): Any = { - // print("path in callWebHdfs : " + path + "\n") + print("path in callWebHdfs : " + path + " , opr : " + opr + "\n") val pathComp = path.split(":") @@ -222,6 +259,7 @@ object WebHdfsConnector { httpc = (method: @switch) match { case "GET" => httpc case "PUT" => httpc.put(data).header("content-type", "application/bahir-webhdfs") + case "DELETE" => httpc.method("DELETE") case "POST" => httpc.postData(data).header("content-type", "application/bahir-webhdfs") } @@ -232,6 +270,8 @@ object WebHdfsConnector { trustCred._3))) } + /* + val out = (outputType: @switch) match { case "" => httpc.asString case "String" => httpc.asString @@ -244,15 +284,14 @@ object WebHdfsConnector { case "HEADERS" => out.headers case "LOCATION" => out.location.mkString(" ") } + */ - /* val resp = (respType : @switch) match { case "BODY" => httpc.asBytes.body case "CODE" => httpc.asString.code case "HEADERS" => httpc.asString.headers case "LOCATION" => httpc.asString.location.mkString(" ") } - */ resp } @@ -289,24 +328,27 @@ object WebHdfsConnector { print("path in getFileStatus : " + filePath + "\n") val fileStatusOpr = s"op=GETFILESTATUS" val returnChk = callWebHdfsAPI(filePath, null, "GET", "CODE", trustStoreCredStr, usrCredStr, - connStr, fileStatusOpr, "STRING") + connStr, fileStatusOpr, "String").asInstanceOf[Integer] print("after file status check in getFileStatus : " + returnChk + "\n") - if (returnChk == "200") { + if (returnChk == 200) { print("within return code 200 in getFileStatus : " + returnChk + "\n") - val fileStatus = callWebHdfsAPI(filePath, null, "GET", "BODY", trustStoreCredStr, usrCredStr, - connStr, fileStatusOpr, "String").asInstanceOf[String] + val fileStatusBytes = callWebHdfsAPI(filePath, null, "GET", "BODY", trustStoreCredStr, usrCredStr, + connStr, fileStatusOpr, "String").asInstanceOf[Array[Byte]] + + val fileStatus = new String(fileStatusBytes, "UTF-8") if (fileStatus.contains("RemoteException")) { print("within remote exception in getFileStatus : " + returnChk + "\n") -// throw new Exception(fileStatus) + null } + else { + val responseMap = scala.util.parsing.json.JSON.parseFull(fileStatus).toList(0) + .asInstanceOf[Map[String, Map[String, Any]]] - val responseMap = scala.util.parsing.json.JSON.parseFull(fileStatus).toList(0) - .asInstanceOf[Map[String, Map[String, Any]]] - - responseMap.getOrElse("FileStatus", null) + responseMap.getOrElse("FileStatus", null) + } } else { @@ -322,8 +364,10 @@ object WebHdfsConnector { // print("path in getListStatus : " + filePath + "\n") val listStatusOpr = s"op=LISTSTATUS" - val listStatus = callWebHdfsAPI(filePath, null, "GET", "BODY", trustStoreCredStr, usrCredStr, - connStr, listStatusOpr, "String").asInstanceOf[String] + val listStatusBytes = callWebHdfsAPI(filePath, null, "GET", "BODY", trustStoreCredStr, usrCredStr, + connStr, listStatusOpr, "String").asInstanceOf[Array[Byte]] + + val listStatus = new String(listStatusBytes, "UTF-8") if (listStatus.contains("RemoteException")) { throw new Exception(listStatus) From d047318a899636242ac83471f58c044dc7be9f45 Mon Sep 17 00:00:00 2001 From: Sourav Mazumder Date: Thu, 22 Dec 2016 12:22:00 -0800 Subject: [PATCH 21/24] [BAHIR-75][WIP] - write files via webhdfs continued --- .../webhdfs/BahirWebHdfsDataSetWrapper.scala | 158 ++++++----- .../webhdfs/util/WebHdfsConnector.scala | 250 ++++++++---------- 2 files changed, 199 insertions(+), 209 deletions(-) diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala index f9a0493d..01a774ff 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala @@ -69,23 +69,19 @@ class BahirWebHdfsFileSystem extends FileSystem { rHdfsUri = uriOrg conf = confOrg - val rfFlg = conf.get("readFullFile") - readFullFile = if (rfFlg == null) false else rfFlg.toBoolean - val usrCredStr = conf.get("usrCredStr") usrCred = if (usrCredStr == null) { throw new Exception("User Credential Has To Be Specified For The Remote HDFS") } else usrCredStr.toString - val conns = conf.get("connections") - connections = if (conns == null) 0 else conns.toInt + val certFile = conf.get("certTrustStoreFile") + val certPwd = conf.get("certTrustStorePwd") - val certFlg = conf.get("certValidationFlg") - certValidation = if (certFlg == null) "Y" else certFlg.toString + certValidation = if (certFile == null || certPwd == null) "N" else s"$certFile:$certPwd" uri = URI.create(uriOrg.getScheme() + "://" + uriOrg.getAuthority()) - println(s"BahirWebHdfsFileSystem: uri=${uri}, connections=${connections}, usercred=${usrCred}") + // println(s"BahirWebHdfsFileSystem: uri=${uri}, connections=${connections}, usercred=${usrCred}") } override def getWorkingDirectory(): Path = { @@ -99,25 +95,20 @@ class BahirWebHdfsFileSystem extends FileSystem { override def rename(srcPath: Path, destPath: Path): Boolean = { - //val destPathModStr = destPath.toString.replace("default/gateway/webhdfs/v1/", "")(1) - val destPathModStr = Path.getPathWithoutSchemeAndAuthority(destPath).toString.replace("/gateway/default/webhdfs/v1","") - println("In rename - Path : " + srcPath + " , dest path : " + destPath + " , mod dest path : "+ destPathModStr) - //println("In rename after mod - Path : " + srcPathModStr + " , dest path : " + destPathModStr) - WebHdfsConnector.renameFile(srcPath.toString, destPathModStr, certValidation, "1000:5000", usrCred) + val destPathModStr = Path.getPathWithoutSchemeAndAuthority(destPath).toString. + replace("/gateway/default/webhdfs/v1", "") + WebHdfsConnector. + renameFile(srcPath.toString, destPathModStr, certValidation, "10000:120000", usrCred) } override def delete(srcPath: Path, recursive: Boolean): Boolean = { - println("In delete - Path : " + srcPath + " , recursive flg : " + recursive) - val srcPathModStr = modifyFilePath(srcPath).toString - println("In delete after mod - Path : " + srcPathModStr + " , recursive flg : " + recursive) - WebHdfsConnector.deleteFile(srcPathModStr, recursive, certValidation, "1000:5000", usrCred) + WebHdfsConnector. + deleteFile(srcPath.toString, recursive, certValidation, "10000:120000", usrCred) } override def mkdirs(srcPath: Path, permission: FsPermission): Boolean = { - println("In MkDirs - Path : " + srcPath) - val srcPathModStr = modifyFilePath(srcPath).toString - println("In MkDirs after mod - Path : " + srcPathModStr) - WebHdfsConnector.makeDirectory(srcPathModStr, permission.toShort, certValidation, "1000:5000", usrCred) + WebHdfsConnector. + makeDirectory(srcPath.toString, permission.toShort, certValidation, "10000:120000", usrCred) } override def append(srcPath: Path, @@ -128,24 +119,21 @@ class BahirWebHdfsFileSystem extends FileSystem { override def getFileStatus(f: Path): FileStatus = { val file = modifyFilePath(f).toString - println("In bahir get filestatus - Path , and mod Path: " + f + file + "\n") var fStatus: FileStatus = fileStatusMap.getOrElse(f.toString, null) - println("In bahir after checking filestatus map - : " + fStatus) val fileStatus = if (fStatus == null) { - println("In bahir before calling webhdfsconnector : ") - val fStatusMap = WebHdfsConnector.getFileStatus(file, certValidation, "1000:5000", usrCred) + val fStatusMap = WebHdfsConnector.getFileStatus(file, certValidation, "10000:120000", usrCred) if (fStatusMap != null) { - fStatus = createFileStatus(f, fStatusMap) - fileStatusMap.put(f.toString, fStatus) + fStatus = createFileStatus(f, fStatusMap) + fileStatusMap.put(f.toString, fStatus) } - fStatus + fStatus } else { fStatus } - println("In bahir before returning from getFileStatis fileStatus : " + fileStatus) + // println("In bahir before returning from getFileStatis fileStatus : " + fileStatus) fileStatus } @@ -155,11 +143,9 @@ class BahirWebHdfsFileSystem extends FileSystem { var lStatus: Array[FileStatus] = listStatusMap.getOrElse(f.toString, null) - // println("file in listStatus: " + file) - val listStatus = if (lStatus == null) { val fStatusMapList = WebHdfsConnector - .getListStatus(file, certValidation, "1000:5000", usrCred) + .getListStatus(file, certValidation, "10000:120000", usrCred) val fileCount = fStatusMapList.length lStatus = new Array[FileStatus](fileCount) var i = 0 @@ -174,7 +160,6 @@ class BahirWebHdfsFileSystem extends FileSystem { lStatus } - // println(" listStatus: " + listStatus) listStatus } @@ -205,9 +190,7 @@ class BahirWebHdfsFileSystem extends FileSystem { } private def modifyFilePath(f: Path): Path = { - // println("file uri : " + f.toUri) val wQryStr = f.toString.replace(getQryStrFromFilePath(f), "") - //val fStr = wQryStr.replace("_temporary", "bahir_tmp") new Path(wQryStr) } @@ -216,37 +199,64 @@ class BahirWebHdfsFileSystem extends FileSystem { val start = fileStr.indexOf("&") val end = fileStr.indexOf(";") - // print("start and end index " + start +"\n") - val qryStr = if (start > 0) fileStr.substring(start, end) else "" - // print("query : " + qryStr + "\n") qryStr } override def open(f: Path, bs: Int): FSDataInputStream = { - println("In bahir open file path - : " + f) - val fileStatus = getFileStatus(f) val blockSize = fileStatus.getBlockSize val fileLength = fileStatus.getLen val file = modifyFilePath(f) - print("file uri in open after modification : " + file + "\n") + // print("file uri in open after modification : " + file + "\n") val qMap = getQryMapFromFilePath(f) val fConnections = if (qMap == null) { - connections + 0 + } + else { + qMap.getOrElse("connections", "0").asInstanceOf[String].toInt + } + + val streamBufferSize = if (qMap == null) { + bs + } + else { + qMap.getOrElse("streamBufferSize", bs.toString).asInstanceOf[String].toInt + } + + val rdBufferSize = if (qMap == null) { + bs + } + else { + qMap.getOrElse("readBufferSize", bs.toString).asInstanceOf[String].toInt + } + + val readBufferSize = if (rdBufferSize <= 0) blockSize else rdBufferSize.toLong + + val fReadFull = if (qMap == null) { + true + } + else { + qMap.getOrElse("readFullFile", true.toString).asInstanceOf[String].toBoolean + } + + val streamFlg = if (qMap == null) { + true } else { - qMap.getOrElse("connections", connections).asInstanceOf[String].toInt + qMap.getOrElse("streamFlg", true.toString).asInstanceOf[String].toBoolean } - new FSDataInputStream(new BahirWebHdfsInputStream(file, bs, blockSize, fileLength, - readFullFile, usrCred, fConnections, certValidation)) + + new FSDataInputStream(new BahirWebHdfsInputStream(file, streamBufferSize, readBufferSize, + blockSize, fileLength, + fReadFull, streamFlg, usrCred, fConnections, certValidation)) } override def create(srcPath: Path, @@ -259,7 +269,7 @@ class BahirWebHdfsFileSystem extends FileSystem { val file = modifyFilePath(srcPath) - print("file uri in create after modification : " + file + "\n") + // print("file uri in create after modification : " + file + "\n") new FSDataOutputStream(new BahirWebHdfsOutputStream(file, bufferSize, blockSize, permission.toShort, replication, overwriteFlg, usrCred, certValidation), null) @@ -295,27 +305,31 @@ class BahirWebHdfsFileSystem extends FileSystem { } class BahirWebHdfsInputStream(fPath: Path, - bufferSz: Int, + strmBufferSz: Int, + rdBufferSz: Long, blockSz: Long, fileSz: Long, readFull: Boolean, + strmFlg: Boolean, usrCrd: String, conns: Int, certValidation: String) extends FSInputStream { val filePath: Path = fPath - val bufferSize: Int = bufferSz + val streamBufferSize: Int = strmBufferSz + val readBufferSize: Long = rdBufferSz val blockSize: Long = blockSz val fileSize: Long = fileSz val readFullFlg: Boolean = readFull + val streamFlg: Boolean = strmFlg val usrCred: String = usrCrd val connections: Int = conns val certValidationFlg: String = certValidation - var pos = 0L + var pos = -1L - var in: ByteArrayInputStream = null + var in: InputStream = null var callCount = 0 @@ -328,24 +342,15 @@ class BahirWebHdfsInputStream(fPath: Path, } override def read(b: Array[Byte], offset: Int, length: Int): Int = { -// if (in == null) createWebHdfsInputStream(pos) callCount += 1 var bCount = in.read(b, offset, length) -// println("In read - call count: " + callCount + " , pos: " + pos + ", offset: " + offset + -// ", length: " + length + ", byte count total: " + bCount) - if (bCount < 0 && pos < fileSize) { -// println("In read - bCount less than 0, call count: " + callCount + ", file size : " + -// fileSize + " , pos : " + pos + ", offset : " + offset + " , length : " + length + -// " , byte count total : " + bCount) - // createWebHdfsInputStream(pos) seek(pos) bCount = in.read(b, offset, length) } pos += bCount - bCount } @@ -354,7 +359,7 @@ class BahirWebHdfsInputStream(fPath: Path, // print("In seek - newpos : " + newPos + " , old pos : " + pos + "\n") if (pos != newPos) { pos = newPos - if (in != null) in.close + close } createWebHdfsInputStream(pos) } @@ -363,21 +368,28 @@ class BahirWebHdfsInputStream(fPath: Path, val poe = if (connections == 0) { if (blockSize > fileSize || readFullFlg == true) { - fileSize + 0 } else { - (floor(pos / blockSize).toLong + 1) * blockSize + 10000 + pos + blockSize } } else { - floor(fileSize / (connections - 1)).toInt + 10000 + pos + fileSize/connections + 1000000 } + if (streamFlg == true) { + val inputStream = WebHdfsConnector + .getFileInputStream(filePath.toString(), streamFlg, readBufferSize, pos, poe, + certValidationFlg, "10000:120000", usrCred) + in = if (streamBufferSize <= 0) inputStream else { + new BufferedInputStream(inputStream, streamBufferSize) + } + } else { + val in = WebHdfsConnector + .getFileInputStream(filePath.toString(), streamFlg, readBufferSize, pos, poe, + certValidationFlg, "10000:120000", usrCred) + } - // println("In read - input stream null , block size : " + blockSize + " , file size : " + - // fileSize + " , red full flg : " + readFullFlg + " , pos : " + pos + " , poe : " + poe +"\n") - - in = WebHdfsConnector - .getFileInputStream(filePath.toString(), pos, poe, certValidationFlg, "1000:50000", usrCred) } /* @@ -388,6 +400,10 @@ class BahirWebHdfsInputStream(fPath: Path, override def getPos(): Long = pos + override def close() : Unit = { + if (in != null) in.close + } + } @@ -413,25 +429,21 @@ class BahirWebHdfsOutputStream(fPath: Path, override def write(b: Int): Unit = { - println("in write single byte: " + b) val singleByte : Array[Byte] = new Array(b)(1) writeBytes(singleByte) } override def write(b: Array[Byte]): Unit = { - println("in write bytes ") writeBytes(b) } override def write(b: Array[Byte], offset: Int, length: Int): Unit = { - println("in write bytes with offset and length : " + offset + " , " + length) writeBytes(b) } private def writeBytes(b: Array[Byte]): Unit = { - println("in provate write bytes ") WebHdfsConnector.writeFile(b, filePath.toString, permission, overwriteFlg, bufferSize, - replication, blockSize, certValidation, "1000:5000", usrCred) + replication, blockSize, certValidation, "10000:120000", usrCred) } } diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala index 38ed32de..1ae6d1bf 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala @@ -18,8 +18,9 @@ package org.apache.bahir.datasource.webhdfs.util import java.io._ -import java.security._ +import java.net.{URL, HttpURLConnection} import javax.net.ssl.{SSLContext, SSLSocketFactory, TrustManagerFactory} +import java.security._ import scala.annotation.switch @@ -36,6 +37,9 @@ object WebHdfsConnector { /* * This function returns a Tuple for credential store which contains flag for validating * Certificate, the Certificate File object and Certificate File Object password + * the value of the parameter cred is "N" it ignores certificate validation, if + * it is "Y" then it downloads a certificate using openssl, else it expects a valid + * path for trust store and password for using the same */ def createTrustStoreCredForExecutors(cred: String, path: String): Tuple3[String, File, String] = { val trustStoreMap = { @@ -46,7 +50,8 @@ object WebHdfsConnector { val tsd = SSLTrustStore.getCertDetails(path) new Tuple3("Y", tsd._1, tsd._2) } else { - throw new Exception("Invalid Certificate Validation Option") + val tsd = cred.split(":") + new Tuple3("Y", new java.io.File(tsd(0)), tsd(1)) } } else { new Tuple3("", null, "") @@ -74,35 +79,6 @@ object WebHdfsConnector { sslc.getSocketFactory() } - // /** - // * This function returns the list of files in a folder with file details as RDD - // */ - // def listFromWebHdfs(sc: SparkContext, path: String, trustStoreCred: String, userCred: String, - // connProp: String): RDD[String] = { - // - // - // - // val conns = connProp.split(":") - // - // val conn = Array(conns(0).toInt, conns(1).toInt) - // - // val usrCred = userCred.split(":") - // - // val trustCred = createTrustStoreCredForExecutors(trustStoreCred, path) - // - // val fileDetails = getFilesDetails(path, trustCred, usrCred, conn) - // - // def g(v:Tuple4[String, Long, Long, Int]) = v._1.split("/").last + "," + - // v._2.toString + "," + v._3.toString + "," + v._4.toString + "\n" - // - // val fds = fileDetails.map(x => g(x)) - // - // val flRdd = sc.parallelize(fds) - // - // flRdd - // - // } - // /** * This function creates a directory in remote HDFS */ @@ -111,31 +87,20 @@ object WebHdfsConnector { trustStoreCredStr: String, connStr: String, userCredStr: String): Boolean = { - - println("In make directory - path : " + path) + /* val webHdfsChkDirOpr = "op=GETFILESTATUS" val returnChkDir = callWebHdfsAPI(path, null, "GET", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsChkDirOpr, "String").asInstanceOf[Integer] + */ - if (returnChkDir == 200) { - throw new Exception("The Directory Already Exists : " + path + "\n") - } - else { - - // val dPartitions = partitionStr.toInt - // - // val textRdd = dataToWrite.repartition(dPartitions) - - val webHdfsMakeDirOpr = s"op=MKDIRS&permission=$permission" - val returnMakeDir = callWebHdfsAPI(path, "".getBytes(), "PUT", "CODE", trustStoreCredStr, userCredStr, + val webHdfsMakeDirOpr = s"op=MKDIRS&permission=$permission" + val returnMakeDir = callWebHdfsAPI(path, "".getBytes(), "PUT", "CODE", + trustStoreCredStr, userCredStr, connStr, webHdfsMakeDirOpr, "String").asInstanceOf[Integer] - println("In makeDirectory - return code : " + returnMakeDir) - if(returnMakeDir != 200) - throw new Exception("The Directory could not be created , Src path, and code: " + path + " , " + returnMakeDir + "\n") - //false - else - true - } + if (returnMakeDir != 200){ + throw new Exception("The Directory could not be created , Src path,Code: " + + path + " , " + returnMakeDir + "\n") + } else true } @@ -148,26 +113,19 @@ object WebHdfsConnector { connStr: String, userCredStr: String): Boolean = { - println("In deleteFile - path : " + path + " , recusrsive flg : " + recursiveFlg) + /* val webHdfsChkDirOpr = "op=GETFILESTATUS" val returnChkDir = callWebHdfsAPI(path, null, "GET", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsChkDirOpr, "String").asInstanceOf[Integer] + */ - if (returnChkDir != 200) { - throw new Exception("The File/Directory Does Not Exist : " + path + "\n") - } - else { - - val webHdfsDeleteDirOpr = s"op=DELETE&recursive=$recursiveFlg" - val returnDelDir = callWebHdfsAPI(path, null, "DELETE", "CODE", trustStoreCredStr, userCredStr, + val webHdfsDeleteDirOpr = s"op=DELETE&recursive=$recursiveFlg" + val returnDelDir = callWebHdfsAPI(path, null, "DELETE", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsDeleteDirOpr, "String").asInstanceOf[Integer] - println("In deleteFile - return code : " + returnDelDir) - if(returnDelDir != 200) - throw new Exception("The File/Directory could not be renamed , Src path, Dest path and code: " + path + " , " + returnDelDir + "\n") - //false - else - true - } + if (returnDelDir != 200){ + throw new Exception("The File/Directory could not be renamed , Src path,Dest path,Code: " + + path + " , " + returnDelDir + "\n") + } else true } /** @@ -184,14 +142,34 @@ object WebHdfsConnector { connStr: String, userCredStr: String): Boolean = { - val webHdfsCreateOpr = s"op=CREATE&overwrite=$overwriteFlg&blockSize=$blockSize" + - s"&replication=$replication&bufferSize=$bufferSize&permission=$permission" - val createUrl = callWebHdfsAPI(path, "".getBytes(), "PUT", "LOCATION", trustStoreCredStr, userCredStr, - connStr, webHdfsCreateOpr, "String").asInstanceOf[String] - val createdCode = callWebHdfsAPI(createUrl, data, "PUT", "CODE", trustStoreCredStr, userCredStr, - connStr, webHdfsCreateOpr, "String").asInstanceOf[Integer] - println("In save file - return code : " + createdCode) - true + /* + val webHdfsChkFileOpr = "op=GETFILESTATUS" + val returnChkFile = callWebHdfsAPI(path, null, "GET", "CODE", trustStoreCredStr, userCredStr, + connStr, webHdfsChkFileOpr, "String").asInstanceOf[Integer] + + val retCode = if (returnChkFile == 200) { + // print("in writeFile, file exists : createdCode : " + returnChkFile + "\n") + true + } else { + */ + + val webHdfsCreateOpr = s"op=CREATE&overwrite=$overwriteFlg&blockSize=$blockSize" + + s"&replication=$replication&bufferSize=$bufferSize" + + s"&permission=$permission" + val createUrl = callWebHdfsAPI(path, "".getBytes(), "PUT", "LOCATION", + trustStoreCredStr, userCredStr, + connStr, webHdfsCreateOpr, "String").asInstanceOf[String] + val createdCode = callWebHdfsAPI(createUrl, data, "PUT", "CODE", trustStoreCredStr, userCredStr, + connStr, webHdfsCreateOpr, "String").asInstanceOf[Integer] + + // print("in writeFile, creaedCode : " + createdCode + "\n") + + true + /* + } + + retCode + */ } @@ -203,27 +181,20 @@ object WebHdfsConnector { trustStoreCredStr: String, connStr: String, userCredStr: String): Boolean = { - - println("In rename file - path : " + path + " , dest path : " + destPath + "\n") + /* val webHdfsChkFileOpr = "op=GETFILESTATUS" val returnChkFile = callWebHdfsAPI(path, null, "GET", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsChkFileOpr, "String").asInstanceOf[Integer] + */ - if (returnChkFile != 200) { - throw new Exception("The File/Directory Does Not Exist , path and code: " + path + " , " + returnChkFile + "\n") - } - else { - val webHdfsRenameOpr = s"op=RENAME&destination=$destPath" - val returnRename = callWebHdfsAPI(path, "".getBytes(), "PUT", "CODE", trustStoreCredStr, userCredStr, + val webHdfsRenameOpr = s"op=RENAME&destination=$destPath" + val returnRename = callWebHdfsAPI(path, "".getBytes(), "PUT", "CODE", + trustStoreCredStr, userCredStr, connStr, webHdfsRenameOpr, "String").asInstanceOf[Integer] - println("In Rename - return code : " + returnRename) - if(returnRename != 200) - throw new Exception("The File/Directory could not be renamed , Src path, Dest path and code: " + path + " , " + destPath + " , " + returnRename + "\n") - //false - else - true - } - + if (returnRename != 200){ + throw new Exception("The File/Directory could not be renamed , Src path,Dest path,Code: " + + path + " , " + destPath + " , " + returnRename + "\n") + } else true } @@ -238,7 +209,7 @@ object WebHdfsConnector { opr: String, outputType: String): Any = { - print("path in callWebHdfs : " + path + " , opr : " + opr + "\n") + // print("path in callWebHdfs : " + path + " , opr : " + opr + "\n") val pathComp = path.split(":") @@ -270,24 +241,10 @@ object WebHdfsConnector { trustCred._3))) } - /* - - val out = (outputType: @switch) match { - case "" => httpc.asString - case "String" => httpc.asString - case "Bytes" => httpc.asBytes - } - - val resp = (respType: @switch) match { - case "BODY" => out.body - case "CODE" => out.code - case "HEADERS" => out.headers - case "LOCATION" => out.location.mkString(" ") - } - */ - val resp = (respType : @switch) match { - case "BODY" => httpc.asBytes.body + case "BODY" => httpc.asString.body + case "BODY-BYTES" => httpc.asBytes.body + case "BODY-STREAM" => getBodyStream(httpc) case "CODE" => httpc.asString.code case "HEADERS" => httpc.asString.headers case "LOCATION" => httpc.asString.location.mkString(" ") @@ -296,28 +253,58 @@ object WebHdfsConnector { resp } + private def getBodyStream(httpReq: scalaj.http.HttpRequest) : InputStream = { + + val conn = (new URL(httpReq.urlBuilder(httpReq))).openConnection.asInstanceOf[HttpURLConnection] + + HttpOptions.method(httpReq.method)(conn) + + httpReq.headers.reverse.foreach{ case (name, value) => + conn.setRequestProperty(name, value) + } + + httpReq.options.reverse.foreach(_(conn)) + + httpReq.connectFunc(httpReq, conn) + + conn.getInputStream + + } + def getFileInputStream(filePath: String, + streamFlg: Boolean, + bufferSize: Long, offset: Long, - length: Long, + end: Long, trustStoreCredStr: String, connStr: String, - usrCredStr: String): ByteArrayInputStream = { + usrCredStr: String): InputStream = { - // print("path in getFileInputStream : " + filePath + "\n") + // print("path in getFileInputStream : " + filePath + " , bufferSize : " + bufferSize + "\n") - val fileGetOpr = if (length > 0) { - s"op=OPEN&offset=$offset&length=$length&bufferSize=$length" + var length = 0L + + val fileGetOpr = if (end > 0) { + length = end - offset + s"op=OPEN&offset=$offset&length=$length&bufferSize=$bufferSize" } else { - s"op=OPEN&offset=$offset" + s"op=OPEN&offset=$offset&bufferSize=$bufferSize" } val getUrl = callWebHdfsAPI(filePath, null, "GET", "LOCATION", trustStoreCredStr, usrCredStr, connStr, fileGetOpr, "String").asInstanceOf[String] - val content = callWebHdfsAPI(getUrl, null, "GET", "BODY", trustStoreCredStr, usrCredStr, - connStr, fileGetOpr, "Bytes").asInstanceOf[Array[Byte]] + if (streamFlg == true) { + callWebHdfsAPI(getUrl, null, "GET", "BODY-STREAM", + trustStoreCredStr, usrCredStr, + connStr, fileGetOpr, "Stream").asInstanceOf[InputStream] + } else { + val content = callWebHdfsAPI(getUrl, null, "GET", "BODY-BYTES", + trustStoreCredStr, usrCredStr, + connStr, fileGetOpr, "Stream").asInstanceOf[Array[Byte]] + new ByteArrayInputStream(content) + } - new ByteArrayInputStream(content) } def getFileStatus(filePath: String, @@ -325,34 +312,28 @@ object WebHdfsConnector { connStr: String, usrCredStr: String): Map[String, Any] = { - print("path in getFileStatus : " + filePath + "\n") val fileStatusOpr = s"op=GETFILESTATUS" val returnChk = callWebHdfsAPI(filePath, null, "GET", "CODE", trustStoreCredStr, usrCredStr, connStr, fileStatusOpr, "String").asInstanceOf[Integer] - print("after file status check in getFileStatus : " + returnChk + "\n") + // print("after file status check in getFileStatus in WebHdfsConnector : " + returnChk + "\n") if (returnChk == 200) { - print("within return code 200 in getFileStatus : " + returnChk + "\n") - val fileStatusBytes = callWebHdfsAPI(filePath, null, "GET", "BODY", trustStoreCredStr, usrCredStr, - connStr, fileStatusOpr, "String").asInstanceOf[Array[Byte]] - - val fileStatus = new String(fileStatusBytes, "UTF-8") + // print("within return code 200 in getFileStatus : " + returnChk + "\n") + val fileStatus = callWebHdfsAPI(filePath, null, "GET", "BODY", trustStoreCredStr, usrCredStr, + connStr, fileStatusOpr, "String").asInstanceOf[String] if (fileStatus.contains("RemoteException")) { - print("within remote exception in getFileStatus : " + returnChk + "\n") - null + null } else { - val responseMap = scala.util.parsing.json.JSON.parseFull(fileStatus).toList(0) - .asInstanceOf[Map[String, Map[String, Any]]] + val responseMap = scala.util.parsing.json.JSON.parseFull(fileStatus).toList(0) + .asInstanceOf[Map[String, Map[String, Any]]] - responseMap.getOrElse("FileStatus", null) + responseMap.getOrElse("FileStatus", null) } } else { - - print("file does not exist : " + filePath + "\n") null } } @@ -361,13 +342,10 @@ object WebHdfsConnector { trustStoreCredStr: String, connStr: String, usrCredStr: String): List[Map[String, Any]] = { - // print("path in getListStatus : " + filePath + "\n") val listStatusOpr = s"op=LISTSTATUS" - val listStatusBytes = callWebHdfsAPI(filePath, null, "GET", "BODY", trustStoreCredStr, usrCredStr, - connStr, listStatusOpr, "String").asInstanceOf[Array[Byte]] - - val listStatus = new String(listStatusBytes, "UTF-8") + val listStatus = callWebHdfsAPI(filePath, null, "GET", "BODY", trustStoreCredStr, usrCredStr, + connStr, listStatusOpr, "String").asInstanceOf[String] if (listStatus.contains("RemoteException")) { throw new Exception(listStatus) From 8beedf99579c35cc47ea3d65cf45828f41d9969b Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Thu, 22 Dec 2016 12:38:54 -0800 Subject: [PATCH 22/24] [BAHIR-75][WIP] - custom WebHdfsFileSystem - minor scalastyle fixes --- .../webhdfs/BahirWebHdfsDataSetWrapper.scala | 18 ++++++------- .../webhdfs/util/SSLTrustStoreUtil.scala | 2 +- .../webhdfs/util/WebHdfsConnector.scala | 26 +++++++++---------- 3 files changed, 21 insertions(+), 25 deletions(-) diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala index 01a774ff..265f6220 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala @@ -21,7 +21,6 @@ import java.io._ import java.net.URI import scala.collection.mutable.HashMap -import scala.math._ import org.apache.bahir.datasource.webhdfs.util.WebHdfsConnector import org.apache.hadoop.conf.Configuration @@ -34,11 +33,12 @@ import org.apache.hadoop.util.Progressable * This class contains functions for reading/writing data from/to remote webhdfs server in Spark * DataSource */ -// TODO: reenable scalastyle checks -// scalastyle:off println class BahirWebHdfsFileSystem extends FileSystem { - println(s" - - - BahirWebHdfsFileSystem class loaded - - - ") + // TODO: use a logger here + // scalastyle:off println + println(s" - - - ${this.getClass.getSimpleName} loaded - - - ") + // scalastyle:on println var uri: URI = null var rHdfsUri: URI = null @@ -81,7 +81,8 @@ class BahirWebHdfsFileSystem extends FileSystem { uri = URI.create(uriOrg.getScheme() + "://" + uriOrg.getAuthority()) - // println(s"BahirWebHdfsFileSystem: uri=${uri}, connections=${connections}, usercred=${usrCred}") +// println(s"BahirWebHdfsFileSystem: uri=${uri}, connections=${connections}, " + +// s"usercred=${usrCred}") } override def getWorkingDirectory(): Path = { @@ -269,7 +270,7 @@ class BahirWebHdfsFileSystem extends FileSystem { val file = modifyFilePath(srcPath) - // print("file uri in create after modification : " + file + "\n") + // println("file uri in create after modification : " + file) new FSDataOutputStream(new BahirWebHdfsOutputStream(file, bufferSize, blockSize, permission.toShort, replication, overwriteFlg, usrCred, certValidation), null) @@ -334,7 +335,7 @@ class BahirWebHdfsInputStream(fPath: Path, var callCount = 0 /* - * This is a dummy implementation as Spark does not use it. We need it here just to satisy + * This is a dummy implementation as Spark does not use it. We need it here just to satisfy * interface contract */ override def read(): Int = { @@ -447,6 +448,3 @@ class BahirWebHdfsOutputStream(fPath: Path, } } -// TODO: reenable scalastyle checks -// scalastyle:on println - diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala index 2e48f7c1..fbb7bc9e 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/SSLTrustStoreUtil.scala @@ -37,7 +37,7 @@ object SSLTrustStore { */ def getCertDetails(path: String): Tuple2[File, String] = { - print("path in ssltrust store getCertDetails : " + path + "\n") + print("path in ssltrust store getCertDetails : " + path + "\n") val pathComp = path.split("/") diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala index 1ae6d1bf..43cbb71d 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/util/WebHdfsConnector.scala @@ -18,9 +18,9 @@ package org.apache.bahir.datasource.webhdfs.util import java.io._ -import java.net.{URL, HttpURLConnection} -import javax.net.ssl.{SSLContext, SSLSocketFactory, TrustManagerFactory} +import java.net.{HttpURLConnection, URL} import java.security._ +import javax.net.ssl.{SSLContext, SSLSocketFactory, TrustManagerFactory} import scala.annotation.switch @@ -30,8 +30,6 @@ import scalaj.http.{Http, HttpOptions} * This object contains all utility functions for reading/writing data from/to remote webhdfs * server. The abstraction maintained in this layer is at the level of RDD */ -// TODO: reenable scalastyle checks -// scalastyle:off println object WebHdfsConnector { /* @@ -97,7 +95,7 @@ object WebHdfsConnector { val returnMakeDir = callWebHdfsAPI(path, "".getBytes(), "PUT", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsMakeDirOpr, "String").asInstanceOf[Integer] - if (returnMakeDir != 200){ + if (returnMakeDir != 200) { throw new Exception("The Directory could not be created , Src path,Code: " + path + " , " + returnMakeDir + "\n") } else true @@ -122,7 +120,7 @@ object WebHdfsConnector { val webHdfsDeleteDirOpr = s"op=DELETE&recursive=$recursiveFlg" val returnDelDir = callWebHdfsAPI(path, null, "DELETE", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsDeleteDirOpr, "String").asInstanceOf[Integer] - if (returnDelDir != 200){ + if (returnDelDir != 200) { throw new Exception("The File/Directory could not be renamed , Src path,Dest path,Code: " + path + " , " + returnDelDir + "\n") } else true @@ -149,7 +147,7 @@ object WebHdfsConnector { val retCode = if (returnChkFile == 200) { // print("in writeFile, file exists : createdCode : " + returnChkFile + "\n") - true + true } else { */ @@ -159,8 +157,8 @@ object WebHdfsConnector { val createUrl = callWebHdfsAPI(path, "".getBytes(), "PUT", "LOCATION", trustStoreCredStr, userCredStr, connStr, webHdfsCreateOpr, "String").asInstanceOf[String] - val createdCode = callWebHdfsAPI(createUrl, data, "PUT", "CODE", trustStoreCredStr, userCredStr, - connStr, webHdfsCreateOpr, "String").asInstanceOf[Integer] + val createdCode = callWebHdfsAPI(createUrl, data, "PUT", "CODE", trustStoreCredStr, + userCredStr, connStr, webHdfsCreateOpr, "String").asInstanceOf[Integer] // print("in writeFile, creaedCode : " + createdCode + "\n") @@ -191,10 +189,13 @@ object WebHdfsConnector { val returnRename = callWebHdfsAPI(path, "".getBytes(), "PUT", "CODE", trustStoreCredStr, userCredStr, connStr, webHdfsRenameOpr, "String").asInstanceOf[Integer] - if (returnRename != 200){ + if (returnRename != 200) { throw new Exception("The File/Directory could not be renamed , Src path,Dest path,Code: " + path + " , " + destPath + " , " + returnRename + "\n") - } else true + } + else { + true + } } @@ -357,6 +358,3 @@ object WebHdfsConnector { } } - -// TODO: reenable scalastyle checks -// scalastyle:on println From b467c5250ef51db4fec642d455740acd56a7caae Mon Sep 17 00:00:00 2001 From: Christian Kadner Date: Thu, 22 Dec 2016 15:19:51 -0800 Subject: [PATCH 23/24] [BAHIR-75][WIP] - remove unnecessary dependencies from pom.xml --- datasource-webhdfs/pom.xml | 20 -------------------- 1 file changed, 20 deletions(-) diff --git a/datasource-webhdfs/pom.xml b/datasource-webhdfs/pom.xml index 795a123c..7e2f5ae9 100755 --- a/datasource-webhdfs/pom.xml +++ b/datasource-webhdfs/pom.xml @@ -40,11 +40,6 @@ scalaj-http_${scala.binary.version} 2.3.0 - - org.apache.commons - commons-csv - 1.3 - org.apache.spark spark-tags_${scala.binary.version} @@ -54,25 +49,10 @@ spark-core_${scala.binary.version} ${spark.version} - - org.apache.spark - spark-sql_${scala.binary.version} - ${spark.version} - - - org.apache.spark - spark-hive_${scala.binary.version} - ${spark.version} - target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - ${basedir}/src/main/resources - - org.apache.maven.plugins From d3de3a725f0dfcab194a02cd3279594aa988292b Mon Sep 17 00:00:00 2001 From: Sourav Mazumder Date: Fri, 23 Dec 2016 15:09:15 -0800 Subject: [PATCH 24/24] [BAHIR-75][WIP] - minor fixes --- .../datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala index 265f6220..8ba35f94 100644 --- a/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala +++ b/datasource-webhdfs/src/main/scala/org/apache/bahir/datasource/webhdfs/BahirWebHdfsDataSetWrapper.scala @@ -241,10 +241,10 @@ class BahirWebHdfsFileSystem extends FileSystem { val readBufferSize = if (rdBufferSize <= 0) blockSize else rdBufferSize.toLong val fReadFull = if (qMap == null) { - true + false } else { - qMap.getOrElse("readFullFile", true.toString).asInstanceOf[String].toBoolean + qMap.getOrElse("readFullFile", false.toString).asInstanceOf[String].toBoolean } val streamFlg = if (qMap == null) { @@ -371,11 +371,11 @@ class BahirWebHdfsInputStream(fPath: Path, if (blockSize > fileSize || readFullFlg == true) { 0 } else { - pos + blockSize + pos + blockSize + 100000 } } else { - pos + fileSize/connections + 1000000 + pos + fileSize/(connections - 1) + 100000 } if (streamFlg == true) {