aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
blob: 07bc8ae148ac8bb457c946fa539593b5cfd6a34a (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
/*
* 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.spark.sql.execution.datasources

import java.util.ServiceLoader

import scala.collection.JavaConverters._
import scala.language.{existentials, implicitConversions}
import scala.util.{Failure, Success, Try}
import scala.util.control.NonFatal

import org.apache.hadoop.fs.Path

import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types.{CalendarIntervalType, StructType}
import org.apache.spark.util.Utils

/**
 * The main class responsible for representing a pluggable Data Source in Spark SQL. In addition to
 * acting as the canonical set of parameters that can describe a Data Source, this class is used to
 * resolve a description to a concrete implementation that can be used in a query plan
 * (either batch or streaming) or to write out data using an external library.
 *
 * From an end user's perspective a DataSource description can be created explicitly using
 * [[org.apache.spark.sql.DataFrameReader]] or CREATE TABLE USING DDL.  Additionally, this class is
 * used when resolving a description from a metastore to a concrete implementation.
 *
 * Many of the arguments to this class are optional, though depending on the specific API being used
 * these optional arguments might be filled in during resolution using either inference or external
 * metadata.  For example, when reading a partitioned table from a file system, partition columns
 * will be inferred from the directory layout even if they are not specified.
 *
 * @param paths A list of file system paths that hold data.  These will be globbed before and
 *              qualified. This option only works when reading from a [[FileFormat]].
 * @param userSpecifiedSchema An optional specification of the schema of the data. When present
 *                            we skip attempting to infer the schema.
 * @param partitionColumns A list of column names that the relation is partitioned by.  When this
 *                         list is empty, the relation is unpartitioned.
 * @param bucketSpec An optional specification for bucketing (hash-partitioning) of the data.
 */
case class DataSource(
    sqlContext: SQLContext,
    className: String,
    paths: Seq[String] = Nil,
    userSpecifiedSchema: Option[StructType] = None,
    partitionColumns: Seq[String] = Seq.empty,
    bucketSpec: Option[BucketSpec] = None,
    options: Map[String, String] = Map.empty) extends Logging {

  case class SourceInfo(name: String, schema: StructType)

  lazy val providingClass: Class[_] = lookupDataSource(className)
  lazy val sourceInfo = sourceSchema()

  /** A map to maintain backward compatibility in case we move data sources around. */
  private val backwardCompatibilityMap = Map(
    "org.apache.spark.sql.jdbc" -> classOf[jdbc.DefaultSource].getCanonicalName,
    "org.apache.spark.sql.jdbc.DefaultSource" -> classOf[jdbc.DefaultSource].getCanonicalName,
    "org.apache.spark.sql.json" -> classOf[json.DefaultSource].getCanonicalName,
    "org.apache.spark.sql.json.DefaultSource" -> classOf[json.DefaultSource].getCanonicalName,
    "org.apache.spark.sql.parquet" -> classOf[parquet.DefaultSource].getCanonicalName,
    "org.apache.spark.sql.parquet.DefaultSource" -> classOf[parquet.DefaultSource].getCanonicalName,
    "com.databricks.spark.csv" -> classOf[csv.DefaultSource].getCanonicalName
  )

  /** Given a provider name, look up the data source class definition. */
  private def lookupDataSource(provider0: String): Class[_] = {
    val provider = backwardCompatibilityMap.getOrElse(provider0, provider0)
    val provider2 = s"$provider.DefaultSource"
    val loader = Utils.getContextOrSparkClassLoader
    val serviceLoader = ServiceLoader.load(classOf[DataSourceRegister], loader)

    serviceLoader.asScala.filter(_.shortName().equalsIgnoreCase(provider)).toList match {
      // the provider format did not match any given registered aliases
      case Nil =>
        Try(loader.loadClass(provider)).orElse(Try(loader.loadClass(provider2))) match {
          case Success(dataSource) =>
            // Found the data source using fully qualified path
            dataSource
          case Failure(error) =>
            if (provider.startsWith("org.apache.spark.sql.hive.orc")) {
              throw new ClassNotFoundException(
                "The ORC data source must be used with Hive support enabled.", error)
            } else {
              if (provider == "avro" || provider == "com.databricks.spark.avro") {
                throw new ClassNotFoundException(
                  s"Failed to find data source: $provider. Please use Spark package " +
                  "http://spark-packages.org/package/databricks/spark-avro",
                  error)
              } else {
                throw new ClassNotFoundException(
                  s"Failed to find data source: $provider. Please find packages at " +
                  "http://spark-packages.org",
                  error)
              }
            }
        }
      case head :: Nil =>
        // there is exactly one registered alias
        head.getClass
      case sources =>
        // There are multiple registered aliases for the input
        sys.error(s"Multiple sources found for $provider " +
          s"(${sources.map(_.getClass.getName).mkString(", ")}), " +
          "please specify the fully qualified class name.")
    }
  }

  private def inferFileFormatSchema(format: FileFormat): StructType = {
    val caseInsensitiveOptions = new CaseInsensitiveMap(options)
    val allPaths = caseInsensitiveOptions.get("path")
    val globbedPaths = allPaths.toSeq.flatMap { path =>
      val hdfsPath = new Path(path)
      val fs = hdfsPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration)
      val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
      SparkHadoopUtil.get.globPathIfNecessary(qualified)
    }.toArray

    val fileCatalog: FileCatalog = new HDFSFileCatalog(sqlContext, options, globbedPaths, None)
    userSpecifiedSchema.orElse {
      format.inferSchema(
        sqlContext,
        caseInsensitiveOptions,
        fileCatalog.allFiles())
    }.getOrElse {
      throw new AnalysisException("Unable to infer schema.  It must be specified manually.")
    }
  }

  /** Returns the name and schema of the source that can be used to continually read data. */
  private def sourceSchema(): SourceInfo = {
    providingClass.newInstance() match {
      case s: StreamSourceProvider =>
        val (name, schema) = s.sourceSchema(sqlContext, userSpecifiedSchema, className, options)
        SourceInfo(name, schema)

      case format: FileFormat =>
        val caseInsensitiveOptions = new CaseInsensitiveMap(options)
        val path = caseInsensitiveOptions.getOrElse("path", {
          throw new IllegalArgumentException("'path' is not specified")
        })
        SourceInfo(s"FileSource[$path]", inferFileFormatSchema(format))

      case _ =>
        throw new UnsupportedOperationException(
          s"Data source $className does not support streamed reading")
    }
  }

  /** Returns a source that can be used to continually read data. */
  def createSource(metadataPath: String): Source = {
    providingClass.newInstance() match {
      case s: StreamSourceProvider =>
        s.createSource(sqlContext, metadataPath, userSpecifiedSchema, className, options)

      case format: FileFormat =>
        val caseInsensitiveOptions = new CaseInsensitiveMap(options)
        val path = caseInsensitiveOptions.getOrElse("path", {
          throw new IllegalArgumentException("'path' is not specified")
        })

        def dataFrameBuilder(files: Array[String]): DataFrame = {
          val newOptions = options.filterKeys(_ != "path") + ("basePath" -> path)
          val newDataSource =
            DataSource(
              sqlContext,
              paths = files,
              userSpecifiedSchema = Some(sourceInfo.schema),
              className = className,
              options = new CaseInsensitiveMap(newOptions))
          Dataset.ofRows(sqlContext, LogicalRelation(newDataSource.resolveRelation()))
        }

        new FileStreamSource(
          sqlContext, metadataPath, path, sourceInfo.schema, dataFrameBuilder)
      case _ =>
        throw new UnsupportedOperationException(
          s"Data source $className does not support streamed reading")
    }
  }

  /** Returns a sink that can be used to continually write data. */
  def createSink(): Sink = {
    providingClass.newInstance() match {
      case s: StreamSinkProvider => s.createSink(sqlContext, options, partitionColumns)
      case format: FileFormat =>
        val caseInsensitiveOptions = new CaseInsensitiveMap(options)
        val path = caseInsensitiveOptions.getOrElse("path", {
          throw new IllegalArgumentException("'path' is not specified")
        })

        new FileStreamSink(sqlContext, path, format)
      case _ =>
        throw new UnsupportedOperationException(
          s"Data source $className does not support streamed writing")
    }
  }

  /**
   * Returns true if there is a single path that has a metadata log indicating which files should
   * be read.
   */
  def hasMetadata(path: Seq[String]): Boolean = {
    path match {
      case Seq(singlePath) =>
        try {
          val hdfsPath = new Path(singlePath)
          val fs = hdfsPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration)
          val metadataPath = new Path(hdfsPath, FileStreamSink.metadataDir)
          val res = fs.exists(metadataPath)
          res
        } catch {
          case NonFatal(e) =>
            logWarning(s"Error while looking for metadata directory.")
            false
        }
      case _ => false
    }
  }

  /** Create a resolved [[BaseRelation]] that can be used to read data from this [[DataSource]] */
  def resolveRelation(): BaseRelation = {
    val caseInsensitiveOptions = new CaseInsensitiveMap(options)
    val relation = (providingClass.newInstance(), userSpecifiedSchema) match {
      // TODO: Throw when too much is given.
      case (dataSource: SchemaRelationProvider, Some(schema)) =>
        dataSource.createRelation(sqlContext, caseInsensitiveOptions, schema)
      case (dataSource: RelationProvider, None) =>
        dataSource.createRelation(sqlContext, caseInsensitiveOptions)
      case (_: SchemaRelationProvider, None) =>
        throw new AnalysisException(s"A schema needs to be specified when using $className.")
      case (_: RelationProvider, Some(_)) =>
        throw new AnalysisException(s"$className does not allow user-specified schemas.")

      // We are reading from the results of a streaming query. Load files from the metadata log
      // instead of listing them using HDFS APIs.
      case (format: FileFormat, _)
          if hasMetadata(caseInsensitiveOptions.get("path").toSeq ++ paths) =>
        val basePath = new Path((caseInsensitiveOptions.get("path").toSeq ++ paths).head)
        val fileCatalog =
          new StreamFileCatalog(sqlContext, basePath)
        val dataSchema = userSpecifiedSchema.orElse {
          format.inferSchema(
            sqlContext,
            caseInsensitiveOptions,
            fileCatalog.allFiles())
        }.getOrElse {
          throw new AnalysisException(
            s"Unable to infer schema for $format at ${fileCatalog.allFiles().mkString(",")}. " +
                "It must be specified manually")
        }

        HadoopFsRelation(
          sqlContext,
          fileCatalog,
          partitionSchema = fileCatalog.partitionSpec().partitionColumns,
          dataSchema = dataSchema,
          bucketSpec = None,
          format,
          options)

      // This is a non-streaming file based datasource.
      case (format: FileFormat, _) =>
        val allPaths = caseInsensitiveOptions.get("path") ++ paths
        val globbedPaths = allPaths.flatMap { path =>
          val hdfsPath = new Path(path)
          val fs = hdfsPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration)
          val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
          val globPath = SparkHadoopUtil.get.globPathIfNecessary(qualified)

          if (globPath.isEmpty) {
            throw new AnalysisException(s"Path does not exist: $qualified")
          }
          // Sufficient to check head of the globPath seq for non-glob scenario
          if (!fs.exists(globPath.head)) {
            throw new AnalysisException(s"Path does not exist: ${globPath.head}")
          }
          globPath
        }.toArray

        // If they gave a schema, then we try and figure out the types of the partition columns
        // from that schema.
        val partitionSchema = userSpecifiedSchema.map { schema =>
          StructType(
            partitionColumns.map { c =>
              // TODO: Case sensitivity.
              schema
                  .find(_.name.toLowerCase() == c.toLowerCase())
                  .getOrElse(throw new AnalysisException(s"Invalid partition column '$c'"))
            })
        }

        val fileCatalog: FileCatalog =
          new HDFSFileCatalog(sqlContext, options, globbedPaths, partitionSchema)

        val dataSchema = userSpecifiedSchema.map { schema =>
          val equality =
            if (sqlContext.conf.caseSensitiveAnalysis) {
              org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution
            } else {
              org.apache.spark.sql.catalyst.analysis.caseInsensitiveResolution
            }

          StructType(schema.filterNot(f => partitionColumns.exists(equality(_, f.name))))
        }.orElse {
          format.inferSchema(
            sqlContext,
            caseInsensitiveOptions,
            fileCatalog.allFiles())
        }.getOrElse {
          throw new AnalysisException(
            s"Unable to infer schema for $format at ${allPaths.take(2).mkString(",")}. " +
              "It must be specified manually")
        }

        val enrichedOptions =
          format.prepareRead(sqlContext, caseInsensitiveOptions, fileCatalog.allFiles())

        HadoopFsRelation(
          sqlContext,
          fileCatalog,
          partitionSchema = fileCatalog.partitionSpec().partitionColumns,
          dataSchema = dataSchema.asNullable,
          bucketSpec = bucketSpec,
          format,
          enrichedOptions)

      case _ =>
        throw new AnalysisException(
          s"$className is not a valid Spark SQL Data Source.")
    }

    relation
  }

  /** Writes the give [[DataFrame]] out to this [[DataSource]]. */
  def write(
      mode: SaveMode,
      data: DataFrame): BaseRelation = {
    if (data.schema.map(_.dataType).exists(_.isInstanceOf[CalendarIntervalType])) {
      throw new AnalysisException("Cannot save interval data type into external storage.")
    }

    providingClass.newInstance() match {
      case dataSource: CreatableRelationProvider =>
        dataSource.createRelation(sqlContext, mode, options, data)
      case format: FileFormat =>
        // Don't glob path for the write path.  The contracts here are:
        //  1. Only one output path can be specified on the write path;
        //  2. Output path must be a legal HDFS style file system path;
        //  3. It's OK that the output path doesn't exist yet;
        val caseInsensitiveOptions = new CaseInsensitiveMap(options)
        val outputPath = {
          val path = new Path(caseInsensitiveOptions.getOrElse("path", {
            throw new IllegalArgumentException("'path' is not specified")
          }))
          val fs = path.getFileSystem(sqlContext.sparkContext.hadoopConfiguration)
          path.makeQualified(fs.getUri, fs.getWorkingDirectory)
        }

        val caseSensitive = sqlContext.conf.caseSensitiveAnalysis
        PartitioningUtils.validatePartitionColumnDataTypes(
          data.schema, partitionColumns, caseSensitive)

        // If we are appending to a table that already exists, make sure the partitioning matches
        // up.  If we fail to load the table for whatever reason, ignore the check.
        if (mode == SaveMode.Append) {
          val existingPartitionColumnSet = try {
            Some(
              resolveRelation()
                .asInstanceOf[HadoopFsRelation]
                .location
                .partitionSpec()
                .partitionColumns
                .fieldNames
                .toSet)
          } catch {
            case e: Exception =>
              None
          }

          existingPartitionColumnSet.foreach { ex =>
            if (ex.map(_.toLowerCase) != partitionColumns.map(_.toLowerCase()).toSet) {
              throw new AnalysisException(
                s"Requested partitioning does not equal existing partitioning: " +
                s"$ex != ${partitionColumns.toSet}.")
            }
          }
        }

        // For partitioned relation r, r.schema's column ordering can be different from the column
        // ordering of data.logicalPlan (partition columns are all moved after data column).  This
        // will be adjusted within InsertIntoHadoopFsRelation.
        val plan =
          InsertIntoHadoopFsRelation(
            outputPath,
            partitionColumns.map(UnresolvedAttribute.quoted),
            bucketSpec,
            format,
            () => Unit, // No existing table needs to be refreshed.
            options,
            data.logicalPlan,
            mode)
        sqlContext.executePlan(plan).toRdd

      case _ =>
        sys.error(s"${providingClass.getCanonicalName} does not allow create table as select.")
    }

    // We replace the schema with that of the DataFrame we just wrote out to avoid re-inferring it.
    copy(userSpecifiedSchema = Some(data.schema.asNullable)).resolveRelation()
  }
}