|
1 | | -/** Copyright 2022 Alibaba Group Holding Limited. |
| 1 | +/* Licensed to the Apache Software Foundation (ASF) under one or more |
| 2 | + * contributor license agreements. See the NOTICE file distributed with |
| 3 | + * this work for additional information regarding copyright ownership. |
| 4 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 5 | + * (the "License"); you may not use this file except in compliance with |
| 6 | + * the License. You may obtain a copy of the License at |
2 | 7 | * |
3 | | - * Licensed under the Apache License, Version 2.0 (the "License"); |
4 | | - * you may not use this file except in compliance with the License. |
5 | | - * You may obtain a copy of the License at |
6 | | - * |
7 | | - * http://www.apache.org/licenses/LICENSE-2.0 |
| 8 | + * http://www.apache.org/licenses/LICENSE-2.0 |
8 | 9 | * |
9 | 10 | * Unless required by applicable law or agreed to in writing, software |
10 | 11 | * distributed under the License is distributed on an "AS IS" BASIS, |
|
15 | 16 |
|
16 | 17 | package com.alibaba.graphar.datasources |
17 | 18 |
|
18 | | -import org.apache.spark.sql.connector.catalog.Table |
| 19 | +import scala.collection.JavaConverters._ |
| 20 | +import java.util |
| 21 | + |
| 22 | +import com.fasterxml.jackson.databind.ObjectMapper |
| 23 | +import org.apache.hadoop.conf.Configuration |
| 24 | +import org.apache.hadoop.fs.Path |
| 25 | + |
| 26 | +import org.apache.spark.sql.connector.catalog.{Table, TableProvider} |
19 | 27 | import org.apache.spark.sql.execution.datasources._ |
| 28 | +import org.apache.spark.sql.SparkSession |
20 | 29 | import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat |
21 | 30 | import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat |
22 | 31 | import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat |
23 | 32 | import org.apache.spark.sql.execution.datasources.v2._ |
24 | 33 | import org.apache.spark.sql.types.StructType |
25 | 34 | import org.apache.spark.sql.util.CaseInsensitiveStringMap |
| 35 | +import org.apache.spark.sql.sources.DataSourceRegister |
| 36 | +import org.apache.spark.sql.connector.expressions.Transform |
26 | 37 |
|
27 | | -/** GarDataSource is a class to provide gar files as the data source for spark. */ |
28 | | -class GarDataSource extends FileDataSourceV2 { |
| 38 | +import com.alibaba.graphar.utils.Utils |
29 | 39 |
|
| 40 | +object GarUtils |
| 41 | +/** GarDataSource is a class to provide gar files as the data source for spark. */ |
| 42 | +class GarDataSource extends TableProvider with DataSourceRegister { |
30 | 43 | /** The default fallback file format is Parquet. */ |
31 | | - override def fallbackFileFormat: Class[_ <: FileFormat] = classOf[ParquetFileFormat] |
| 44 | + def fallbackFileFormat: Class[_ <: FileFormat] = classOf[ParquetFileFormat] |
| 45 | + |
| 46 | + lazy val sparkSession = SparkSession.active |
32 | 47 |
|
33 | 48 | /** The string that represents the format name. */ |
34 | 49 | override def shortName(): String = "gar" |
35 | 50 |
|
| 51 | + protected def getPaths(map: CaseInsensitiveStringMap): Seq[String] = { |
| 52 | + val objectMapper = new ObjectMapper() |
| 53 | + val paths = Option(map.get("paths")).map { pathStr => |
| 54 | + objectMapper.readValue(pathStr, classOf[Array[String]]).toSeq |
| 55 | + }.getOrElse(Seq.empty) |
| 56 | + paths ++ Option(map.get("path")).toSeq |
| 57 | + } |
| 58 | + |
| 59 | + protected def getOptionsWithoutPaths(map: CaseInsensitiveStringMap): CaseInsensitiveStringMap = { |
| 60 | + val withoutPath = map.asCaseSensitiveMap().asScala.filterKeys { k => |
| 61 | + !k.equalsIgnoreCase("path") && !k.equalsIgnoreCase("paths") |
| 62 | + } |
| 63 | + new CaseInsensitiveStringMap(withoutPath.toMap.asJava) |
| 64 | + } |
| 65 | + |
| 66 | + protected def getTableName(map: CaseInsensitiveStringMap, paths: Seq[String]): String = { |
| 67 | + val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions( |
| 68 | + map.asCaseSensitiveMap().asScala.toMap) |
| 69 | + val name = shortName() + " " + paths.map(qualifiedPathName(_, hadoopConf)).mkString(",") |
| 70 | + Utils.redact(sparkSession.sessionState.conf.stringRedactionPattern, name) |
| 71 | + } |
| 72 | + |
| 73 | + private def qualifiedPathName(path: String, hadoopConf: Configuration): String = { |
| 74 | + val hdfsPath = new Path(path) |
| 75 | + val fs = hdfsPath.getFileSystem(hadoopConf) |
| 76 | + hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toString |
| 77 | + } |
| 78 | + |
36 | 79 | /** Provide a table from the data source. */ |
37 | | - override def getTable(options: CaseInsensitiveStringMap): Table = { |
| 80 | + def getTable(options: CaseInsensitiveStringMap): Table = { |
38 | 81 | val paths = getPaths(options) |
39 | 82 | val tableName = getTableName(options, paths) |
40 | 83 | val optionsWithoutPaths = getOptionsWithoutPaths(options) |
41 | 84 | GarTable(tableName, sparkSession, optionsWithoutPaths, paths, None, getFallbackFileFormat(options)) |
42 | 85 | } |
43 | 86 |
|
44 | 87 | /** Provide a table from the data source with specific schema. */ |
45 | | - override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { |
| 88 | + def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { |
46 | 89 | val paths = getPaths(options) |
47 | 90 | val tableName = getTableName(options, paths) |
48 | 91 | val optionsWithoutPaths = getOptionsWithoutPaths(options) |
49 | | - GarTable(tableName, sparkSession, optionsWithoutPaths, paths, Some(schema), getFallbackFileFormat(options)) |
| 92 | + GarTable(tableName, sparkSession, optionsWithoutPaths, paths, Some(schema), getFallbackFileFormat(options)) |
| 93 | + } |
| 94 | + |
| 95 | + override def supportsExternalMetadata(): Boolean = true |
| 96 | + |
| 97 | + private var t: Table = null |
| 98 | + |
| 99 | + override def inferSchema(options: CaseInsensitiveStringMap): StructType = { |
| 100 | + if (t == null) t = getTable(options) |
| 101 | + t.schema() |
| 102 | + } |
| 103 | + |
| 104 | + override def inferPartitioning(options: CaseInsensitiveStringMap): Array[Transform] = { |
| 105 | + Array.empty |
| 106 | + } |
| 107 | + |
| 108 | + override def getTable(schema: StructType, |
| 109 | + partitioning: Array[Transform], |
| 110 | + properties: util.Map[String, String]): Table = { |
| 111 | + // If the table is already loaded during schema inference, return it directly. |
| 112 | + if (t != null) { |
| 113 | + t |
| 114 | + } else { |
| 115 | + getTable(new CaseInsensitiveStringMap(properties), schema) |
| 116 | + } |
50 | 117 | } |
51 | 118 |
|
52 | 119 | // Get the actual fall back file format. |
|
0 commit comments