From 80b9153f2026a1d56a9cfed768d087799a9cca34 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 27 Jun 2018 19:33:05 +0800 Subject: [PATCH 01/35] RANGER-2128: Implementation of Spark SQL plugin with hive as external catalog --- plugin-spark/.gitignore | 1 + plugin-spark/pom.xml | 114 +++++ plugin-spark/scalastyle-config.xml | 321 ++++++++++++ .../authorizer/RangerSparkSQLExtension.scala | 27 + .../scala/org/apache/spark/sql/Logging.scala | 58 +++ .../sql/catalyst/optimizer/Authorizer.scala | 169 +++++++ .../optimizer/HivePrivilegeObject.scala | 76 +++ .../apache/spark/sql/hive/AuthzUtils.scala | 45 ++ .../spark/sql/hive/PrivilegesBuilder.scala | 470 ++++++++++++++++++ .../spark/sql/hive/client/AuthzImpl.scala | 93 ++++ pom.xml | 2 + src/main/assembly/plugin-spark.xml | 83 ++++ 12 files changed, 1459 insertions(+) create mode 100644 plugin-spark/.gitignore create mode 100644 plugin-spark/pom.xml create mode 100644 plugin-spark/scalastyle-config.xml create mode 100644 plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/Logging.scala create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizer.scala create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/HivePrivilegeObject.scala create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/hive/AuthzUtils.scala create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/AuthzImpl.scala create mode 100644 src/main/assembly/plugin-spark.xml diff --git a/plugin-spark/.gitignore b/plugin-spark/.gitignore new file mode 100644 index 0000000000..2f7896d1d1 --- /dev/null +++ b/plugin-spark/.gitignore @@ -0,0 +1 @@ +target/ diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml new file mode 100644 index 0000000000..7d23e4fb0a --- /dev/null +++ b/plugin-spark/pom.xml @@ -0,0 +1,114 @@ + + + + 4.0.0 + ranger-spark-sql-plugin + Spark SQL Security Plugin + jar + + 2.11.8 + 2.11 + 2.2.6 + 2.2.1 + provided + 1.2.1.spark2 + + + ranger + org.apache.ranger + 1.1.0-SNAPSHOT + .. + + + + + org.scala-lang + scala-library + ${scala.version} + provided + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${spark.version} + ${spark.scope} + + + org.apache.spark + spark-hive_${scala.binary.version} + ${spark.version} + ${spark.scope} + + + + + + + net.alchim31.maven + scala-maven-plugin + 3.2.2 + + + eclipse-add-source + + add-source + + + + scala-compile-first + + compile + + + + scala-test-compile-first + + testCompile + + + + + ${scala.version} + incremental + true + + -unchecked + -deprecation + -feature + -explaintypes + -Yno-adapted-args + + + -Xms1024m + -Xmx1024m + -XX:ReservedCodeCacheSize=512M + + + -source + ${java.version.required} + -target + ${java.version.required} + -Xlint:all,-serial,-path,-try + + + + + + diff --git a/plugin-spark/scalastyle-config.xml b/plugin-spark/scalastyle-config.xml new file mode 100644 index 0000000000..f8c378576f --- /dev/null +++ b/plugin-spark/scalastyle-config.xml @@ -0,0 +1,321 @@ + + + + + Scalastyle standard configuration + + + + + + + + + + + + + + + + + + + + + + + + true + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW + + + + + + ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY, CATCH, FINALLY, LARROW, RARROW + + + + + + + + + ^println$ + + + + + @VisibleForTesting + + + + + Runtime\.getRuntime\.addShutdownHook + + + + + mutable\.SynchronizedBuffer + + + + + Await\.ready + + + + + + JavaConversions + Instead of importing implicits in scala.collection.JavaConversions._, import + scala.collection.JavaConverters._ and use .asScala / .asJava methods + + + + org\.apache\.commons\.lang\. + Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead + of Commons Lang 2 (package org.apache.commons.lang.*) + + + + extractOpt + Use Utils.jsonOption(x).map(.extract[T]) instead of .extractOpt[T], as the latter + is slower. + + + + + java,scala,3rdParty,yaooqinn + javax?\..* + scala\..* + (?!yaooqinn).* + org\.apache\.ranger\..* + + + + + + COMMA + + + + + + \)\{ + + + + + (?m)^(\s*)/[*][*].*$(\r|)\n^\1 [*] + Use Javadoc style indentation for multiline comments + + + + case[^\n>]*=>\s*\{ + Omit braces in case clauses. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 800> + + + + + 30 + + + + + 10 + + + + + 50 + + + + + + + + + + + -1,0,1,2,3 + + + diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala new file mode 100644 index 0000000000..b5af039795 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala @@ -0,0 +1,27 @@ +/* + * 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.ranger.authorization.spark.authorizer + +import org.apache.spark.sql.SparkSessionExtensions +import org.apache.spark.sql.catalyst.optimizer.Authorizer + +class RangerSparkSQLExtension extends (SparkSessionExtensions => Unit) { + override def apply(ext: SparkSessionExtensions): Unit = { + ext.injectOptimizerRule(Authorizer) + } +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/Logging.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/Logging.scala new file mode 100644 index 0000000000..59715e2d5e --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/Logging.scala @@ -0,0 +1,58 @@ +/* + * 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 + +import org.slf4j.LoggerFactory + +trait Logging { + lazy val logger = LoggerFactory.getLogger(this.getClass) + + def trace(message: => Any): Unit = { + if (logger.isTraceEnabled) { + logger.trace(message.toString) + } + } + + def debug(message: => Any): Unit = { + if (logger.isDebugEnabled) { + logger.debug(message.toString) + } + } + + def info(message: => Any): Unit = { + if (logger.isInfoEnabled) { + logger.info(message.toString) + } + } + + def warn(message: => Any): Unit = { + logger.warn(message.toString) + } + + def warn(message: => Any, t: Throwable): Unit = { + logger.warn(message.toString, t) + } + + def error(message: => Any, t: Throwable): Unit = { + logger.error(message.toString, t) + } + + def error(message: => Any): Unit = { + logger.error(message.toString) + } +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizer.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizer.scala new file mode 100644 index 0000000000..1132fc2a91 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizer.scala @@ -0,0 +1,169 @@ +/* + * 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.catalyst.optimizer + +import org.apache.hadoop.hive.ql.plan.HiveOperation +import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveAuthzContext, HiveOperationType} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.hive.{HiveExternalCatalog, PrivilegesBuilder} +import org.apache.spark.sql.hive.client.AuthzImpl +import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand + +/** + * An Optimizer Rule to do Hive Authorization V2 for Spark SQL. + */ +case class Authorizer(spark: SparkSession) extends Rule[LogicalPlan] { + + /** + * Visit the [[LogicalPlan]] recursively to get all hive privilege objects, check the privileges + * using Hive Authorizer V2 which provide sql based authorization and can implements + * ranger-hive-plugins. + * If the user is authorized, then the original plan will be returned; otherwise, interrupted by + * some particular privilege exceptions. + * @param plan a spark LogicalPlan for verifying privileges + * @return a plan itself which has gone through the privilege check. + */ + override def apply(plan: LogicalPlan): LogicalPlan = { + val operationType: HiveOperationType = getOperationType(plan) + val authzContext: HiveAuthzContext = getHiveAuthzContext(plan) + val (in, out) = PrivilegesBuilder.build(plan) + spark.sharedState.externalCatalog match { + case _: HiveExternalCatalog => + AuthzImpl.checkPrivileges(spark, operationType, in, out, authzContext) + case _ => + } + // We just return the original plan here, so this rule will be executed only once + plan + } + + /** + * Mapping of [[LogicalPlan]] -> [[HiveOperation]] + * @param plan a spark LogicalPlan + * @return + */ + private[optimizer] def getHiveOperation(plan: LogicalPlan): HiveOperation = { + plan match { + case c: Command => c match { + case _: AlterDatabasePropertiesCommand => HiveOperation.ALTERDATABASE + case p if p.nodeName == "AlterTableAddColumnsCommand" => HiveOperation.ALTERTABLE_ADDCOLS + case _: AlterTableAddPartitionCommand => HiveOperation.ALTERTABLE_ADDPARTS + case p if p.nodeName == "AlterTableChangeColumnCommand" => + HiveOperation.ALTERTABLE_RENAMECOL + case _: AlterTableDropPartitionCommand => HiveOperation.ALTERTABLE_DROPPARTS + case _: AlterTableRecoverPartitionsCommand => HiveOperation.MSCK + case _: AlterTableRenamePartitionCommand => HiveOperation.ALTERTABLE_RENAMEPART + case a: AlterTableRenameCommand => + if (!a.isView) HiveOperation.ALTERTABLE_RENAME else HiveOperation.ALTERVIEW_RENAME + case _: AlterTableSetPropertiesCommand + | _: AlterTableUnsetPropertiesCommand => HiveOperation.ALTERTABLE_PROPERTIES + case _: AlterTableSerDePropertiesCommand => HiveOperation.ALTERTABLE_SERDEPROPERTIES + case _: AlterTableSetLocationCommand => HiveOperation.ALTERTABLE_LOCATION + case _: AlterViewAsCommand => HiveOperation.QUERY + // case _: AlterViewAsCommand => HiveOperation.ALTERVIEW_AS + + case _: AnalyzeColumnCommand => HiveOperation.QUERY + // case _: AnalyzeTableCommand => HiveOperation.ANALYZE_TABLE + // Hive treat AnalyzeTableCommand as QUERY, obey it. + case _: AnalyzeTableCommand => HiveOperation.QUERY + case p if p.nodeName == "AnalyzePartitionCommand" => HiveOperation.QUERY + + case _: CreateDatabaseCommand => HiveOperation.CREATEDATABASE + case _: CreateDataSourceTableAsSelectCommand + | _: CreateHiveTableAsSelectCommand => HiveOperation.CREATETABLE_AS_SELECT + case _: CreateFunctionCommand => HiveOperation.CREATEFUNCTION + case _: CreateTableCommand + | _: CreateDataSourceTableCommand => HiveOperation.CREATETABLE + case _: CreateTableLikeCommand => HiveOperation.CREATETABLE + case _: CreateViewCommand + | _: CacheTableCommand + | _: CreateTempViewUsing => HiveOperation.CREATEVIEW + + case p if p.nodeName == "DescribeColumnCommand" => HiveOperation.DESCTABLE + case _: DescribeDatabaseCommand => HiveOperation.DESCDATABASE + case _: DescribeFunctionCommand => HiveOperation.DESCFUNCTION + case _: DescribeTableCommand => HiveOperation.DESCTABLE + + case _: DropDatabaseCommand => HiveOperation.DROPDATABASE + // Hive don't check privileges for `drop function command`, what about a unverified user + // try to drop functions. + // We treat permanent functions as tables for verifying. + case d: DropFunctionCommand if !d.isTemp => HiveOperation.DROPTABLE + case d: DropFunctionCommand if d.isTemp => HiveOperation.DROPFUNCTION + case _: DropTableCommand => HiveOperation.DROPTABLE + + case e: ExplainCommand => getHiveOperation(e.logicalPlan) + + case _: InsertIntoDataSourceCommand => HiveOperation.QUERY + case p if p.nodeName == "InsertIntoDataSourceDirCommand" => HiveOperation.QUERY + case _: InsertIntoHadoopFsRelationCommand => HiveOperation.CREATETABLE_AS_SELECT + case p if p.nodeName == "InsertIntoHiveDirCommand" => HiveOperation.QUERY + case p if p.nodeName == "InsertIntoHiveTable" => HiveOperation.QUERY + + case _: LoadDataCommand => HiveOperation.LOAD + + case p if p.nodeName == "SaveIntoDataSourceCommand" => HiveOperation.QUERY + case s: SetCommand if s.kv.isEmpty || s.kv.get._2.isEmpty => HiveOperation.SHOWCONF + case _: SetDatabaseCommand => HiveOperation.SWITCHDATABASE + case _: ShowCreateTableCommand => HiveOperation.SHOW_CREATETABLE + case _: ShowColumnsCommand => HiveOperation.SHOWCOLUMNS + case _: ShowDatabasesCommand => HiveOperation.SHOWDATABASES + case _: ShowFunctionsCommand => HiveOperation.SHOWFUNCTIONS + case _: ShowPartitionsCommand => HiveOperation.SHOWPARTITIONS + case _: ShowTablesCommand => HiveOperation.SHOWTABLES + case _: ShowTablePropertiesCommand => HiveOperation.SHOW_TBLPROPERTIES + case s: StreamingExplainCommand => + getHiveOperation(s.queryExecution.optimizedPlan) + + case _: TruncateTableCommand => HiveOperation.TRUNCATETABLE + + case _: UncacheTableCommand => HiveOperation.DROPVIEW + + // Commands that do not need build privilege goes as explain type + case _ => + // AddFileCommand + // AddJarCommand + // ... + HiveOperation.EXPLAIN + } + case _ => HiveOperation.QUERY + } + } + + private[this] def getOperationType(logicalPlan: LogicalPlan): HiveOperationType = { + HiveOperationType.valueOf(getHiveOperation(logicalPlan).name()) + } + + /** + * Provides context information in authorization check call that can be used for + * auditing and/or authorization. + */ + private[this] def getHiveAuthzContext( + logicalPlan: LogicalPlan, + command: Option[String] = None): HiveAuthzContext = { + val authzContextBuilder = new HiveAuthzContext.Builder() + // set the sql query string, [[LogicalPlan]] contains such information in 2.2 or higher version + // so this is for evolving.. + val cmd = command.getOrElse("") + authzContextBuilder.setCommandString(cmd) + authzContextBuilder.build() + } +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/HivePrivilegeObject.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/HivePrivilegeObject.scala new file mode 100644 index 0000000000..591618b575 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/HivePrivilegeObject.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.spark.sql.catalyst.optimizer + +import java.util.{List => JList} + +import org.apache.hadoop.hive.ql.security.authorization.plugin.{HivePrivilegeObject => HBO} +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.{HivePrivilegeObjectType, HivePrivObjectActionType} + +/** + * Helper class for initializing [[HBO]] with more Constructors. + */ +private[sql] object HivePrivilegeObject { + def apply( + `type`: HivePrivilegeObjectType, + dbname: String, + objectName: String, + partKeys: JList[String], + columns: JList[String], + actionType: HivePrivObjectActionType, + commandParams: JList[String]): HBO = { + new HBO( + `type`, dbname, objectName, partKeys, columns, actionType, commandParams) + } + + def apply( + `type`: HivePrivilegeObjectType, + dbname: String, + objectName: String, + partKeys: JList[String], + columns: JList[String], + commandParams: JList[String]): HBO = { + apply( + `type`, dbname, objectName, partKeys, columns, HivePrivObjectActionType.OTHER, commandParams) + } + + def apply( + `type`: HivePrivilegeObjectType, + dbname: String, + objectName: String, + partKeys: JList[String], + columns: JList[String]): HBO = { + apply( + `type`, dbname, objectName, partKeys, columns, HivePrivObjectActionType.OTHER, null) + } + + def apply( + `type`: HivePrivilegeObjectType, + dbname: String, + objectName: String, + actionType: HivePrivObjectActionType): HBO = { + apply(`type`, dbname, objectName, null, null, actionType, null) + } + + def apply( + `type`: HivePrivilegeObjectType, + dbname: String, + objectName: String): HBO = { + apply(`type`, dbname, objectName, HivePrivObjectActionType.OTHER) + } +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/AuthzUtils.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/AuthzUtils.scala new file mode 100644 index 0000000000..3fea7851b5 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/AuthzUtils.scala @@ -0,0 +1,45 @@ +/* + * 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.hive + +import scala.util.{Failure, Success, Try} + +private[hive] object AuthzUtils { + + def getFieldVal(o: Any, name: String): Any = { + Try { + val field = o.getClass.getDeclaredField(name) + field.setAccessible(true) + field.get(o) + } match { + case Success(value) => value + case Failure(exception) => throw exception + } + } + + def setFieldVal(o: Any, name: String, value: Any): Unit = { + Try { + val field = o.getClass.getDeclaredField(name) + field.setAccessible(true) + field.set(o, value.asInstanceOf[AnyRef]) + } match { + case Failure(exception) => throw exception + case _ => + } + } +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala new file mode 100644 index 0000000000..2b52939780 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala @@ -0,0 +1,470 @@ +/* + * 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.hive + +import java.util.{ArrayList => JAList, List => JList} + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.hive.ql.security.authorization.plugin.{HivePrivilegeObject => HPO} +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.{HivePrivilegeObjectType, HivePrivObjectActionType} + +import org.apache.spark.sql.SaveMode +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.expressions.NamedExpression +import org.apache.spark.sql.catalyst.optimizer.HivePrivilegeObject +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.hive.AuthzUtils._ +import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand +import org.apache.spark.sql.types.StructField + +/** + * [[LogicalPlan]] -> list of [[HivePrivilegeObject]]s + */ +private[sql] object PrivilegesBuilder { + + /** + * Build input and output privilege objects from a Spark's [[LogicalPlan]] + * + * For [[ExplainCommand]]s, build its child. + * For [[RunnableCommand]]s, build outputs if it has an target to write, build inputs for the + * inside query if exists. + * + * For other queries, build inputs. + * + * @param plan A Spark [[LogicalPlan]] + */ + def build(plan: LogicalPlan): (JList[HPO], JList[HPO]) = { + + def doBuild(plan: LogicalPlan): (JList[HPO], JList[HPO]) = { + val inputObjs = new JAList[HPO] + val outputObjs = new JAList[HPO] + plan match { + // RunnableCommand + case cmd: Command => buildCommand(cmd, inputObjs, outputObjs) + // Queries + case _ => buildQuery(plan, inputObjs) + } + (inputObjs, outputObjs) + } + + plan match { + case e: ExplainCommand => doBuild(e.logicalPlan) + case p => doBuild(p) + } + } + + /** + * Build HivePrivilegeObjects from Spark LogicalPlan + * @param plan a Spark LogicalPlan used to generate HivePrivilegeObjects + * @param hivePrivilegeObjects input or output hive privilege object list + * @param projectionList Projection list after pruning + */ + private[this] def buildQuery( + plan: LogicalPlan, + hivePrivilegeObjects: JList[HPO], + projectionList: Seq[NamedExpression] = null): Unit = { + + /** + * Columns in Projection take priority for column level privilege checking + * @param table catalogTable of a given relation + */ + def mergeProjection(table: CatalogTable): Unit = { + if (projectionList == null) { + addTableOrViewLevelObjs( + table.identifier, + hivePrivilegeObjects, + table.partitionColumnNames, + table.schema.fieldNames) + } else if (projectionList.isEmpty) { + addTableOrViewLevelObjs(table.identifier, hivePrivilegeObjects) + } else { + addTableOrViewLevelObjs( + table.identifier, + hivePrivilegeObjects, + table.partitionColumnNames.filter(projectionList.map(_.name).contains(_)), + projectionList.map(_.name)) + } + } + + plan match { + case p: Project => buildQuery(p.child, hivePrivilegeObjects, p.projectList) + + case h if h.nodeName == "HiveTableRelation" => + mergeProjection(getFieldVal(h, "tableMeta").asInstanceOf[CatalogTable]) + + case m if m.nodeName == "MetastoreRelation" => + mergeProjection(getFieldVal(m, "catalogTable").asInstanceOf[CatalogTable]) + + case l: LogicalRelation if l.catalogTable.nonEmpty => mergeProjection(l.catalogTable.get) + + case u: UnresolvedRelation => + // Normally, we shouldn't meet UnresolvedRelation here in an optimized plan. + // Unfortunately, the real world is always a place where miracles happen. + // We check the privileges directly without resolving the plan and leave everything + // to spark to do. + addTableOrViewLevelObjs(u.tableIdentifier, hivePrivilegeObjects) + + case p => + for (child <- p.children) { + buildQuery(child, hivePrivilegeObjects, projectionList) + } + } + } + + /** + * Build HivePrivilegeObjects from Spark LogicalPlan + * @param plan a Spark LogicalPlan used to generate HivePrivilegeObjects + * @param inputObjs input hive privilege object list + * @param outputObjs output hive privilege object list + */ + private[this] def buildCommand( + plan: LogicalPlan, + inputObjs: JList[HPO], + outputObjs: JList[HPO]): Unit = { + plan match { + case a: AlterDatabasePropertiesCommand => addDbLevelObjs(a.databaseName, outputObjs) + + case a if a.nodeName == "AlterTableAddColumnsCommand" => + addTableOrViewLevelObjs( + getFieldVal(a, "table").asInstanceOf[TableIdentifier], + inputObjs, + columns = getFieldVal(a, "colsToAdd").asInstanceOf[Seq[StructField]].map(_.name)) + addTableOrViewLevelObjs( + getFieldVal(a, "table").asInstanceOf[TableIdentifier], + outputObjs, + columns = getFieldVal(a, "colsToAdd").asInstanceOf[Seq[StructField]].map(_.name)) + + case a: AlterTableAddPartitionCommand => + addTableOrViewLevelObjs(a.tableName, inputObjs) + addTableOrViewLevelObjs(a.tableName, outputObjs) + + case a if a.nodeName == "AlterTableChangeColumnCommand" => + addTableOrViewLevelObjs( + getFieldVal(a, "tableName").asInstanceOf[TableIdentifier], + inputObjs, + columns = Seq(getFieldVal(a, "columnName").asInstanceOf[String])) + + case a: AlterTableDropPartitionCommand => + addTableOrViewLevelObjs(a.tableName, inputObjs) + addTableOrViewLevelObjs(a.tableName, outputObjs) + + case a: AlterTableRecoverPartitionsCommand => + addTableOrViewLevelObjs(a.tableName, inputObjs) + addTableOrViewLevelObjs(a.tableName, outputObjs) + + case a: AlterTableRenameCommand if !a.isView || a.oldName.database.nonEmpty => + // rename tables / permanent views + addTableOrViewLevelObjs(a.oldName, inputObjs) + addTableOrViewLevelObjs(a.newName, outputObjs) + + case a: AlterTableRenamePartitionCommand => + addTableOrViewLevelObjs(a.tableName, inputObjs) + addTableOrViewLevelObjs(a.tableName, outputObjs) + + case a: AlterTableSerDePropertiesCommand => + addTableOrViewLevelObjs(a.tableName, inputObjs) + addTableOrViewLevelObjs(a.tableName, outputObjs) + + case a: AlterTableSetLocationCommand => + addTableOrViewLevelObjs(a.tableName, inputObjs) + addTableOrViewLevelObjs(a.tableName, outputObjs) + + case a: AlterTableSetPropertiesCommand => + addTableOrViewLevelObjs(a.tableName, inputObjs) + addTableOrViewLevelObjs(a.tableName, outputObjs) + + case a: AlterTableUnsetPropertiesCommand => + addTableOrViewLevelObjs(a.tableName, inputObjs) + addTableOrViewLevelObjs(a.tableName, outputObjs) + + case a: AlterViewAsCommand => + if (a.name.database.nonEmpty) { + // it's a permanent view + addTableOrViewLevelObjs(a.name, outputObjs) + } + buildQuery(a.query, inputObjs) + + case a: AnalyzeColumnCommand => + addTableOrViewLevelObjs( + a.tableIdent, inputObjs, columns = a.columnNames) + addTableOrViewLevelObjs( + a.tableIdent, outputObjs, columns = a.columnNames) + + case a if a.nodeName == "AnalyzePartitionCommand" => + addTableOrViewLevelObjs( + getFieldVal(a, "tableIdent").asInstanceOf[TableIdentifier], inputObjs) + addTableOrViewLevelObjs( + getFieldVal(a, "tableIdent").asInstanceOf[TableIdentifier], outputObjs) + + case a: AnalyzeTableCommand => + addTableOrViewLevelObjs(a.tableIdent, inputObjs, columns = Seq("RAW__DATA__SIZE")) + addTableOrViewLevelObjs(a.tableIdent, outputObjs) + + case c: CacheTableCommand => c.plan.foreach { + buildQuery(_, inputObjs) + } + + case c: CreateDatabaseCommand => addDbLevelObjs(c.databaseName, outputObjs) + + case c: CreateDataSourceTableAsSelectCommand => + addDbLevelObjs(c.table.identifier, outputObjs) + addTableOrViewLevelObjs(c.table.identifier, outputObjs, mode = c.mode) + buildQuery(c.query, inputObjs) + + case c: CreateDataSourceTableCommand => + addTableOrViewLevelObjs(c.table.identifier, outputObjs) + + case c: CreateFunctionCommand if !c.isTemp => + addDbLevelObjs(c.databaseName, outputObjs) + addFunctionLevelObjs(c.databaseName, c.functionName, outputObjs) + + case c: CreateHiveTableAsSelectCommand => + addDbLevelObjs(c.tableDesc.identifier, outputObjs) + addTableOrViewLevelObjs(c.tableDesc.identifier, outputObjs) + buildQuery(c.query, inputObjs) + + case c: CreateTableCommand => addTableOrViewLevelObjs(c.table.identifier, outputObjs) + + case c: CreateTableLikeCommand => + addDbLevelObjs(c.targetTable, outputObjs) + addTableOrViewLevelObjs(c.targetTable, outputObjs) + // hive don't handle source table's privileges, we should not obey that, because + // it will cause meta information leak + addDbLevelObjs(c.sourceTable, inputObjs) + addTableOrViewLevelObjs(c.sourceTable, inputObjs) + + case c: CreateViewCommand => + c.viewType match { + case PersistedView => + // PersistedView will be tied to a database + addDbLevelObjs(c.name, outputObjs) + addTableOrViewLevelObjs(c.name, outputObjs) + case _ => + } + buildQuery(c.child, inputObjs) + + case d if d.nodeName == "DescribeColumnCommand" => + addTableOrViewLevelObjs( + getFieldVal(d, "table").asInstanceOf[TableIdentifier], + inputObjs, + columns = getFieldVal(d, "colNameParts").asInstanceOf[Seq[String]]) + + case d: DescribeDatabaseCommand => + addDbLevelObjs(d.databaseName, inputObjs) + + case d: DescribeFunctionCommand => + addFunctionLevelObjs(d.functionName.database, d.functionName.funcName, inputObjs) + + case d: DescribeTableCommand => addTableOrViewLevelObjs(d.table, inputObjs) + + case d: DropDatabaseCommand => + // outputObjs are enough for privilege check, adding inputObjs for consistency with hive + // behaviour in case of some unexpected issues. + addDbLevelObjs(d.databaseName, inputObjs) + addDbLevelObjs(d.databaseName, outputObjs) + + case d: DropFunctionCommand => + addFunctionLevelObjs(d.databaseName, d.functionName, outputObjs) + + case d: DropTableCommand => addTableOrViewLevelObjs(d.tableName, outputObjs) + + case i: InsertIntoDataSourceCommand => + i.logicalRelation.catalogTable.foreach { table => + addTableOrViewLevelObjs( + table.identifier, + outputObjs) + } + buildQuery(i.query, inputObjs) + + case i if i.nodeName =="InsertIntoDataSourceDirCommand" => + buildQuery(getFieldVal(i, "query").asInstanceOf[LogicalPlan], inputObjs) + + case i: InsertIntoHadoopFsRelationCommand => + // we are able to get the override mode here, but ctas for hive table with text/orc + // format and parquet with spark.sql.hive.convertMetastoreParquet=false can success + // with privilege checking without claiming for UPDATE privilege of target table, + // which seems to be same with Hive behaviour. + // So, here we ignore the overwrite mode for such a consistency. + i.catalogTable foreach { t => + addTableOrViewLevelObjs( + t.identifier, + outputObjs, + i.partitionColumns.map(_.name), + t.schema.fieldNames) + } + buildQuery(i.query, inputObjs) + + case i if i.nodeName == "InsertIntoHiveDirCommand" => + buildQuery(getFieldVal(i, "query").asInstanceOf[LogicalPlan], inputObjs) + + case i if i.nodeName == "InsertIntoHiveTable" => + addTableOrViewLevelObjs( + getFieldVal(i, "table").asInstanceOf[CatalogTable].identifier, outputObjs) + buildQuery(getFieldVal(i, "query").asInstanceOf[LogicalPlan], inputObjs) + + case l: LoadDataCommand => addTableOrViewLevelObjs(l.table, outputObjs) + + case s if s.nodeName == "SaveIntoDataSourceCommand" => + buildQuery(getFieldVal(s, "query").asInstanceOf[LogicalPlan], outputObjs) + + case s: SetDatabaseCommand => addDbLevelObjs(s.databaseName, inputObjs) + + case s: ShowColumnsCommand => addTableOrViewLevelObjs(s.tableName, inputObjs) + + case s: ShowCreateTableCommand => addTableOrViewLevelObjs(s.table, inputObjs) + + case s: ShowFunctionsCommand => s.db.foreach(addDbLevelObjs(_, inputObjs)) + + case s: ShowPartitionsCommand => addTableOrViewLevelObjs(s.tableName, inputObjs) + + case s: ShowTablePropertiesCommand => addTableOrViewLevelObjs(s.table, inputObjs) + + case s: ShowTablesCommand => addDbLevelObjs(s.databaseName, inputObjs) + + case s: TruncateTableCommand => addTableOrViewLevelObjs(s.tableName, outputObjs) + + case _ => + // AddFileCommand + // AddJarCommand + // AnalyzeColumnCommand + // ClearCacheCommand + // CreateTempViewUsing + // ListFilesCommand + // ListJarsCommand + // RefreshTable + // RefreshTable + // ResetCommand + // SetCommand + // ShowDatabasesCommand + // StreamingExplainCommand + // UncacheTableCommand + } + } + + /** + * Add database level hive privilege objects to input or output list + * @param dbName database name as hive privilege object + * @param hivePrivilegeObjects input or output list + */ + private[this] def addDbLevelObjs( + dbName: String, + hivePrivilegeObjects: JList[HPO]): Unit = { + hivePrivilegeObjects.add( + HivePrivilegeObject(HivePrivilegeObjectType.DATABASE, dbName, dbName)) + } + + /** + * Add database level hive privilege objects to input or output list + * @param dbOption an option of database name as hive privilege object + * @param hivePrivilegeObjects input or output hive privilege object list + */ + private def addDbLevelObjs( + dbOption: Option[String], + hivePrivilegeObjects: JList[HPO]): Unit = { + dbOption match { + case Some(db) => + hivePrivilegeObjects.add( + HivePrivilegeObject(HivePrivilegeObjectType.DATABASE, db, db)) + case _ => + } + } + + /** + * Add database level hive privilege objects to input or output list + * @param tableIdentifier table identifier contains database name as hive privilege object + * @param hivePrivilegeObjects input or output hive privilege object list + */ + private def addDbLevelObjs( + tableIdentifier: TableIdentifier, + hivePrivilegeObjects: JList[HPO]): Unit = { + tableIdentifier.database match { + case Some(db) => + hivePrivilegeObjects.add( + HivePrivilegeObject(HivePrivilegeObjectType.DATABASE, db, db)) + case _ => + } + } + + /** + * Add table level hive privilege objects to input or output list + * @param tableIdentifier table identifier contains database name, and table name as hive + * privilege object + * @param hivePrivilegeObjects input or output list + * @param mode Append or overwrite + */ + private def addTableOrViewLevelObjs( + tableIdentifier: TableIdentifier, + hivePrivilegeObjects: JList[HPO], + partKeys: Seq[String] = null, + columns: Seq[String] = null, + mode: SaveMode = SaveMode.ErrorIfExists, + cmdParams: Seq[String] = null): Unit = { + tableIdentifier.database match { + case Some(db) => + val tbName = tableIdentifier.table + val hivePrivObjectActionType = getHivePrivObjActionType(mode) + hivePrivilegeObjects.add( + HivePrivilegeObject( + HivePrivilegeObjectType.TABLE_OR_VIEW, + db, + tbName, + partKeys.asJava, + columns.asJava, + hivePrivObjectActionType, + cmdParams.asJava)) + case _ => + } + } + + /** + * Add function level hive privilege objects to input or output list + * @param databaseName database name + * @param functionName function name as hive privilege object + * @param hivePrivilegeObjects input or output list + */ + private def addFunctionLevelObjs( + databaseName: Option[String], + functionName: String, + hivePrivilegeObjects: JList[HPO]): Unit = { + databaseName match { + case Some(db) => + hivePrivilegeObjects.add( + HivePrivilegeObject(HivePrivilegeObjectType.FUNCTION, db, functionName)) + case _ => + } + } + + /** + * HivePrivObjectActionType INSERT or INSERT_OVERWRITE + * + * @param mode Append or Overwrite + * @return + */ + private def getHivePrivObjActionType(mode: SaveMode): HivePrivObjectActionType = { + mode match { + case SaveMode.Append => HivePrivObjectActionType.INSERT + case SaveMode.Overwrite => HivePrivObjectActionType.INSERT_OVERWRITE + case _ => HivePrivObjectActionType.OTHER + } + } +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/AuthzImpl.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/AuthzImpl.scala new file mode 100644 index 0000000000..2981cbec1d --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/AuthzImpl.scala @@ -0,0 +1,93 @@ +/* + * 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.hive.client + +import java.util.{List => JList} + +import org.apache.hadoop.hive.ql.security.authorization.plugin._ +import org.apache.hadoop.hive.ql.session.SessionState + +import org.apache.spark.sql.{Logging, SparkSession} +import org.apache.spark.sql.hive.{AuthzUtils, HiveExternalCatalog} + +/** + * A Tool for Authorizer implementation. + * + * The [[SessionState]] generates the authorizer and authenticator, we use these to check + * the privileges of a Spark LogicalPlan, which is mapped to hive privilege objects and operation + * type. + * + * [[SparkSession]] with hive catalog implemented has its own instance of [[SessionState]]. I am + * strongly willing to reuse it, but for the reason that it belongs to an isolated classloader + * which makes it unreachable for us to visit it in Spark's context classloader. So, when + * [[ClassCastException]] occurs, we turn off [[IsolatedClientLoader]] to use Spark's builtin + * Hive client jars to generate a new metastore client to replace the original one, once it is + * generated, will be reused then. + * + */ +object AuthzImpl extends Logging { + def checkPrivileges( + spark: SparkSession, + hiveOpType: HiveOperationType, + inputObjs: JList[HivePrivilegeObject], + outputObjs: JList[HivePrivilegeObject], + context: HiveAuthzContext): Unit = { + val client = spark.sharedState + .externalCatalog.asInstanceOf[HiveExternalCatalog] + .client + val clientImpl = try { + client.asInstanceOf[HiveClientImpl] + } catch { + case _: ClassCastException => + val clientLoader = + AuthzUtils.getFieldVal(client, "clientLoader").asInstanceOf[IsolatedClientLoader] + AuthzUtils.setFieldVal(clientLoader, "isolationOn", false) + clientLoader.cachedHive = null + val newClient = clientLoader.createClient() + AuthzUtils.setFieldVal( + spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog], + "client", + newClient) + newClient.asInstanceOf[HiveClientImpl] + } + + val authz = clientImpl.state.getAuthorizerV2 + if (authz != null) { + try { + authz.checkPrivileges(hiveOpType, inputObjs, outputObjs, context) + } catch { + case hae: HiveAccessControlException => + error( + s""" + |+===============================+ + ||Spark SQL Authorization Failure| + ||-------------------------------| + ||${hae.getMessage} + ||-------------------------------| + ||Spark SQL Authorization Failure| + |+===============================+ + """.stripMargin) + throw hae + case e: Exception => throw e + } + } else { + warn("Authorizer V2 not configured. Skip privilege checking") + } + } +} + diff --git a/pom.xml b/pom.xml index 2a2540a0d5..3dcee01484 100644 --- a/pom.xml +++ b/pom.xml @@ -224,6 +224,7 @@ ranger-kylin-plugin-shim plugin-elasticsearch ranger-elasticsearch-plugin-shim + plugin-spark @@ -251,6 +252,7 @@ src/main/assembly/plugin-sqoop.xml src/main/assembly/plugin-kylin.xml src/main/assembly/plugin-elasticsearch.xml + src/main/assembly/plugin-spark.xml diff --git a/src/main/assembly/plugin-spark.xml b/src/main/assembly/plugin-spark.xml new file mode 100644 index 0000000000..57d4808f8d --- /dev/null +++ b/src/main/assembly/plugin-spark.xml @@ -0,0 +1,83 @@ + + + + spark-plugin + + tar.gz + zip + + ${project.name}-${project.version}-spark-plugin + true + + + + false + false + 755 + 644 + /jars + + + org.apache.ranger:ranger-hive-plugin-shim + org.apache.ranger:ranger-plugin-classloader + org.apache.ranger:ranger-spark-sql-plugin + + + + + false + false + 755 + 644 + + + /jars/ranger-hive-plugin-impl + + org.eclipse.persistence:eclipselink + org.eclipse.persistence:javax.persistence + org.apache.httpcomponents:httpmime:jar:${httpcomponents.httpmime.version} + org.apache.httpcomponents:httpclient:jar:${httpcomponents.httpclient.version} + org.apache.httpcomponents:httpcore:jar:${httpcomponents.httpcore.version} + org.noggit:noggit:jar:${noggit.version} + org.apache.solr:solr-solrj + + false + + + /jars/ranger-hive-plugin-impl + + + org.apache.ranger:ranger-plugins-audit + org.apache.ranger:ranger-plugins-cred + org.apache.ranger:ranger-plugins-common + org.apache.ranger:ranger-hive-plugin + + + + + + + / + ${project.build.outputDirectory} + + version + + 444 + + + From 8cde24db320b2577cb5d7aec733761e0c5cd0f7d Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 29 Jun 2018 20:07:45 +0800 Subject: [PATCH 02/35] ranger-hive-plugin 0.6.3 --- plugin-spark/pom.xml | 165 ++++++++++++++++++++++++++++- pom.xml | 1 + src/main/assembly/plugin-spark.xml | 45 ++++---- 3 files changed, 184 insertions(+), 27 deletions(-) diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml index 7d23e4fb0a..eba353c8bf 100644 --- a/plugin-spark/pom.xml +++ b/plugin-spark/pom.xml @@ -19,7 +19,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - ranger-spark-sql-plugin + ranger-spark-plugin Spark SQL Security Plugin jar @@ -36,7 +36,6 @@ 1.1.0-SNAPSHOT .. - org.scala-lang @@ -56,6 +55,168 @@ ${spark.version} ${spark.scope} + + + + org.apache.ranger + ranger-hive-plugin + ${spark.ranger.plugin.version} + + + + org.apache.ranger + ranger-hive-plugin-shim + ${spark.ranger.plugin.version} + + + * + * + + + + + org.apache.ranger + ranger-plugin-classloader + ${spark.ranger.plugin.version} + + + * + * + + + + + org.apache.ranger + ranger-plugins-common + ${spark.ranger.plugin.version} + + + * + * + + + + + org.apache.ranger + ranger-plugins-cred + ${spark.ranger.plugin.version} + + + * + * + + + + + org.apache.ranger + ranger-plugins-audit + ${spark.ranger.plugin.version} + + + * + * + + + + + org.eclipse.persistence + eclipselink + ${eclipse.jpa.version} + + + * + * + + + + + com.google.code.gson + gson + ${gson.version} + + + * + * + + + + + org.eclipse.persistence + javax.persistence + ${javax.persistence.version} + + + * + * + + + + + org.apache.httpcomponents + httpcore + 4.4.4 + + + * + * + + + + + org.apache.httpcomponents + httpmime + 4.5.2 + + + * + * + + + + + org.apache.httpcomponents + httpclient + 4.5.2 + + + * + * + + + + + com.sun.jersey + jersey-bundle + 1.17.1 + + + * + * + + + + + org.noggit + noggit + ${noggit.version} + + + * + * + + + + + org.apache.solr + solr-solrj + 5.5.1 + + + * + * + + + diff --git a/pom.xml b/pom.xml index 3dcee01484..14b53082cb 100644 --- a/pom.xml +++ b/pom.xml @@ -170,6 +170,7 @@ 7.0.91 1.7 3.4.6 + 0.6.3 diff --git a/src/main/assembly/plugin-spark.xml b/src/main/assembly/plugin-spark.xml index 57d4808f8d..0aadd8ab7c 100644 --- a/src/main/assembly/plugin-spark.xml +++ b/src/main/assembly/plugin-spark.xml @@ -26,46 +26,41 @@ - false - false - 755 - 644 /jars - - - org.apache.ranger:ranger-hive-plugin-shim - org.apache.ranger:ranger-plugin-classloader - org.apache.ranger:ranger-spark-sql-plugin - - - - - false - false + true 755 644 + false + + /jars + + org.apache.ranger:ranger-hive-plugin-shim + org.apache.ranger:ranger-plugin-classloader + + /jars/ranger-hive-plugin-impl org.eclipse.persistence:eclipselink + com.google.code.gson:gson org.eclipse.persistence:javax.persistence - org.apache.httpcomponents:httpmime:jar:${httpcomponents.httpmime.version} - org.apache.httpcomponents:httpclient:jar:${httpcomponents.httpclient.version} - org.apache.httpcomponents:httpcore:jar:${httpcomponents.httpcore.version} - org.noggit:noggit:jar:${noggit.version} + org.apache.httpcomponents:httpmime + org.apache.httpcomponents:httpclient + org.apache.httpcomponents:httpcore + org.noggit:noggit org.apache.solr:solr-solrj + com.sun.jersey:jersey-bundle + org.apache.ranger:ranger-hive-plugin + org.apache.ranger:ranger-plugins-common + org.apache.ranger:ranger-plugins-cred + org.apache.ranger:ranger-plugins-audit - false - /jars/ranger-hive-plugin-impl - org.apache.ranger:ranger-plugins-audit - org.apache.ranger:ranger-plugins-cred - org.apache.ranger:ranger-plugins-common - org.apache.ranger:ranger-hive-plugin + org.apache.ranger:ranger-spark-plugin From ec2e815e36a508288f34662074cfa3975c7f6adf Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 17 Jan 2019 11:10:03 +0800 Subject: [PATCH 03/35] fix conficts / upgrade to the newest spark authorizer --- plugin-spark/pom.xml | 2 +- .../authorizer/RangerSparkSQLExtension.scala | 6 +- .../spark/authorizer/authorizer.scala | 26 +++ .../scala/org/apache/spark/sql/Logging.scala | 58 ------ .../sql/catalyst/optimizer/Authorizable.scala | 193 ++++++++++++++++++ .../sql/catalyst/optimizer/Authorizer.scala | 150 +------------- .../optimizer/AuthorizerExtension.scala | 29 +++ .../optimizer/HivePrivilegeObject.scala | 22 +- .../apache/spark/sql/hive/AuthzUtils.scala | 22 +- .../spark/sql/hive/PrivilegesBuilder.scala | 18 +- .../spark/sql/hive/client/AuthzImpl.scala | 62 ++++-- 11 files changed, 329 insertions(+), 259 deletions(-) create mode 100644 plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/authorizer.scala delete mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/Logging.scala create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/AuthorizerExtension.scala diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml index eba353c8bf..33c9fa275c 100644 --- a/plugin-spark/pom.xml +++ b/plugin-spark/pom.xml @@ -26,7 +26,7 @@ 2.11.8 2.11 2.2.6 - 2.2.1 + 2.2.3 provided 1.2.1.spark2 diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala index b5af039795..9354822fc7 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala @@ -18,10 +18,10 @@ package org.apache.ranger.authorization.spark.authorizer import org.apache.spark.sql.SparkSessionExtensions -import org.apache.spark.sql.catalyst.optimizer.Authorizer +import org.apache.spark.sql.catalyst.optimizer.AuthorizerExtension -class RangerSparkSQLExtension extends (SparkSessionExtensions => Unit) { +class RangerSparkSQLExtension extends Extensions { override def apply(ext: SparkSessionExtensions): Unit = { - ext.injectOptimizerRule(Authorizer) + ext.injectOptimizerRule(AuthorizerExtension) } } diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/authorizer.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/authorizer.scala new file mode 100644 index 0000000000..af21a17db6 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/authorizer.scala @@ -0,0 +1,26 @@ +/* + * 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.ranger.authorization.spark + +import org.apache.spark.sql.SparkSessionExtensions + +package object authorizer { + + type Extensions = SparkSessionExtensions => Unit + +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/Logging.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/Logging.scala deleted file mode 100644 index 59715e2d5e..0000000000 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/Logging.scala +++ /dev/null @@ -1,58 +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.spark.sql - -import org.slf4j.LoggerFactory - -trait Logging { - lazy val logger = LoggerFactory.getLogger(this.getClass) - - def trace(message: => Any): Unit = { - if (logger.isTraceEnabled) { - logger.trace(message.toString) - } - } - - def debug(message: => Any): Unit = { - if (logger.isDebugEnabled) { - logger.debug(message.toString) - } - } - - def info(message: => Any): Unit = { - if (logger.isInfoEnabled) { - logger.info(message.toString) - } - } - - def warn(message: => Any): Unit = { - logger.warn(message.toString) - } - - def warn(message: => Any, t: Throwable): Unit = { - logger.warn(message.toString, t) - } - - def error(message: => Any, t: Throwable): Unit = { - logger.error(message.toString, t) - } - - def error(message: => Any): Unit = { - logger.error(message.toString) - } -} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala new file mode 100644 index 0000000000..7724df35fc --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala @@ -0,0 +1,193 @@ +/* + * 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.catalyst.optimizer + +import java.io.File + +import org.apache.commons.logging.LogFactory +import org.apache.hadoop.hive.ql.plan.HiveOperation +import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveAuthzContext, HiveOperationType} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.datasources.{CreateTempViewUsing, InsertIntoDataSourceCommand, InsertIntoHadoopFsRelationCommand} +import org.apache.spark.sql.hive.{HiveExternalCatalog, PrivilegesBuilder} +import org.apache.spark.sql.hive.client.AuthzImpl +import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand +import org.apache.spark.util.Utils + +trait Authorizable extends Rule[LogicalPlan] { + + private val logger = LogFactory.getLog(classOf[Authorizable]) + + def spark: SparkSession + + /** + * Visit the [[LogicalPlan]] recursively to get all hive privilege objects, check the privileges + * using Hive Authorizer V2 which provide sql based authorization and can implements + * ranger-hive-plugins. + * If the user is authorized, then the original plan will be returned; otherwise, interrupted by + * some particular privilege exceptions. + * @param plan a spark LogicalPlan for verifying privileges + * @return a plan itself which has gone through the privilege check. + */ + override def apply(plan: LogicalPlan): LogicalPlan = { + val operationType: HiveOperationType = getOperationType(plan) + val authzContext = new HiveAuthzContext.Builder().build() + val (in, out) = PrivilegesBuilder.build(plan) + spark.sharedState.externalCatalog match { + case _: HiveExternalCatalog => + AuthzImpl.checkPrivileges(spark, operationType, in, out, authzContext) + case _ => + } + // iff no exception. + // We just return the original plan here, so this rule will be executed only once. + plan + } + + def policyCacheDir: Option[String] = { + Option(spark.sparkContext.hadoopConfiguration.get("ranger.plugin.hive.policy.cache.dir")) + } + + + def createCacheDirIfNonExists(dir: String): Unit = { + val file = new File(dir) + if (!file.exists()) { + if (file.mkdirs()) { + logger.info("Creating ranger policy cache directory at " + file.getAbsolutePath) + file.deleteOnExit() + } else { + logger.warn("Unable to create ranger policy cache directory at " + file.getAbsolutePath) + } + } + } + + policyCacheDir match { + case Some(dir) => createCacheDirIfNonExists(dir) + case _ => + // load resources from ranger configuration files + Option(Utils.getContextOrSparkClassLoader.getResource("ranger-hive-security.xml")) match { + case Some(url) => + spark.sparkContext.hadoopConfiguration.addResource(url) + policyCacheDir match { + case Some(dir) => createCacheDirIfNonExists(dir) + case _ => + } + case _ => + } + } + + /** + * Mapping of [[LogicalPlan]] -> [[HiveOperation]] + * @param plan a spark LogicalPlan + * @return + */ + def getHiveOperation(plan: LogicalPlan): HiveOperation = { + plan match { + case c: Command => c match { + case _: AlterDatabasePropertiesCommand => HiveOperation.ALTERDATABASE + case p if p.nodeName == "AlterTableAddColumnsCommand" => HiveOperation.ALTERTABLE_ADDCOLS + case _: AlterTableAddPartitionCommand => HiveOperation.ALTERTABLE_ADDPARTS + case p if p.nodeName == "AlterTableChangeColumnCommand" => + HiveOperation.ALTERTABLE_RENAMECOL + case _: AlterTableDropPartitionCommand => HiveOperation.ALTERTABLE_DROPPARTS + case _: AlterTableRecoverPartitionsCommand => HiveOperation.MSCK + case _: AlterTableRenamePartitionCommand => HiveOperation.ALTERTABLE_RENAMEPART + case a: AlterTableRenameCommand => + if (!a.isView) HiveOperation.ALTERTABLE_RENAME else HiveOperation.ALTERVIEW_RENAME + case _: AlterTableSetPropertiesCommand + | _: AlterTableUnsetPropertiesCommand => HiveOperation.ALTERTABLE_PROPERTIES + case _: AlterTableSerDePropertiesCommand => HiveOperation.ALTERTABLE_SERDEPROPERTIES + case _: AlterTableSetLocationCommand => HiveOperation.ALTERTABLE_LOCATION + case _: AlterViewAsCommand => HiveOperation.QUERY + // case _: AlterViewAsCommand => HiveOperation.ALTERVIEW_AS + + case _: AnalyzeColumnCommand => HiveOperation.QUERY + // case _: AnalyzeTableCommand => HiveOperation.ANALYZE_TABLE + // Hive treat AnalyzeTableCommand as QUERY, obey it. + case _: AnalyzeTableCommand => HiveOperation.QUERY + case p if p.nodeName == "AnalyzePartitionCommand" => HiveOperation.QUERY + + case _: CreateDatabaseCommand => HiveOperation.CREATEDATABASE + case _: CreateDataSourceTableAsSelectCommand + | _: CreateHiveTableAsSelectCommand => HiveOperation.CREATETABLE_AS_SELECT + case _: CreateFunctionCommand => HiveOperation.CREATEFUNCTION + case _: CreateTableCommand + | _: CreateDataSourceTableCommand => HiveOperation.CREATETABLE + case _: CreateTableLikeCommand => HiveOperation.CREATETABLE + case _: CreateViewCommand + | _: CacheTableCommand + | _: CreateTempViewUsing => HiveOperation.CREATEVIEW + + case p if p.nodeName == "DescribeColumnCommand" => HiveOperation.DESCTABLE + case _: DescribeDatabaseCommand => HiveOperation.DESCDATABASE + case _: DescribeFunctionCommand => HiveOperation.DESCFUNCTION + case _: DescribeTableCommand => HiveOperation.DESCTABLE + + case _: DropDatabaseCommand => HiveOperation.DROPDATABASE + // Hive don't check privileges for `drop function command`, what about a unverified user + // try to drop functions. + // We treat permanent functions as tables for verifying. + case d: DropFunctionCommand if !d.isTemp => HiveOperation.DROPTABLE + case d: DropFunctionCommand if d.isTemp => HiveOperation.DROPFUNCTION + case _: DropTableCommand => HiveOperation.DROPTABLE + + case e: ExplainCommand => getHiveOperation(e.logicalPlan) + + case _: InsertIntoDataSourceCommand => HiveOperation.QUERY + case p if p.nodeName == "InsertIntoDataSourceDirCommand" => HiveOperation.QUERY + case _: InsertIntoHadoopFsRelationCommand => HiveOperation.CREATETABLE_AS_SELECT + case p if p.nodeName == "InsertIntoHiveDirCommand" => HiveOperation.QUERY + case p if p.nodeName == "InsertIntoHiveTable" => HiveOperation.QUERY + + case _: LoadDataCommand => HiveOperation.LOAD + + case p if p.nodeName == "SaveIntoDataSourceCommand" => HiveOperation.QUERY + case s: SetCommand if s.kv.isEmpty || s.kv.get._2.isEmpty => HiveOperation.SHOWCONF + case _: SetDatabaseCommand => HiveOperation.SWITCHDATABASE + case _: ShowCreateTableCommand => HiveOperation.SHOW_CREATETABLE + case _: ShowColumnsCommand => HiveOperation.SHOWCOLUMNS + case _: ShowDatabasesCommand => HiveOperation.SHOWDATABASES + case _: ShowFunctionsCommand => HiveOperation.SHOWFUNCTIONS + case _: ShowPartitionsCommand => HiveOperation.SHOWPARTITIONS + case _: ShowTablesCommand => HiveOperation.SHOWTABLES + case _: ShowTablePropertiesCommand => HiveOperation.SHOW_TBLPROPERTIES + case s: StreamingExplainCommand => + getHiveOperation(s.queryExecution.optimizedPlan) + + case _: TruncateTableCommand => HiveOperation.TRUNCATETABLE + + case _: UncacheTableCommand => HiveOperation.DROPVIEW + + // Commands that do not need build privilege goes as explain type + case _ => + // AddFileCommand + // AddJarCommand + // ... + HiveOperation.EXPLAIN + } + case _ => HiveOperation.QUERY + } + } + + def getOperationType(logicalPlan: LogicalPlan): HiveOperationType = { + HiveOperationType.valueOf(getHiveOperation(logicalPlan).name()) + } +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizer.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizer.scala index 1132fc2a91..be2c263163 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizer.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizer.scala @@ -6,7 +6,7 @@ * (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 + * 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, @@ -17,153 +17,17 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.hadoop.hive.ql.plan.HiveOperation -import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveAuthzContext, HiveOperationType} import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.hive.{HiveExternalCatalog, PrivilegesBuilder} -import org.apache.spark.sql.hive.client.AuthzImpl -import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand /** * An Optimizer Rule to do Hive Authorization V2 for Spark SQL. + * + * For Apache Spark 2.1.x */ -case class Authorizer(spark: SparkSession) extends Rule[LogicalPlan] { - - /** - * Visit the [[LogicalPlan]] recursively to get all hive privilege objects, check the privileges - * using Hive Authorizer V2 which provide sql based authorization and can implements - * ranger-hive-plugins. - * If the user is authorized, then the original plan will be returned; otherwise, interrupted by - * some particular privilege exceptions. - * @param plan a spark LogicalPlan for verifying privileges - * @return a plan itself which has gone through the privilege check. - */ - override def apply(plan: LogicalPlan): LogicalPlan = { - val operationType: HiveOperationType = getOperationType(plan) - val authzContext: HiveAuthzContext = getHiveAuthzContext(plan) - val (in, out) = PrivilegesBuilder.build(plan) - spark.sharedState.externalCatalog match { - case _: HiveExternalCatalog => - AuthzImpl.checkPrivileges(spark, operationType, in, out, authzContext) - case _ => - } - // We just return the original plan here, so this rule will be executed only once - plan - } - - /** - * Mapping of [[LogicalPlan]] -> [[HiveOperation]] - * @param plan a spark LogicalPlan - * @return - */ - private[optimizer] def getHiveOperation(plan: LogicalPlan): HiveOperation = { - plan match { - case c: Command => c match { - case _: AlterDatabasePropertiesCommand => HiveOperation.ALTERDATABASE - case p if p.nodeName == "AlterTableAddColumnsCommand" => HiveOperation.ALTERTABLE_ADDCOLS - case _: AlterTableAddPartitionCommand => HiveOperation.ALTERTABLE_ADDPARTS - case p if p.nodeName == "AlterTableChangeColumnCommand" => - HiveOperation.ALTERTABLE_RENAMECOL - case _: AlterTableDropPartitionCommand => HiveOperation.ALTERTABLE_DROPPARTS - case _: AlterTableRecoverPartitionsCommand => HiveOperation.MSCK - case _: AlterTableRenamePartitionCommand => HiveOperation.ALTERTABLE_RENAMEPART - case a: AlterTableRenameCommand => - if (!a.isView) HiveOperation.ALTERTABLE_RENAME else HiveOperation.ALTERVIEW_RENAME - case _: AlterTableSetPropertiesCommand - | _: AlterTableUnsetPropertiesCommand => HiveOperation.ALTERTABLE_PROPERTIES - case _: AlterTableSerDePropertiesCommand => HiveOperation.ALTERTABLE_SERDEPROPERTIES - case _: AlterTableSetLocationCommand => HiveOperation.ALTERTABLE_LOCATION - case _: AlterViewAsCommand => HiveOperation.QUERY - // case _: AlterViewAsCommand => HiveOperation.ALTERVIEW_AS - - case _: AnalyzeColumnCommand => HiveOperation.QUERY - // case _: AnalyzeTableCommand => HiveOperation.ANALYZE_TABLE - // Hive treat AnalyzeTableCommand as QUERY, obey it. - case _: AnalyzeTableCommand => HiveOperation.QUERY - case p if p.nodeName == "AnalyzePartitionCommand" => HiveOperation.QUERY - - case _: CreateDatabaseCommand => HiveOperation.CREATEDATABASE - case _: CreateDataSourceTableAsSelectCommand - | _: CreateHiveTableAsSelectCommand => HiveOperation.CREATETABLE_AS_SELECT - case _: CreateFunctionCommand => HiveOperation.CREATEFUNCTION - case _: CreateTableCommand - | _: CreateDataSourceTableCommand => HiveOperation.CREATETABLE - case _: CreateTableLikeCommand => HiveOperation.CREATETABLE - case _: CreateViewCommand - | _: CacheTableCommand - | _: CreateTempViewUsing => HiveOperation.CREATEVIEW - - case p if p.nodeName == "DescribeColumnCommand" => HiveOperation.DESCTABLE - case _: DescribeDatabaseCommand => HiveOperation.DESCDATABASE - case _: DescribeFunctionCommand => HiveOperation.DESCFUNCTION - case _: DescribeTableCommand => HiveOperation.DESCTABLE - - case _: DropDatabaseCommand => HiveOperation.DROPDATABASE - // Hive don't check privileges for `drop function command`, what about a unverified user - // try to drop functions. - // We treat permanent functions as tables for verifying. - case d: DropFunctionCommand if !d.isTemp => HiveOperation.DROPTABLE - case d: DropFunctionCommand if d.isTemp => HiveOperation.DROPFUNCTION - case _: DropTableCommand => HiveOperation.DROPTABLE - - case e: ExplainCommand => getHiveOperation(e.logicalPlan) - - case _: InsertIntoDataSourceCommand => HiveOperation.QUERY - case p if p.nodeName == "InsertIntoDataSourceDirCommand" => HiveOperation.QUERY - case _: InsertIntoHadoopFsRelationCommand => HiveOperation.CREATETABLE_AS_SELECT - case p if p.nodeName == "InsertIntoHiveDirCommand" => HiveOperation.QUERY - case p if p.nodeName == "InsertIntoHiveTable" => HiveOperation.QUERY - - case _: LoadDataCommand => HiveOperation.LOAD - - case p if p.nodeName == "SaveIntoDataSourceCommand" => HiveOperation.QUERY - case s: SetCommand if s.kv.isEmpty || s.kv.get._2.isEmpty => HiveOperation.SHOWCONF - case _: SetDatabaseCommand => HiveOperation.SWITCHDATABASE - case _: ShowCreateTableCommand => HiveOperation.SHOW_CREATETABLE - case _: ShowColumnsCommand => HiveOperation.SHOWCOLUMNS - case _: ShowDatabasesCommand => HiveOperation.SHOWDATABASES - case _: ShowFunctionsCommand => HiveOperation.SHOWFUNCTIONS - case _: ShowPartitionsCommand => HiveOperation.SHOWPARTITIONS - case _: ShowTablesCommand => HiveOperation.SHOWTABLES - case _: ShowTablePropertiesCommand => HiveOperation.SHOW_TBLPROPERTIES - case s: StreamingExplainCommand => - getHiveOperation(s.queryExecution.optimizedPlan) - - case _: TruncateTableCommand => HiveOperation.TRUNCATETABLE - - case _: UncacheTableCommand => HiveOperation.DROPVIEW - - // Commands that do not need build privilege goes as explain type - case _ => - // AddFileCommand - // AddJarCommand - // ... - HiveOperation.EXPLAIN - } - case _ => HiveOperation.QUERY - } - } - - private[this] def getOperationType(logicalPlan: LogicalPlan): HiveOperationType = { - HiveOperationType.valueOf(getHiveOperation(logicalPlan).name()) - } - - /** - * Provides context information in authorization check call that can be used for - * auditing and/or authorization. - */ - private[this] def getHiveAuthzContext( - logicalPlan: LogicalPlan, - command: Option[String] = None): HiveAuthzContext = { - val authzContextBuilder = new HiveAuthzContext.Builder() - // set the sql query string, [[LogicalPlan]] contains such information in 2.2 or higher version - // so this is for evolving.. - val cmd = command.getOrElse("") - authzContextBuilder.setCommandString(cmd) - authzContextBuilder.build() +object Authorizer extends Rule[LogicalPlan] with Authorizable { + override def spark: SparkSession = { + SparkSession.getActiveSession.getOrElse(SparkSession.getDefaultSession.get) } } diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/AuthorizerExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/AuthorizerExtension.scala new file mode 100644 index 0000000000..8ef8a103c6 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/AuthorizerExtension.scala @@ -0,0 +1,29 @@ +/* + * 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.catalyst.optimizer + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule + +/** + * An Optimizer Rule to do Hive Authorization V2 for Spark SQL. + * + * For Apache Spark 2.2.x and later + */ +case class AuthorizerExtension(spark: SparkSession) extends Rule[LogicalPlan] with Authorizable diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/HivePrivilegeObject.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/HivePrivilegeObject.scala index 591618b575..2873f4456e 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/HivePrivilegeObject.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/HivePrivilegeObject.scala @@ -1,18 +1,18 @@ /* * 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 + * 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 + * 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. + * 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.catalyst.optimizer diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/AuthzUtils.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/AuthzUtils.scala index 3fea7851b5..bbfe212630 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/AuthzUtils.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/AuthzUtils.scala @@ -1,18 +1,18 @@ /* * 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 + * 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 + * 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. + * 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.hive diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala index 2b52939780..8be91f1367 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala @@ -79,24 +79,22 @@ private[sql] object PrivilegesBuilder { * @param hivePrivilegeObjects input or output hive privilege object list * @param projectionList Projection list after pruning */ - private[this] def buildQuery( + private def buildQuery( plan: LogicalPlan, hivePrivilegeObjects: JList[HPO], - projectionList: Seq[NamedExpression] = null): Unit = { + projectionList: Seq[NamedExpression] = Nil): Unit = { /** * Columns in Projection take priority for column level privilege checking * @param table catalogTable of a given relation */ def mergeProjection(table: CatalogTable): Unit = { - if (projectionList == null) { + if (projectionList.isEmpty) { addTableOrViewLevelObjs( table.identifier, hivePrivilegeObjects, table.partitionColumnNames, table.schema.fieldNames) - } else if (projectionList.isEmpty) { - addTableOrViewLevelObjs(table.identifier, hivePrivilegeObjects) } else { addTableOrViewLevelObjs( table.identifier, @@ -137,7 +135,7 @@ private[sql] object PrivilegesBuilder { * @param inputObjs input hive privilege object list * @param outputObjs output hive privilege object list */ - private[this] def buildCommand( + private def buildCommand( plan: LogicalPlan, inputObjs: JList[HPO], outputObjs: JList[HPO]): Unit = { @@ -366,7 +364,7 @@ private[sql] object PrivilegesBuilder { * @param dbName database name as hive privilege object * @param hivePrivilegeObjects input or output list */ - private[this] def addDbLevelObjs( + private def addDbLevelObjs( dbName: String, hivePrivilegeObjects: JList[HPO]): Unit = { hivePrivilegeObjects.add( @@ -415,10 +413,10 @@ private[sql] object PrivilegesBuilder { private def addTableOrViewLevelObjs( tableIdentifier: TableIdentifier, hivePrivilegeObjects: JList[HPO], - partKeys: Seq[String] = null, - columns: Seq[String] = null, + partKeys: Seq[String] = Nil, + columns: Seq[String] = Nil, mode: SaveMode = SaveMode.ErrorIfExists, - cmdParams: Seq[String] = null): Unit = { + cmdParams: Seq[String] = Nil): Unit = { tableIdentifier.database match { case Some(db) => val tbName = tableIdentifier.table diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/AuthzImpl.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/AuthzImpl.scala index 2981cbec1d..c0fc3614c3 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/AuthzImpl.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/AuthzImpl.scala @@ -19,11 +19,13 @@ package org.apache.spark.sql.hive.client import java.util.{List => JList} +import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.ql.security.authorization.plugin._ import org.apache.hadoop.hive.ql.session.SessionState - -import org.apache.spark.sql.{Logging, SparkSession} +import org.apache.hadoop.security.UserGroupInformation +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.hive.{AuthzUtils, HiveExternalCatalog} +import org.apache.spark.sql.internal.NonClosableMutableURLClassLoader /** * A Tool for Authorizer implementation. @@ -40,7 +42,10 @@ import org.apache.spark.sql.hive.{AuthzUtils, HiveExternalCatalog} * generated, will be reused then. * */ -object AuthzImpl extends Logging { +object AuthzImpl { + + private val logger = LogFactory.getLog(getClass.getSimpleName.stripSuffix("$")) + def checkPrivileges( spark: SparkSession, hiveOpType: HiveOperationType, @@ -57,6 +62,8 @@ object AuthzImpl extends Logging { val clientLoader = AuthzUtils.getFieldVal(client, "clientLoader").asInstanceOf[IsolatedClientLoader] AuthzUtils.setFieldVal(clientLoader, "isolationOn", false) + AuthzUtils.setFieldVal(clientLoader, + "classLoader", new NonClosableMutableURLClassLoader(clientLoader.baseClassLoader)) clientLoader.cachedHive = null val newClient = clientLoader.createClient() AuthzUtils.setFieldVal( @@ -66,28 +73,39 @@ object AuthzImpl extends Logging { newClient.asInstanceOf[HiveClientImpl] } + val state = clientImpl.state + SessionState.setCurrentSessionState(state) + val user = UserGroupInformation.getCurrentUser.getShortUserName + if (state.getAuthenticator.getUserName != user) { + val hiveConf = state.getConf + val newState = new SessionState(hiveConf, user) + SessionState.start(newState) + AuthzUtils.setFieldVal(clientImpl, "state", newState) + } + val authz = clientImpl.state.getAuthorizerV2 - if (authz != null) { - try { - authz.checkPrivileges(hiveOpType, inputObjs, outputObjs, context) - } catch { - case hae: HiveAccessControlException => - error( - s""" - |+===============================+ - ||Spark SQL Authorization Failure| - ||-------------------------------| - ||${hae.getMessage} - ||-------------------------------| - ||Spark SQL Authorization Failure| - |+===============================+ + clientImpl.withHiveState { + if (authz != null) { + try { + authz.checkPrivileges(hiveOpType, inputObjs, outputObjs, context) + } catch { + case hae: HiveAccessControlException => + error( + s""" + |+===============================+ + ||Spark SQL Authorization Failure| + ||-------------------------------| + ||${hae.getMessage} + ||-------------------------------| + ||Spark SQL Authorization Failure| + |+===============================+ """.stripMargin) - throw hae - case e: Exception => throw e + throw hae + case e: Exception => throw e + } + } else { + logger.warn("Authorizer V2 not configured. Skipping privilege checking") } - } else { - warn("Authorizer V2 not configured. Skip privilege checking") } } } - From e973e4ed6dcb9ec0de8b12269e3b6a681a4a9ea5 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 17 Jan 2019 12:00:23 +0800 Subject: [PATCH 04/35] using hive plugin of project version --- .../authorizer/RangerHiveAccessRequest.java | 22 ++++++++++++++++++- .../hive/authorizer/RangerHiveAuthorizer.java | 16 +++++++++++++- plugin-spark/pom.xml | 14 ++++++------ src/main/assembly/plugin-spark.xml | 4 ++++ 4 files changed, 47 insertions(+), 9 deletions(-) diff --git a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAccessRequest.java b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAccessRequest.java index a8bf5435b9..cd690a52d9 100644 --- a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAccessRequest.java +++ b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAccessRequest.java @@ -19,19 +19,27 @@ package org.apache.ranger.authorization.hive.authorizer; +import java.util.ArrayList; import java.util.Date; +import java.util.List; import java.util.Set; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext; import org.apache.ranger.plugin.policyengine.RangerAccessRequestImpl; import org.apache.ranger.plugin.policyengine.RangerPolicyEngine; import org.apache.ranger.plugin.util.RangerAccessRequestUtil; public class RangerHiveAccessRequest extends RangerAccessRequestImpl { + private static final Log LOG = LogFactory.getLog(RangerHiveAccessRequest.class); + private HiveAccessType accessType = HiveAccessType.NONE; + private boolean isForwardedAddressesMethodAvailable = true; + public RangerHiveAccessRequest() { super(); } @@ -53,7 +61,7 @@ public RangerHiveAccessRequest(RangerHiveResource resource, if(context != null) { this.setRequestData(context.getCommandString()); - this.setForwardedAddresses(context.getForwardedAddresses()); + this.setForwardedAddresses(getForwardedAddresses(context)); this.setRemoteIPAddress(context.getIpAddress()); } @@ -65,6 +73,18 @@ public RangerHiveAccessRequest(RangerHiveResource resource, this.setClusterName(clusterName); } + protected List getForwardedAddresses(HiveAuthzContext context) { + if (isForwardedAddressesMethodAvailable) { + try { + return context.getForwardedAddresses(); + } catch (NoSuchMethodError ex) { + LOG.warn("Method getForwardedAddresses() not found in HiveAuthzContext class. Possibily Hive is older version. We will ignore this and not call this method again", ex); + isForwardedAddressesMethodAvailable = false; + } + } + return new ArrayList(); + } + public RangerHiveAccessRequest(RangerHiveResource resource, String user, Set userGroups, diff --git a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java index 0f3bee547d..80537e0d6c 100644 --- a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java +++ b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java @@ -88,6 +88,8 @@ public class RangerHiveAuthorizer extends RangerHiveAuthorizerBase { private static volatile RangerHivePlugin hivePlugin = null; + private boolean isForwardedAddressesMethodAvailable = true; + public RangerHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory, HiveConf hiveConf, HiveAuthenticationProvider hiveAuthenticator, @@ -1662,7 +1664,7 @@ private String toString(HiveOperationType hiveOpType, sb.append("'clientType':").append(sessionContext == null ? null : sessionContext.getClientType()); sb.append(", 'commandString':").append(context == null ? "null" : context.getCommandString()); sb.append(", 'ipAddress':").append(context == null ? "null" : context.getIpAddress()); - sb.append(", 'forwardedAddresses':").append(context == null ? "null" : StringUtils.join(context.getForwardedAddresses(), ", ")); + sb.append(", 'forwardedAddresses':").append(context == null ? "null" : StringUtils.join(getForwardedAddresses(context), ", ")); sb.append(", 'sessionString':").append(sessionContext == null ? "null" : sessionContext.getSessionString()); sb.append("}"); @@ -1674,6 +1676,18 @@ private String toString(HiveOperationType hiveOpType, return sb.toString(); } + protected List getForwardedAddresses(HiveAuthzContext context) { + if (isForwardedAddressesMethodAvailable) { + try { + return context.getForwardedAddresses(); + } catch (NoSuchMethodError ex) { + LOG.warn("Method getForwardedAddresses() not found in HiveAuthzContext class. Possibily Hive is older version. We will ignore this and not call this method again", + ex); + isForwardedAddressesMethodAvailable = false; + } + } + return new ArrayList(); + } private StringBuilder toString(List privObjs, StringBuilder sb) { if(privObjs != null && privObjs.size() > 0) { toString(privObjs.get(0), sb); diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml index 33c9fa275c..ba61df3c93 100644 --- a/plugin-spark/pom.xml +++ b/plugin-spark/pom.xml @@ -33,7 +33,7 @@ ranger org.apache.ranger - 1.1.0-SNAPSHOT + 2.0.0-SNAPSHOT .. @@ -60,13 +60,13 @@ org.apache.ranger ranger-hive-plugin - ${spark.ranger.plugin.version} + ${project.version} org.apache.ranger ranger-hive-plugin-shim - ${spark.ranger.plugin.version} + ${project.version} * @@ -77,7 +77,7 @@ org.apache.ranger ranger-plugin-classloader - ${spark.ranger.plugin.version} + ${project.version} * @@ -88,7 +88,7 @@ org.apache.ranger ranger-plugins-common - ${spark.ranger.plugin.version} + ${project.version} * @@ -99,7 +99,7 @@ org.apache.ranger ranger-plugins-cred - ${spark.ranger.plugin.version} + ${project.version} * @@ -110,7 +110,7 @@ org.apache.ranger ranger-plugins-audit - ${spark.ranger.plugin.version} + ${project.version} * diff --git a/src/main/assembly/plugin-spark.xml b/src/main/assembly/plugin-spark.xml index 0aadd8ab7c..658c806916 100644 --- a/src/main/assembly/plugin-spark.xml +++ b/src/main/assembly/plugin-spark.xml @@ -55,6 +55,10 @@ org.apache.ranger:ranger-plugins-common org.apache.ranger:ranger-plugins-cred org.apache.ranger:ranger-plugins-audit + org.codehaus.jackson:jackson-core-asl + org.codehaus.jackson:jackson-jaxrs + org.codehaus.jackson:jackson-mapper-asl + org.codehaus.jackson:jackson-xc From d31a2e3a06343f6349fc21420931eb7bc602e088 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 17 Jan 2019 15:20:16 +0800 Subject: [PATCH 05/35] typo --- .../authorization/hive/authorizer/RangerHiveAuthorizer.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java index 80537e0d6c..f6d6793000 100644 --- a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java +++ b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java @@ -1688,6 +1688,7 @@ protected List getForwardedAddresses(HiveAuthzContext context) { } return new ArrayList(); } + private StringBuilder toString(List privObjs, StringBuilder sb) { if(privObjs != null && privObjs.size() > 0) { toString(privObjs.get(0), sb); From 8429f64d9b5b92418550e3a9941276abefa69122 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 21 Jan 2019 17:00:02 +0800 Subject: [PATCH 06/35] add the other artifacts versions --- plugin-spark/pom.xml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml index ba61df3c93..8e39333463 100644 --- a/plugin-spark/pom.xml +++ b/plugin-spark/pom.xml @@ -154,7 +154,7 @@ org.apache.httpcomponents httpcore - 4.4.4 + ${httpcomponents.httpcore.version} * @@ -165,7 +165,7 @@ org.apache.httpcomponents httpmime - 4.5.2 + ${httpcomponents.httpmime.version} * @@ -176,7 +176,7 @@ org.apache.httpcomponents httpclient - 4.5.2 + ${httpcomponents.httpclient.version} * @@ -187,7 +187,7 @@ com.sun.jersey jersey-bundle - 1.17.1 + ${jersey-bundle.version} * @@ -209,7 +209,7 @@ org.apache.solr solr-solrj - 5.5.1 + ${solr.version} * From dd1b897fb51d7781500a1aba814ca237e836bbf0 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sun, 31 Mar 2019 01:38:54 +0800 Subject: [PATCH 07/35] apply row level filter --- .../authorizer/RangerHiveAccessRequest.java | 24 +- .../hive/authorizer/RangerHiveAuthorizer.java | 19 +- plugin-spark/.gitignore | 1 + plugin-spark/pom.xml | 96 ++-- plugin-spark/scalastyle-config.xml | 321 ------------- .../authorizer/RangerSparkAccessRequest.scala | 115 +++++ .../authorizer/RangerSparkAuditHandler.scala | 26 ++ .../authorizer/RangerSparkAuthorizer.scala | 429 ++++++++++++++++++ .../RangerSparkAuthorizerFactory.scala | 32 ++ .../spark/authorizer/RangerSparkPlugin.scala | 67 +++ .../authorizer/RangerSparkResource.scala | 90 ++++ .../authorizer/RangerSparkSQLExtension.scala | 22 +- .../spark/authorizer/SparkAccessType.scala | 25 + .../spark/authorizer/SparkObjectType.scala | 24 + .../spark/authorizer/authorizer.scala | 28 +- .../org/apache/spark/sql/AuthzUtils.scala | 45 ++ .../sql/catalyst/optimizer/Authorizable.scala | 14 +- .../optimizer/RangerSparkRowFilter.scala | 92 ++++ ...bject.scala => SparkPrivilegeObject.scala} | 2 +- .../apache/spark/sql/hive/AuthzUtils.scala | 45 -- .../spark/sql/hive/PrivilegesBuilder.scala | 16 +- ...zImpl.scala => RangerSparkAuthzImpl.scala} | 9 +- src/main/assembly/plugin-spark.xml | 31 -- 23 files changed, 1066 insertions(+), 507 deletions(-) delete mode 100644 plugin-spark/scalastyle-config.xml create mode 100644 plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAccessRequest.scala create mode 100644 plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuditHandler.scala create mode 100644 plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala create mode 100644 plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizerFactory.scala create mode 100644 plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkPlugin.scala create mode 100644 plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala create mode 100644 plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkAccessType.scala create mode 100644 plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkObjectType.scala create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/AuthzUtils.scala create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilter.scala rename plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/{HivePrivilegeObject.scala => SparkPrivilegeObject.scala} (98%) delete mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/hive/AuthzUtils.scala rename plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/{AuthzImpl.scala => RangerSparkAuthzImpl.scala} (96%) diff --git a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAccessRequest.java b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAccessRequest.java index cd690a52d9..81fa83411f 100644 --- a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAccessRequest.java +++ b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAccessRequest.java @@ -19,27 +19,19 @@ package org.apache.ranger.authorization.hive.authorizer; -import java.util.ArrayList; import java.util.Date; -import java.util.List; import java.util.Set; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext; import org.apache.ranger.plugin.policyengine.RangerAccessRequestImpl; import org.apache.ranger.plugin.policyengine.RangerPolicyEngine; import org.apache.ranger.plugin.util.RangerAccessRequestUtil; public class RangerHiveAccessRequest extends RangerAccessRequestImpl { - private static final Log LOG = LogFactory.getLog(RangerHiveAccessRequest.class); - private HiveAccessType accessType = HiveAccessType.NONE; - private boolean isForwardedAddressesMethodAvailable = true; - public RangerHiveAccessRequest() { super(); } @@ -61,7 +53,7 @@ public RangerHiveAccessRequest(RangerHiveResource resource, if(context != null) { this.setRequestData(context.getCommandString()); - this.setForwardedAddresses(getForwardedAddresses(context)); + this.setForwardedAddresses(context.getForwardedAddresses()); this.setRemoteIPAddress(context.getIpAddress()); } @@ -73,18 +65,6 @@ public RangerHiveAccessRequest(RangerHiveResource resource, this.setClusterName(clusterName); } - protected List getForwardedAddresses(HiveAuthzContext context) { - if (isForwardedAddressesMethodAvailable) { - try { - return context.getForwardedAddresses(); - } catch (NoSuchMethodError ex) { - LOG.warn("Method getForwardedAddresses() not found in HiveAuthzContext class. Possibily Hive is older version. We will ignore this and not call this method again", ex); - isForwardedAddressesMethodAvailable = false; - } - } - return new ArrayList(); - } - public RangerHiveAccessRequest(RangerHiveResource resource, String user, Set userGroups, diff --git a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java index f6d6793000..8aebb8b1a2 100644 --- a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java +++ b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java @@ -88,8 +88,6 @@ public class RangerHiveAuthorizer extends RangerHiveAuthorizerBase { private static volatile RangerHivePlugin hivePlugin = null; - private boolean isForwardedAddressesMethodAvailable = true; - public RangerHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory, HiveConf hiveConf, HiveAuthenticationProvider hiveAuthenticator, @@ -520,7 +518,7 @@ public List filterListCmdObjects(List if (result == null) { LOG.error("filterListCmdObjects: Internal error: null RangerAccessResult object received back from isAccessAllowed()!"); } else if (!result.getIsAllowed()) { - if (!LOG.isDebugEnabled()) { + if (LOG.isDebugEnabled()) { String path = resource.getAsString(); LOG.debug(String.format("filterListCmdObjects: Permission denied: user [%s] does not have [%s] privilege on [%s]. resource[%s], request[%s], result[%s]", user, request.getHiveAccessType().name(), path, resource, request, result)); @@ -1664,7 +1662,7 @@ private String toString(HiveOperationType hiveOpType, sb.append("'clientType':").append(sessionContext == null ? null : sessionContext.getClientType()); sb.append(", 'commandString':").append(context == null ? "null" : context.getCommandString()); sb.append(", 'ipAddress':").append(context == null ? "null" : context.getIpAddress()); - sb.append(", 'forwardedAddresses':").append(context == null ? "null" : StringUtils.join(getForwardedAddresses(context), ", ")); + sb.append(", 'forwardedAddresses':").append(context == null ? "null" : StringUtils.join(context.getForwardedAddresses(), ", ")); sb.append(", 'sessionString':").append(sessionContext == null ? "null" : sessionContext.getSessionString()); sb.append("}"); @@ -1676,19 +1674,6 @@ private String toString(HiveOperationType hiveOpType, return sb.toString(); } - protected List getForwardedAddresses(HiveAuthzContext context) { - if (isForwardedAddressesMethodAvailable) { - try { - return context.getForwardedAddresses(); - } catch (NoSuchMethodError ex) { - LOG.warn("Method getForwardedAddresses() not found in HiveAuthzContext class. Possibily Hive is older version. We will ignore this and not call this method again", - ex); - isForwardedAddressesMethodAvailable = false; - } - } - return new ArrayList(); - } - private StringBuilder toString(List privObjs, StringBuilder sb) { if(privObjs != null && privObjs.size() > 0) { toString(privObjs.get(0), sb); diff --git a/plugin-spark/.gitignore b/plugin-spark/.gitignore index 2f7896d1d1..3db6e9adbf 100644 --- a/plugin-spark/.gitignore +++ b/plugin-spark/.gitignore @@ -1 +1,2 @@ target/ +dependency-reduced-pom.xml \ No newline at end of file diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml index 8e39333463..c66a4d88ac 100644 --- a/plugin-spark/pom.xml +++ b/plugin-spark/pom.xml @@ -29,6 +29,7 @@ 2.2.3 provided 1.2.1.spark2 + ranger_spark_project ranger @@ -56,35 +57,13 @@ ${spark.scope} - - - org.apache.ranger - ranger-hive-plugin - ${project.version} - + + + + + + - - org.apache.ranger - ranger-hive-plugin-shim - ${project.version} - - - * - * - - - - - org.apache.ranger - ranger-plugin-classloader - ${project.version} - - - * - * - - - org.apache.ranger ranger-plugins-common @@ -217,6 +196,25 @@ + + + + org.scalatest + scalatest_${scala.binary.version} + 3.0.3 + + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + test-jar + + + + org.mockito + mockito-core + @@ -270,6 +268,48 @@ + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + com.google.code.gson:gson + com.sun.jersey:jersey-bundle + org.apache.httpcomponents:httpclient + org.apache.httpcomponents:httpcore + org.apache.httpcomponents:httpmime + org.apache.ranger:ranger-plugins-common + org.apache.ranger:ranger-plugins-cred + org.apache.ranger:ranger-plugins-audit + org.apache.solr:solr-solrj + org.codehaus.jackson:jackson-core-asl + org.codehaus.jackson:jackson-jaxrs + org.codehaus.jackson:jackson-mapper-asl + org.codehaus.jackson:jackson-xc + org.eclipse.persistence:eclipselink + org.eclipse.persistence:javax.persistence + org.noggit:noggit + + + + + com.sun.jersey + ${ranger.spark.package}.jersey + + + + + + package + + shade + + + + diff --git a/plugin-spark/scalastyle-config.xml b/plugin-spark/scalastyle-config.xml deleted file mode 100644 index f8c378576f..0000000000 --- a/plugin-spark/scalastyle-config.xml +++ /dev/null @@ -1,321 +0,0 @@ - - - - - Scalastyle standard configuration - - - - - - - - - - - - - - - - - - - - - - - - true - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW - - - - - - ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY, CATCH, FINALLY, LARROW, RARROW - - - - - - - - - ^println$ - - - - - @VisibleForTesting - - - - - Runtime\.getRuntime\.addShutdownHook - - - - - mutable\.SynchronizedBuffer - - - - - Await\.ready - - - - - - JavaConversions - Instead of importing implicits in scala.collection.JavaConversions._, import - scala.collection.JavaConverters._ and use .asScala / .asJava methods - - - - org\.apache\.commons\.lang\. - Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead - of Commons Lang 2 (package org.apache.commons.lang.*) - - - - extractOpt - Use Utils.jsonOption(x).map(.extract[T]) instead of .extractOpt[T], as the latter - is slower. - - - - - java,scala,3rdParty,yaooqinn - javax?\..* - scala\..* - (?!yaooqinn).* - org\.apache\.ranger\..* - - - - - - COMMA - - - - - - \)\{ - - - - - (?m)^(\s*)/[*][*].*$(\r|)\n^\1 [*] - Use Javadoc style indentation for multiline comments - - - - case[^\n>]*=>\s*\{ - Omit braces in case clauses. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - 800> - - - - - 30 - - - - - 10 - - - - - 50 - - - - - - - - - - - -1,0,1,2,3 - - - diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAccessRequest.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAccessRequest.scala new file mode 100644 index 0000000000..df4a93b97e --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAccessRequest.scala @@ -0,0 +1,115 @@ +/* + * 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.ranger.authorization.spark.authorizer + +import java.util.Date + +import java.util.{Set => JSet} + +import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveAuthzContext, HiveAuthzSessionContext, HiveOperationType} +import org.apache.ranger.authorization.spark.authorizer.SparkAccessType.SparkAccessType +import org.apache.ranger.plugin.policyengine.{RangerAccessRequestImpl, RangerPolicyEngine} +import org.apache.ranger.plugin.util.RangerAccessRequestUtil + +class RangerSparkAccessRequest private extends RangerAccessRequestImpl { + + private var accessType = SparkAccessType.NONE + + def this( + resource: RangerSparkResource, + user: String, + userGroups: JSet[String], + hiveOpTypeName: String, + accessType: SparkAccessType, + context: HiveAuthzContext, + sessionContext: HiveAuthzSessionContext, + clusterName: String) { + this() + this.setResource(resource) + this.setUser(user) + this.setUserGroups(userGroups) + this.setAccessTime(new Date) + this.setAction(hiveOpTypeName) + this.setSparkAccessType(accessType) + this.setUser(user) + if (context != null) { + this.setRequestData(context.getCommandString) + this.setRemoteIPAddress(context.getIpAddress) + } + if (sessionContext != null) { + this.setClientType(Option(sessionContext.getClientType).map(_.toString).orNull) + this.setSessionId(sessionContext.getSessionString) + } + this.setClusterName(clusterName) + } + + def this( + resource: RangerSparkResource, + user: String, + userGroups: JSet[String], + hiveOpType: HiveOperationType, + accessType: SparkAccessType, + context: HiveAuthzContext, + sessionContext: HiveAuthzSessionContext, + clusterName: String) = { + this(resource, user, userGroups, hiveOpType.name(), accessType, context, sessionContext, + clusterName) + } + + def this( + resource: RangerSparkResource, + user: String, + userGroups: JSet[String], + context: HiveAuthzContext, + sessionContext: HiveAuthzSessionContext, + clusterName: String) = { + this(resource, user, userGroups, "METADATA OPERATION", SparkAccessType.USE, context, + sessionContext, clusterName) + } + + def getSparkAccessType: SparkAccessType = accessType + + def setSparkAccessType(accessType: SparkAccessType): Unit = { + this.accessType = accessType + accessType match { + case SparkAccessType.USE => this.setAccessType(RangerPolicyEngine.ANY_ACCESS) + case SparkAccessType.ADMIN => this.setAccessType(RangerPolicyEngine.ADMIN_ACCESS) + case _ => this.setAccessType(accessType.toString.toLowerCase) + } + } + + def copy(): RangerSparkAccessRequest = { + val ret = new RangerSparkAccessRequest() + ret.setResource(getResource) + ret.setAccessType(getAccessType) + ret.setUser(getUser) + ret.setUserGroups(getUserGroups) + ret.setAccessTime(getAccessTime) + ret.setAction(getAction) + ret.setClientIPAddress(getClientIPAddress) + ret.setRemoteIPAddress(getRemoteIPAddress) + ret.setForwardedAddresses(getForwardedAddresses) + ret.setRequestData(getRequestData) + ret.setClientType(getClientType) + ret.setSessionId(getSessionId) + ret.setContext(RangerAccessRequestUtil.copyContext(getContext)) + ret.accessType = accessType + ret.setClusterName(getClusterName) + ret + } +} diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuditHandler.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuditHandler.scala new file mode 100644 index 0000000000..9303808e1d --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuditHandler.scala @@ -0,0 +1,26 @@ +/* + * 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.ranger.authorization.spark.authorizer + +import org.apache.ranger.plugin.audit.RangerDefaultAuditHandler + +class RangerSparkAuditHandler extends RangerDefaultAuditHandler { + + // TODO(Kent Yao): Implementing meaningfully audit functions + +} diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala new file mode 100644 index 0000000000..de83a61fa3 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala @@ -0,0 +1,429 @@ +/* + * 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.ranger.authorization.spark.authorizer + +import java.util.{List => JList} + +import org.apache.commons.lang.StringUtils +import org.apache.commons.logging.LogFactory +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.permission.FsAction +import org.apache.hadoop.hive.common.FileUtils +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.security.authorization.plugin._ +import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.{HivePrivilegeObjectType, HivePrivObjectActionType} +import org.apache.hadoop.security.UserGroupInformation +import org.apache.ranger.authorization.spark.authorizer.SparkAccessType.SparkAccessType +import org.apache.ranger.authorization.spark.authorizer.SparkObjectType.SparkObjectType +import org.apache.ranger.authorization.utils.StringUtil +import org.apache.ranger.plugin.policyengine.RangerAccessRequest +import org.apache.ranger.plugin.util.RangerPerfTracer + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +class RangerSparkAuthorizer( + metastoreClientFactory: HiveMetastoreClientFactory, + hiveConf: HiveConf, + hiveAuthenticator: HiveAuthenticationProvider, + sessionContext: HiveAuthzSessionContext) extends HiveAuthorizer { + + import RangerSparkAuthorizer._ + private val mUgi = if (hiveAuthenticator == null) { + null + } else { + Option(hiveAuthenticator.getUserName).map(UserGroupInformation.createRemoteUser).orNull + } + + private val sparkPlugin = RangerSparkPlugin.build().sessionContext(sessionContext).getOrCreate() + + override def getVersion: HiveAuthorizer.VERSION = HiveAuthorizer.VERSION.V1 + + override def checkPrivileges( + hiveOpType: HiveOperationType, + inputsHObjs: JList[SparkPrivilegeObject], + outputHObjs: JList[SparkPrivilegeObject], + context: HiveAuthzContext): Unit = { + + if (mUgi == null) { + throw new HiveAccessControlException("Permission denied: user information not available") + } + val user = mUgi.getShortUserName + val groups = mUgi.getGroupNames.toSet.asJava + val auditHandler = new RangerSparkAuditHandler + val perf = if (RangerPerfTracer.isPerfTraceEnabled(PERF_SPARKAUTH_REQUEST_LOG)) { + RangerPerfTracer.getPerfTracer(PERF_SPARKAUTH_REQUEST_LOG, + "RangerSparkAuthorizer.checkPrivileges()") + } else { + null + } + try { + val requests = new ArrayBuffer[RangerSparkAccessRequest]() + if (inputsHObjs.isEmpty && hiveOpType == HiveOperationType.SHOWDATABASES) { + val resource = new RangerSparkResource(SparkObjectType.DATABASE, None) + requests += new RangerSparkAccessRequest(resource, user, groups, hiveOpType.name, + SparkAccessType.USE, context, sessionContext, sparkPlugin.getClusterName) + } + + def addAccessRequest(objs: JList[SparkPrivilegeObject], isInput: Boolean): Unit = { + objs.asScala.foreach { obj => + val resource = getSparkResource(obj, hiveOpType) + if (resource != null) { + val objectName = obj.getObjectName + val objectType = resource.getObjectType + if (objectType == SparkObjectType.URI && isPathInFSScheme(objectName)) { + val fsAction = getURIAccessType(hiveOpType) + if (!isURIAccessAllowed(user, fsAction, objectName, hiveConf)) { + throw new HiveAccessControlException(s"Permission denied: user [$user] does not" + + s" have [${fsAction.name}] privilege on [$objectName]") + } + } else { + val accessType = getAccessType(obj, hiveOpType, objectType, isInput) + if (accessType != SparkAccessType.NONE && !requests.exists( + o => o.getSparkAccessType == accessType && o.getResource == resource)) { + requests += new RangerSparkAccessRequest(resource, user, groups, hiveOpType, + accessType, context, sessionContext, sparkPlugin.getClusterName) + } + } + } + } + } + + addAccessRequest(inputsHObjs, isInput = true) + addAccessRequest(outputHObjs, isInput = false) + requests.foreach { request => + val resource = request.getResource.asInstanceOf[RangerSparkResource] + if (resource.getObjectType == SparkObjectType.COLUMN && + StringUtils.contains(resource.getColumn, ",")) { + resource.setServiceDef(sparkPlugin.getServiceDef) + val colReqs: JList[RangerAccessRequest] = resource.getColumn.split(",") + .filter(StringUtils.isNotBlank).map { c => + val colRes = new RangerSparkResource(SparkObjectType.COLUMN, + Option(resource.getDatabase), resource.getTable, c) + val colReq = request.copy() + colReq.setResource(colRes) + colReq.asInstanceOf[RangerAccessRequest] + }.toList.asJava + val colResults = sparkPlugin.isAccessAllowed(colReqs, auditHandler) + if (colResults != null) { + for (c <- colResults.asScala) { + if (c != null && !c.getIsAllowed) { + throw new HiveAccessControlException(s"Permission denied: user [$user] does not" + + s" have [${request.getSparkAccessType}] privilege on [${resource.getAsString}]") + } + } + } + } else { + val result = sparkPlugin.isAccessAllowed(request, auditHandler) + if (result != null && result.getIsAllowed) { + throw new HiveAccessControlException(s"Permission denied: user [$user] does not" + + s" have [${request.getSparkAccessType}] privilege on [${resource.getAsString}]") + } + } + } + } finally { + // TODO(Kent Yao) add auditHandler.flush() + RangerPerfTracer.log(perf) + } + } + + override def filterListCmdObjects( + listObjs: JList[SparkPrivilegeObject], + context: HiveAuthzContext): JList[SparkPrivilegeObject] = { + if (LOG.isDebugEnabled) LOG.debug(s"==> filterListCmdObjects($listObjs, $context)") + + val perf = if (RangerPerfTracer.isPerfTraceEnabled(PERF_SPARKAUTH_REQUEST_LOG)) { + RangerPerfTracer.getPerfTracer(PERF_SPARKAUTH_REQUEST_LOG, + "RangerSparkAuthorizer.filterListCmdObjects()") + } else { + null + } + + val ret = + if (listObjs == null) { + LOG.debug("filterListCmdObjects: meta objects list was null!") + null + } else if (listObjs.isEmpty) { + listObjs + } else if (mUgi == null) { + LOG.warn("filterListCmdObjects: user information not available") + listObjs + } else { + if (LOG.isDebugEnabled) { + LOG.debug(s"filterListCmdObjects: number of input objects[${listObjs.size}]") + } + val user = mUgi.getShortUserName + val groups = mUgi.getGroupNames.toSet.asJava + if (LOG.isDebugEnabled) { + LOG.debug(s"filterListCmdObjects: user[$user], groups[$groups]") + } + + listObjs.asScala.filter { obj => + if (LOG.isDebugEnabled) { + LOG.debug(s"filterListCmdObjects: actionType[${obj.getActionType}]," + + s" objectType[${obj.getType}], objectName[${obj.getObjectName}]," + + s" dbName[${obj.getDbname}], columns[${obj.getColumns}]," + + s" partitionKeys[${obj.getPartKeys}];" + + s" context: commandString[${Option(context).map(_.getCommandString).getOrElse("")}" + + s"], ipAddress[${Option(context).map(_.getIpAddress).getOrElse("")}]") + } + createSparkResource(obj) match { + case Some(resource) => + val request = new RangerSparkAccessRequest( + resource, user, groups, context, sessionContext, sparkPlugin.getClusterName) + val result = sparkPlugin.isAccessAllowed(request) + if (request == null) { + LOG.error("filterListCmdObjects: Internal error: null RangerAccessResult object" + + " received back from isAccessAllowed()") + false + } else if (!result.getIsAllowed) { + if (LOG.isDebugEnabled) { + val path = resource.getAsString + LOG.debug(s"filterListCmdObjects: Permission denied: user [$user] does not have" + + s" [${request.getSparkAccessType}] privilege on [$path]. resource[$resource]," + + s" request[$request], result[$result]") + } + false + } else { + true + } + case _ => + LOG.error("filterListCmdObjects: RangerSparkResource returned by createHiveResource" + + " is null") + false + } + }.asJava + } + RangerPerfTracer.log(perf) + ret + } + + def getSparkResource( + hiveObj: SparkPrivilegeObject, + hiveOpType: HiveOperationType): RangerSparkResource = { + import SparkObjectType._ + val objectType = getObjectType(hiveObj, hiveOpType) + val resource = objectType match { + case DATABASE => RangerSparkResource(objectType, Option(hiveObj.getDbname)) + case TABLE | VIEW | PARTITION | FUNCTION => + RangerSparkResource(objectType, Option(hiveObj.getDbname), hiveObj.getObjectName) + case COLUMN => + RangerSparkResource(objectType, Option(hiveObj.getDbname), hiveObj.getObjectName, + StringUtils.join(hiveObj.getColumns, ",")) + case URI => RangerSparkResource(objectType, Option(hiveObj.getObjectName)) + case _ => null + } + if (resource != null) resource.setServiceDef(sparkPlugin.getServiceDef) + resource + } + + private def isPathInFSScheme(objectName: String): Boolean = { + objectName.nonEmpty && sparkPlugin.fsScheme.exists(objectName.startsWith) + } + + + ///////////////////////////////////////////////////////////////////////////////////////////////// + // Spark SQL supports no Hive DCLs, remain the functions with a default implementation // + ///////////////////////////////////////////////////////////////////////////////////////////////// + + override def grantPrivileges( + hivePrincipals: JList[HivePrincipal], + hivePrivileges: JList[HivePrivilege], + hivePrivObject: SparkPrivilegeObject, + grantorPrincipal: HivePrincipal, + grantOption: Boolean): Unit = {} + + override def revokePrivileges(hivePrincipals: JList[HivePrincipal], + hivePrivileges: JList[HivePrivilege], + hivePrivObject: SparkPrivilegeObject, + grantorPrincipal: HivePrincipal, + grantOption: Boolean): Unit = {} + + override def createRole(roleName: String, adminGrantor: HivePrincipal): Unit = {} + + override def dropRole(roleName: String): Unit = {} + + override def getPrincipalGrantInfoForRole(roleName: String): JList[HiveRoleGrant] = { + Seq.empty.asJava + } + + override def getRoleGrantInfoForPrincipal(principal: HivePrincipal): JList[HiveRoleGrant] = { + Seq.empty.asJava + } + + override def grantRole( + hivePrincipals: JList[HivePrincipal], + roles: JList[String], + grantOption: Boolean, + grantorPrinc: HivePrincipal): Unit = {} + + override def revokeRole( + hivePrincipals: JList[HivePrincipal], + roles: JList[String], + grantOption: Boolean, + grantorPrinc: HivePrincipal): Unit = {} + + override def getAllRoles: JList[String] = Seq.empty.asJava + + override def showPrivileges( + principal: HivePrincipal, + privObj: SparkPrivilegeObject): JList[HivePrivilegeInfo] = Seq.empty.asJava + + override def setCurrentRole(roleName: String): Unit = {} + + override def getCurrentRoleNames: JList[String] = Seq.empty.asJava + + override def applyAuthorizationConfigPolicy(hiveConf: HiveConf): Unit = {} +} + +object RangerSparkAuthorizer { + import HivePrivilegeObjectType._ + + private val LOG = LogFactory.getLog(classOf[RangerSparkAuthorizer].getSimpleName.stripSuffix("$")) + + private val PERF_SPARKAUTH_REQUEST_LOG = RangerPerfTracer.getPerfLogger("sparkauth.request") + + def createSparkResource(privilegeObject: SparkPrivilegeObject): Option[RangerSparkResource] = { + val objectName = privilegeObject.getObjectName + val dbName = privilegeObject.getDbname + val objectType = privilegeObject.getType + objectType match { + case DATABASE => + Some(RangerSparkResource(SparkObjectType.DATABASE, Option(objectName))) + case TABLE_OR_VIEW => + Some(RangerSparkResource(SparkObjectType.DATABASE, Option(dbName), objectName)) + case _ => + LOG.warn(s"RangerSparkAuthorizer.createHiveResource: unexpected objectType: $objectType") + None + } + } + + private def getObjectType( + hiveObj: SparkPrivilegeObject, + hiveOpType: HiveOperationType): SparkObjectType = hiveObj.getType match { + case DATABASE | null => SparkObjectType.DATABASE + case PARTITION => SparkObjectType.PARTITION + case TABLE_OR_VIEW if hiveOpType.name.toLowerCase.contains("view") => SparkObjectType.VIEW + case TABLE_OR_VIEW => SparkObjectType.TABLE + case FUNCTION => SparkObjectType.FUNCTION + case DFS_URI | LOCAL_URI => SparkObjectType.URI + case _ => SparkObjectType.NONE + } + + private def getURIAccessType(hiveOpType: HiveOperationType): FsAction = { + import HiveOperationType._ + + hiveOpType match { + case LOAD | IMPORT => FsAction.READ + case EXPORT => FsAction.WRITE + case CREATEDATABASE | CREATETABLE | CREATETABLE_AS_SELECT | ALTERDATABASE | + ALTERDATABASE_OWNER | ALTERTABLE_ADDCOLS | ALTERTABLE_REPLACECOLS | + ALTERTABLE_RENAMECOL | ALTERTABLE_RENAMEPART | ALTERTABLE_RENAME | + ALTERTABLE_DROPPARTS | ALTERTABLE_ADDPARTS | ALTERTABLE_TOUCH | + ALTERTABLE_ARCHIVE | ALTERTABLE_UNARCHIVE | ALTERTABLE_PROPERTIES | + ALTERTABLE_SERIALIZER | ALTERTABLE_PARTCOLTYPE | ALTERTABLE_SERDEPROPERTIES | + ALTERTABLE_CLUSTER_SORT | ALTERTABLE_BUCKETNUM | ALTERTABLE_UPDATETABLESTATS | + ALTERTABLE_UPDATEPARTSTATS | ALTERTABLE_PROTECTMODE | ALTERTABLE_FILEFORMAT | + ALTERTABLE_LOCATION | ALTERINDEX_PROPS | ALTERTABLE_MERGEFILES | ALTERTABLE_SKEWED | + ALTERTABLE_COMPACT | ALTERPARTITION_SERIALIZER | ALTERPARTITION_SERIALIZER | + ALTERPARTITION_SERDEPROPERTIES | ALTERPARTITION_BUCKETNUM | ALTERPARTITION_PROTECTMODE | + ALTERPARTITION_FILEFORMAT | ALTERPARTITION_LOCATION | ALTERPARTITION_MERGEFILES | + ALTERTBLPART_SKEWED_LOCATION | QUERY => FsAction.ALL + case _ => FsAction.NONE + } + } + + private def isURIAccessAllowed( + userName: String, action: FsAction, uri: String, conf: HiveConf): Boolean = action match { + case FsAction.NONE => true + case _ => + try { + val filePath = new Path(uri) + val fs = FileSystem.get(filePath.toUri, conf) + val fileStat = fs.globStatus(filePath) + if (fileStat != null && fileStat.nonEmpty) fileStat.forall { file => + FileUtils.isOwnerOfFileHierarchy(fs, file, userName) || + FileUtils.isActionPermittedForFileHierarchy(fs, file, userName, action) + } else { + val file = FileUtils.getPathOrParentThatExists(fs, filePath) + FileUtils.checkFileAccessWithImpersonation(fs, file, action, userName) + true + } + } catch { + case e: Exception => + LOG.error("Error getting permissions for " + uri, e) + false + } + } + + private def getAccessType( + hiveObj: SparkPrivilegeObject, + hiveOpType: HiveOperationType, + sparkObjectType: SparkObjectType, + isInput: Boolean): SparkAccessType = { + sparkObjectType match { + case SparkObjectType.URI if isInput => SparkAccessType.READ + case SparkObjectType.URI => SparkAccessType.WRITE + case _ => hiveObj.getActionType match { + case HivePrivObjectActionType.INSERT | HivePrivObjectActionType.INSERT_OVERWRITE | + HivePrivObjectActionType.UPDATE | HivePrivObjectActionType.DELETE => + SparkAccessType.UPDATE + case HivePrivObjectActionType.OTHER => + import HiveOperationType._ + hiveOpType match { + case CREATEDATABASE if hiveObj.getType == HivePrivilegeObjectType.DATABASE => + SparkAccessType.CREATE + case CREATEFUNCTION if hiveObj.getType == HivePrivilegeObjectType.FUNCTION => + SparkAccessType.CREATE + case CREATETABLE | CREATEVIEW | CREATETABLE_AS_SELECT + if hiveObj.getType == HivePrivilegeObjectType.TABLE_OR_VIEW => + if (isInput) SparkAccessType.SELECT else SparkAccessType.CREATE + case ALTERDATABASE | ALTERDATABASE_OWNER | ALTERINDEX_PROPS | ALTERINDEX_REBUILD | + ALTERPARTITION_BUCKETNUM | ALTERPARTITION_FILEFORMAT | ALTERPARTITION_LOCATION | + ALTERPARTITION_MERGEFILES | ALTERPARTITION_PROTECTMODE | + ALTERPARTITION_SERDEPROPERTIES | ALTERPARTITION_SERIALIZER | ALTERTABLE_ADDCOLS | + ALTERTABLE_ADDPARTS | ALTERTABLE_ARCHIVE | ALTERTABLE_BUCKETNUM | + ALTERTABLE_CLUSTER_SORT | ALTERTABLE_COMPACT | ALTERTABLE_DROPPARTS | + ALTERTABLE_FILEFORMAT | ALTERTABLE_LOCATION | ALTERTABLE_MERGEFILES | + ALTERTABLE_PARTCOLTYPE | ALTERTABLE_PROPERTIES | ALTERTABLE_PROTECTMODE | + ALTERTABLE_RENAME | ALTERTABLE_RENAMECOL | ALTERTABLE_RENAMEPART | + ALTERTABLE_REPLACECOLS | ALTERTABLE_SERDEPROPERTIES | ALTERTABLE_SERIALIZER | + ALTERTABLE_SKEWED | ALTERTABLE_TOUCH | ALTERTABLE_UNARCHIVE | + ALTERTABLE_UPDATEPARTSTATS | ALTERTABLE_UPDATETABLESTATS | + ALTERTBLPART_SKEWED_LOCATION | ALTERVIEW_AS | ALTERVIEW_PROPERTIES | + ALTERVIEW_RENAME | DROPVIEW_PROPERTIES | MSCK => SparkAccessType.ALTER + case DROPFUNCTION | DROPINDEX | DROPTABLE | DROPVIEW | DROPDATABASE => + SparkAccessType.DROP + case IMPORT => if (isInput) SparkAccessType.SELECT else SparkAccessType.CREATE + case EXPORT | LOAD => if (isInput) SparkAccessType.SELECT else SparkAccessType.UPDATE + case QUERY | SHOW_TABLESTATUS | SHOW_CREATETABLE | SHOWINDEXES | SHOWPARTITIONS | + SHOW_TBLPROPERTIES | ANALYZE_TABLE => SparkAccessType.SELECT + case SHOWCOLUMNS | DESCTABLE => + StringUtil.toLower(RangerSparkPlugin.showColumnsOption) match { + case "show-all" => SparkAccessType.USE + case _ => SparkAccessType.SELECT + } + case SHOWDATABASES | SWITCHDATABASE | DESCDATABASE| SHOWTABLES => SparkAccessType.USE + case TRUNCATETABLE => SparkAccessType.UPDATE + case _ => SparkAccessType.NONE + } + } + } + } +} diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizerFactory.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizerFactory.scala new file mode 100644 index 0000000000..2150cfb5bf --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizerFactory.scala @@ -0,0 +1,32 @@ +/* + * 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.ranger.authorization.spark.authorizer + +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveAuthorizer, HiveAuthorizerFactory, HiveAuthzSessionContext, HiveMetastoreClientFactory} +import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider + +class RangerSparkAuthorizerFactory extends HiveAuthorizerFactory { + override def createHiveAuthorizer( + metastoreClientFactory: HiveMetastoreClientFactory, + conf: HiveConf, + hiveAuthenticator: HiveAuthenticationProvider, + ctx: HiveAuthzSessionContext): HiveAuthorizer = { + new RangerSparkAuthorizer(metastoreClientFactory, conf, hiveAuthenticator, ctx) + } +} diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkPlugin.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkPlugin.scala new file mode 100644 index 0000000000..a18354a72f --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkPlugin.scala @@ -0,0 +1,67 @@ +/* + * 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.ranger.authorization.spark.authorizer + +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext.CLIENT_TYPE +import org.apache.ranger.authorization.hadoop.config.RangerConfiguration +import org.apache.ranger.plugin.service.RangerBasePlugin + +class RangerSparkPlugin(appType: String) extends RangerBasePlugin("spark", appType) { + lazy val fsScheme: Array[String] = RangerConfiguration.getInstance() + .get("ranger.plugin.spark.urlauth.filesystem.schemes", "hdfs:,file:") + .split(",") + .map(_.trim) +} + +object RangerSparkPlugin { + + private val rangerConf: RangerConfiguration = RangerConfiguration.getInstance + + val showColumnsOption: String = rangerConf.get( + "xasecure.spark.describetable.showcolumns.authorization.option", "NONE") + + def build(): Builder = new Builder + + class Builder { + + @volatile private var appType: String = "unknown" + @volatile private var sparkPlugin: RangerSparkPlugin = _ + + def sessionContext(sessionContext: HiveAuthzSessionContext): Builder = { + appType = Option(sessionContext).map(_.getClientType) match { + case Some(v) => v match { + case CLIENT_TYPE.HIVECLI => "hiveCLI" + case CLIENT_TYPE.HIVESERVER2 => "hiveServer2" + } + case _ => "unknown" + } + this + } + + def getOrCreate(): RangerSparkPlugin = RangerSparkPlugin.synchronized { + if (sparkPlugin == null) { + sparkPlugin = new RangerSparkPlugin(appType) + sparkPlugin.init() + sparkPlugin + } else { + sparkPlugin + } + } + } +} diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala new file mode 100644 index 0000000000..1dd4c713f4 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala @@ -0,0 +1,90 @@ +/* + * 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.ranger.authorization.spark.authorizer + +import org.apache.ranger.authorization.spark.authorizer.SparkObjectType.SparkObjectType +import org.apache.ranger.plugin.policyengine.RangerAccessResourceImpl + +class RangerSparkResource( + objectType: SparkObjectType, + databaseOrUrl: Option[String], + tableOrUdf: String, + column: String) extends RangerAccessResourceImpl { + import SparkObjectType._ + import RangerSparkResource._ + + def this(objectType: SparkObjectType, databaseOrUrl: Option[String], tableOrUdf: String) = { + this(objectType, databaseOrUrl, tableOrUdf, null) + } + + def this(objectType: SparkObjectType, databaseOrUrl: Option[String]) = { + this(objectType, databaseOrUrl, null) + } + + objectType match { + case DATABASE => setValue(KEY_DATABASE, databaseOrUrl.getOrElse("*")) + case FUNCTION => + setValue(KEY_DATABASE, databaseOrUrl.getOrElse("")) + setValue(KEY_UDF, tableOrUdf) + case COLUMN => + setValue(KEY_DATABASE, databaseOrUrl) + setValue(KEY_TABLE, tableOrUdf) + setValue(KEY_COLUMN, column) + case TABLE | VIEW | PARTITION => + setValue(KEY_DATABASE, databaseOrUrl) + setValue(KEY_TABLE, tableOrUdf) + case URI => setValue(KEY_URL, databaseOrUrl) + case _ => + } + + def getObjectType: SparkObjectType = objectType + + def getDatabase: String = getValue(KEY_DATABASE).asInstanceOf[String] + + def getTable: String = getValue(KEY_TABLE).asInstanceOf[String] + + def getUdf: String = getValue(KEY_UDF).asInstanceOf[String] + + def getColumn: String = getValue(KEY_COLUMN).asInstanceOf[String] + + def getUrl: String = getValue(KEY_URL).asInstanceOf[String] + +} + +object RangerSparkResource { + + def apply(objectType: SparkObjectType, databaseOrUrl: Option[String], tableOrUdf: String, + column: String): RangerSparkResource = { + new RangerSparkResource(objectType, databaseOrUrl, tableOrUdf, column) + } + + def apply(objectType: SparkObjectType, databaseOrUrl: Option[String], + tableOrUdf: String): RangerSparkResource = { + new RangerSparkResource(objectType, databaseOrUrl, tableOrUdf) + } + + def apply(objectType: SparkObjectType, databaseOrUrl: Option[String]): RangerSparkResource = { + new RangerSparkResource(objectType, databaseOrUrl) + } + + private val KEY_DATABASE = "database" + private val KEY_TABLE = "table" + private val KEY_UDF = "udf" + private val KEY_COLUMN = "column" + private val KEY_URL = "url" +} diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala index 9354822fc7..f8988083ea 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala @@ -1,18 +1,18 @@ /* * 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 + * 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 + * 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. + * 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.ranger.authorization.spark.authorizer diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkAccessType.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkAccessType.scala new file mode 100644 index 0000000000..1ce31d1697 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkAccessType.scala @@ -0,0 +1,25 @@ +/* + * 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.ranger.authorization.spark.authorizer + +object SparkAccessType extends Enumeration { + type SparkAccessType = Value + + val NONE, CREATE, ALTER, DROP, SELECT, UPDATE, USE, READ, WRITE, ALL, ADMIN = Value +} + diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkObjectType.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkObjectType.scala new file mode 100644 index 0000000000..7871e836f0 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkObjectType.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.ranger.authorization.spark.authorizer + +object SparkObjectType extends Enumeration { + type SparkObjectType = Value + + val NONE, DATABASE, TABLE, VIEW, PARTITION, COLUMN, FUNCTION, URI = Value +} diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/authorizer.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/authorizer.scala index af21a17db6..8a85ce07fc 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/authorizer.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/authorizer.scala @@ -1,26 +1,32 @@ /* * 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 + * 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 + * 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. + * 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.ranger.authorization.spark +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType import org.apache.spark.sql.SparkSessionExtensions package object authorizer { type Extensions = SparkSessionExtensions => Unit + type SparkPrivilegeObject = HivePrivilegeObject + + type SparkPrivilegeObjectType = HivePrivilegeObjectType + } diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/AuthzUtils.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/AuthzUtils.scala new file mode 100644 index 0000000000..ae124bf75a --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/AuthzUtils.scala @@ -0,0 +1,45 @@ +/* + * 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 + +import scala.util.{Failure, Success, Try} + +private[sql] object AuthzUtils { + + def getFieldVal(o: Any, name: String): Any = { + Try { + val field = o.getClass.getDeclaredField(name) + field.setAccessible(true) + field.get(o) + } match { + case Success(value) => value + case Failure(exception) => throw exception + } + } + + def setFieldVal(o: Any, name: String, value: Any): Unit = { + Try { + val field = o.getClass.getDeclaredField(name) + field.setAccessible(true) + field.set(o, value.asInstanceOf[AnyRef]) + } match { + case Failure(exception) => throw exception + case _ => + } + } +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala index 7724df35fc..215cdc46e2 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala @@ -22,15 +22,14 @@ import java.io.File import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.ql.plan.HiveOperation import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveAuthzContext, HiveOperationType} - import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTempViewUsing, InsertIntoDataSourceCommand, InsertIntoHadoopFsRelationCommand} -import org.apache.spark.sql.hive.{HiveExternalCatalog, PrivilegesBuilder} -import org.apache.spark.sql.hive.client.AuthzImpl +import org.apache.spark.sql.hive.client.RangerSparkAuthzImpl import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand +import org.apache.spark.sql.hive.{HiveExternalCatalog, PrivilegesBuilder} import org.apache.spark.util.Utils trait Authorizable extends Rule[LogicalPlan] { @@ -54,12 +53,13 @@ trait Authorizable extends Rule[LogicalPlan] { val (in, out) = PrivilegesBuilder.build(plan) spark.sharedState.externalCatalog match { case _: HiveExternalCatalog => - AuthzImpl.checkPrivileges(spark, operationType, in, out, authzContext) + RangerSparkAuthzImpl.checkPrivileges(spark, operationType, in, out, authzContext) case _ => } - // iff no exception. - // We just return the original plan here, so this rule will be executed only once. - plan + + // Row level filtering + new RangerSparkRowFilter(spark).build(plan) + // TODO(Kent Yao) applying column masking } def policyCacheDir: Option[String] = { diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilter.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilter.scala new file mode 100644 index 0000000000..808f0ec0a3 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilter.scala @@ -0,0 +1,92 @@ +/* + * 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.catalyst.optimizer + +import scala.collection.JavaConverters._ + +import org.apache.commons.lang.StringUtils +import org.apache.hadoop.security.UserGroupInformation +import org.apache.ranger.authorization.spark.authorizer._ +import org.apache.ranger.plugin.policyengine.RangerAccessResult +import org.apache.spark.sql.AuthzUtils.getFieldVal +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.{Dataset, SparkSession} + +class RangerSparkRowFilter(spark: SparkSession) { + + import RangerSparkRowFilter._ + + private lazy val sparkPlugin = RangerSparkPlugin.build().getOrCreate() + + /** + * Transform a Relation to a parsed [[LogicalPlan]] with specified row filter expressions + * @param plan the original [[LogicalPlan]] + * @param table a Spark [[CatalogTable]] representation + * @return A new Spark [[LogicalPlan]] with specified row filter expressions + */ + private def applyingRowFilterExpr(plan: LogicalPlan, table: CatalogTable): LogicalPlan = { + val auditHandler = new RangerSparkAuditHandler() + try { + val identifier = table.identifier + val resource = + RangerSparkResource(SparkObjectType.TABLE, identifier.database, identifier.table) + val ugi = UserGroupInformation.getCurrentUser + val request = new RangerSparkAccessRequest(resource, ugi.getShortUserName, + ugi.getGroupNames.toSet.asJava, SparkObjectType.TABLE.toString, SparkAccessType.SELECT, + null, null, sparkPlugin.getClusterName) + val result = sparkPlugin.evalRowFilterPolicies(request, auditHandler) + if (isRowFilterEnabled(result)) { + val sql = s"select ${plan.output.map(_.name).mkString(",")} from ${table.qualifiedName}" + + s" where ${result.getFilterExpr}" + spark.sessionState.sqlParser.parsePlan(sql) + } else { + plan + } + } catch { + case e: Exception => throw e + } + } + + /** + * Transform a spark logical plan to another plan with the row filer expressions + * @param plan the original [[LogicalPlan]] + * @return the logical plan with row filer expressions applied + */ + def build(plan: LogicalPlan): LogicalPlan = { + var newPlan = plan + newPlan = plan transform { + case h if h.nodeName == "HiveTableRelation" => + val ct = getFieldVal(h, "tableMeta").asInstanceOf[CatalogTable] + applyingRowFilterExpr(h, ct) + case m if m.nodeName == "MetastoreRelation" => + val ct = getFieldVal(m, "catalogTable").asInstanceOf[CatalogTable] + applyingRowFilterExpr(m, ct) + case l: LogicalRelation if l.catalogTable.isDefined => + applyingRowFilterExpr(l, l.catalogTable.get) + } + Dataset.ofRows(spark, newPlan).queryExecution.optimizedPlan + } +} + +object RangerSparkRowFilter { + def isRowFilterEnabled(result: RangerAccessResult): Boolean = { + result != null && result.isRowFilterEnabled && StringUtils.isNotEmpty(result.getFilterExpr) + } +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/HivePrivilegeObject.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SparkPrivilegeObject.scala similarity index 98% rename from plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/HivePrivilegeObject.scala rename to plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SparkPrivilegeObject.scala index 2873f4456e..f1c7565b4f 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/HivePrivilegeObject.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SparkPrivilegeObject.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObje /** * Helper class for initializing [[HBO]] with more Constructors. */ -private[sql] object HivePrivilegeObject { +private[sql] object SparkPrivilegeObject { def apply( `type`: HivePrivilegeObjectType, dbname: String, diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/AuthzUtils.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/AuthzUtils.scala deleted file mode 100644 index bbfe212630..0000000000 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/AuthzUtils.scala +++ /dev/null @@ -1,45 +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.spark.sql.hive - -import scala.util.{Failure, Success, Try} - -private[hive] object AuthzUtils { - - def getFieldVal(o: Any, name: String): Any = { - Try { - val field = o.getClass.getDeclaredField(name) - field.setAccessible(true) - field.get(o) - } match { - case Success(value) => value - case Failure(exception) => throw exception - } - } - - def setFieldVal(o: Any, name: String, value: Any): Unit = { - Try { - val field = o.getClass.getDeclaredField(name) - field.setAccessible(true) - field.set(o, value.asInstanceOf[AnyRef]) - } match { - case Failure(exception) => throw exception - case _ => - } - } -} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala index 8be91f1367..5c82e5addc 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala @@ -29,16 +29,16 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.NamedExpression -import org.apache.spark.sql.catalyst.optimizer.HivePrivilegeObject +import org.apache.spark.sql.catalyst.optimizer.SparkPrivilegeObject import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources._ -import org.apache.spark.sql.hive.AuthzUtils._ +import org.apache.spark.sql.AuthzUtils._ import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand import org.apache.spark.sql.types.StructField /** - * [[LogicalPlan]] -> list of [[HivePrivilegeObject]]s + * [[LogicalPlan]] -> list of [[SparkPrivilegeObject]]s */ private[sql] object PrivilegesBuilder { @@ -368,7 +368,7 @@ private[sql] object PrivilegesBuilder { dbName: String, hivePrivilegeObjects: JList[HPO]): Unit = { hivePrivilegeObjects.add( - HivePrivilegeObject(HivePrivilegeObjectType.DATABASE, dbName, dbName)) + SparkPrivilegeObject(HivePrivilegeObjectType.DATABASE, dbName, dbName)) } /** @@ -382,7 +382,7 @@ private[sql] object PrivilegesBuilder { dbOption match { case Some(db) => hivePrivilegeObjects.add( - HivePrivilegeObject(HivePrivilegeObjectType.DATABASE, db, db)) + SparkPrivilegeObject(HivePrivilegeObjectType.DATABASE, db, db)) case _ => } } @@ -398,7 +398,7 @@ private[sql] object PrivilegesBuilder { tableIdentifier.database match { case Some(db) => hivePrivilegeObjects.add( - HivePrivilegeObject(HivePrivilegeObjectType.DATABASE, db, db)) + SparkPrivilegeObject(HivePrivilegeObjectType.DATABASE, db, db)) case _ => } } @@ -422,7 +422,7 @@ private[sql] object PrivilegesBuilder { val tbName = tableIdentifier.table val hivePrivObjectActionType = getHivePrivObjActionType(mode) hivePrivilegeObjects.add( - HivePrivilegeObject( + SparkPrivilegeObject( HivePrivilegeObjectType.TABLE_OR_VIEW, db, tbName, @@ -447,7 +447,7 @@ private[sql] object PrivilegesBuilder { databaseName match { case Some(db) => hivePrivilegeObjects.add( - HivePrivilegeObject(HivePrivilegeObjectType.FUNCTION, db, functionName)) + SparkPrivilegeObject(HivePrivilegeObjectType.FUNCTION, db, functionName)) case _ => } } diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/AuthzImpl.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/RangerSparkAuthzImpl.scala similarity index 96% rename from plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/AuthzImpl.scala rename to plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/RangerSparkAuthzImpl.scala index c0fc3614c3..6eeaf725f6 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/AuthzImpl.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/RangerSparkAuthzImpl.scala @@ -18,13 +18,12 @@ package org.apache.spark.sql.hive.client import java.util.{List => JList} - import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.ql.security.authorization.plugin._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.security.UserGroupInformation -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.hive.{AuthzUtils, HiveExternalCatalog} +import org.apache.spark.sql.{AuthzUtils, SparkSession} +import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.internal.NonClosableMutableURLClassLoader /** @@ -42,7 +41,7 @@ import org.apache.spark.sql.internal.NonClosableMutableURLClassLoader * generated, will be reused then. * */ -object AuthzImpl { +private[sql] object RangerSparkAuthzImpl { private val logger = LogFactory.getLog(getClass.getSimpleName.stripSuffix("$")) @@ -90,7 +89,7 @@ object AuthzImpl { authz.checkPrivileges(hiveOpType, inputObjs, outputObjs, context) } catch { case hae: HiveAccessControlException => - error( + logger.error( s""" |+===============================+ ||Spark SQL Authorization Failure| diff --git a/src/main/assembly/plugin-spark.xml b/src/main/assembly/plugin-spark.xml index 658c806916..1d357beaec 100644 --- a/src/main/assembly/plugin-spark.xml +++ b/src/main/assembly/plugin-spark.xml @@ -31,37 +31,6 @@ 755 644 false - - - /jars - - org.apache.ranger:ranger-hive-plugin-shim - org.apache.ranger:ranger-plugin-classloader - - - - /jars/ranger-hive-plugin-impl - - org.eclipse.persistence:eclipselink - com.google.code.gson:gson - org.eclipse.persistence:javax.persistence - org.apache.httpcomponents:httpmime - org.apache.httpcomponents:httpclient - org.apache.httpcomponents:httpcore - org.noggit:noggit - org.apache.solr:solr-solrj - com.sun.jersey:jersey-bundle - org.apache.ranger:ranger-hive-plugin - org.apache.ranger:ranger-plugins-common - org.apache.ranger:ranger-plugins-cred - org.apache.ranger:ranger-plugins-audit - org.codehaus.jackson:jackson-core-asl - org.codehaus.jackson:jackson-jaxrs - org.codehaus.jackson:jackson-mapper-asl - org.codehaus.jackson:jackson-xc - - - org.apache.ranger:ranger-spark-plugin From 41054c1de47a8e4caf13efd50a93b3a67dc5c5f1 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 1 Apr 2019 16:02:46 +0800 Subject: [PATCH 08/35] spark version 2.2.1 --- plugin-spark/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml index c66a4d88ac..7928f0c7e1 100644 --- a/plugin-spark/pom.xml +++ b/plugin-spark/pom.xml @@ -26,7 +26,7 @@ 2.11.8 2.11 2.2.6 - 2.2.3 + 2.2.1 provided 1.2.1.spark2 ranger_spark_project From 5d0beb7b6fbf29363e91a5935d53e141553d8543 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 1 Apr 2019 18:43:20 +0800 Subject: [PATCH 09/35] pom adding spark.hive version --- plugin-spark/pom.xml | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml index 7928f0c7e1..ffa314db57 100644 --- a/plugin-spark/pom.xml +++ b/plugin-spark/pom.xml @@ -57,12 +57,12 @@ ${spark.scope} - - - - - - + + org.spark-project.hive + hive-exec + 1.2.1.spark2 + provided + org.apache.ranger From 2a0f5867376a0f40fd52eb95e9f242a2112dda9c Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 1 Apr 2019 19:36:50 +0800 Subject: [PATCH 10/35] pom --- plugin-spark/pom.xml | 8 -------- .../spark/sql/catalyst/optimizer/Authorizable.scala | 1 - 2 files changed, 9 deletions(-) diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml index ffa314db57..ff34185fcf 100644 --- a/plugin-spark/pom.xml +++ b/plugin-spark/pom.xml @@ -56,14 +56,6 @@ ${spark.version} ${spark.scope} - - - org.spark-project.hive - hive-exec - 1.2.1.spark2 - provided - - org.apache.ranger ranger-plugins-common diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala index 215cdc46e2..bd2e6cbed7 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala @@ -66,7 +66,6 @@ trait Authorizable extends Rule[LogicalPlan] { Option(spark.sparkContext.hadoopConfiguration.get("ranger.plugin.hive.policy.cache.dir")) } - def createCacheDirIfNonExists(dir: String): Unit = { val file = new File(dir) if (!file.exists()) { From c7422d1b6fff0112ebf71aa1a55bed09620be7bf Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 2 Apr 2019 10:41:00 +0800 Subject: [PATCH 11/35] policy cache dir --- .../spark/authorizer/RangerSparkPlugin.scala | 23 ++++++++++++ .../sql/catalyst/optimizer/Authorizable.scala | 36 +------------------ 2 files changed, 24 insertions(+), 35 deletions(-) diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkPlugin.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkPlugin.scala index a18354a72f..fa3d630035 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkPlugin.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkPlugin.scala @@ -17,16 +17,39 @@ package org.apache.ranger.authorization.spark.authorizer +import java.io.{File, IOException} + +import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext.CLIENT_TYPE import org.apache.ranger.authorization.hadoop.config.RangerConfiguration import org.apache.ranger.plugin.service.RangerBasePlugin class RangerSparkPlugin(appType: String) extends RangerBasePlugin("spark", appType) { + import RangerSparkPlugin._ + + private val LOG = LogFactory.getLog(classOf[RangerSparkPlugin]) + lazy val fsScheme: Array[String] = RangerConfiguration.getInstance() .get("ranger.plugin.spark.urlauth.filesystem.schemes", "hdfs:,file:") .split(",") .map(_.trim) + + override def init(): Unit = { + super.init() + val cacheDir = new File(rangerConf.get("ranger.plugin.spark.policy.cache.dir")) + if (cacheDir.exists() && + (!cacheDir.isDirectory || !cacheDir.canRead || !cacheDir.canWrite)) { + throw new IOException("Policy cache directory already exists at" + + cacheDir.getAbsolutePath + ", but it is unavailable") + } + + if (!cacheDir.exists() && !cacheDir.mkdirs()) { + throw new IOException("Unable to create ranger policy cache directory at" + + cacheDir.getAbsolutePath) + } + LOG.info("Policy cache directory successfully set to " + cacheDir.getAbsolutePath) + } } object RangerSparkPlugin { diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala index bd2e6cbed7..189b6cf168 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.optimizer -import java.io.File - import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.ql.plan.HiveOperation import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveAuthzContext, HiveOperationType} @@ -27,10 +25,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTempViewUsing, InsertIntoDataSourceCommand, InsertIntoHadoopFsRelationCommand} +import org.apache.spark.sql.hive.{HiveExternalCatalog, PrivilegesBuilder} import org.apache.spark.sql.hive.client.RangerSparkAuthzImpl import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand -import org.apache.spark.sql.hive.{HiveExternalCatalog, PrivilegesBuilder} -import org.apache.spark.util.Utils trait Authorizable extends Rule[LogicalPlan] { @@ -62,37 +59,6 @@ trait Authorizable extends Rule[LogicalPlan] { // TODO(Kent Yao) applying column masking } - def policyCacheDir: Option[String] = { - Option(spark.sparkContext.hadoopConfiguration.get("ranger.plugin.hive.policy.cache.dir")) - } - - def createCacheDirIfNonExists(dir: String): Unit = { - val file = new File(dir) - if (!file.exists()) { - if (file.mkdirs()) { - logger.info("Creating ranger policy cache directory at " + file.getAbsolutePath) - file.deleteOnExit() - } else { - logger.warn("Unable to create ranger policy cache directory at " + file.getAbsolutePath) - } - } - } - - policyCacheDir match { - case Some(dir) => createCacheDirIfNonExists(dir) - case _ => - // load resources from ranger configuration files - Option(Utils.getContextOrSparkClassLoader.getResource("ranger-hive-security.xml")) match { - case Some(url) => - spark.sparkContext.hadoopConfiguration.addResource(url) - policyCacheDir match { - case Some(dir) => createCacheDirIfNonExists(dir) - case _ => - } - case _ => - } - } - /** * Mapping of [[LogicalPlan]] -> [[HiveOperation]] * @param plan a spark LogicalPlan From 1ff8c133df85c77eb9a98dd2c6516b1941ba426a Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 2 Apr 2019 11:16:20 +0800 Subject: [PATCH 12/35] shade transformer --- plugin-spark/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml index ff34185fcf..1b5c63b3b9 100644 --- a/plugin-spark/pom.xml +++ b/plugin-spark/pom.xml @@ -299,6 +299,11 @@ shade + + + + + From f40623b133f702043601c3ea32c9a8e9943c340c Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 2 Apr 2019 14:36:02 +0800 Subject: [PATCH 13/35] test RangerAdminRESTClient --- .../admin/client/RangerAdminRESTClient.java | 46 +++++++++++-------- 1 file changed, 28 insertions(+), 18 deletions(-) diff --git a/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java b/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java index dddfbc7fef..d1d183423e 100644 --- a/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java +++ b/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java @@ -104,23 +104,34 @@ public ServicePolicies getServicePoliciesIfUpdated(final long lastKnownVersion, UserGroupInformation user = MiscUtil.getUGILoginUser(); boolean isSecureMode = user != null && UserGroupInformation.isSecurityEnabled(); ClientResponse response = null; - if (isSecureMode) { - if (LOG.isDebugEnabled()) { - LOG.debug("Checking Service policy if updated as user : " + user); - } - PrivilegedAction action = new PrivilegedAction() { - public ClientResponse run() { - WebResource secureWebResource = createWebResource(RangerRESTUtils.REST_URL_POLICY_GET_FOR_SECURE_SERVICE_IF_UPDATED + serviceName) - .queryParam(RangerRESTUtils.REST_PARAM_LAST_KNOWN_POLICY_VERSION, Long.toString(lastKnownVersion)) - .queryParam(RangerRESTUtils.REST_PARAM_LAST_ACTIVATION_TIME, Long.toString(lastActivationTimeInMillis)) - .queryParam(RangerRESTUtils.REST_PARAM_PLUGIN_ID, pluginId) - .queryParam(RangerRESTUtils.REST_PARAM_CLUSTER_NAME, clusterName); - return secureWebResource.accept(RangerRESTUtils.REST_MIME_TYPE_JSON).get(ClientResponse.class); - } - }; - response = user.doAs(action); - } else { - if (LOG.isDebugEnabled()) { +// if (isSecureMode) { +// if (LOG.isDebugEnabled()) { +// LOG.debug("Checking Service policy if updated as user : " + user); +// } +// PrivilegedAction action = new PrivilegedAction() { +// public ClientResponse run() { +// WebResource secureWebResource = createWebResource(RangerRESTUtils.REST_URL_POLICY_GET_FOR_SECURE_SERVICE_IF_UPDATED + serviceName) +// .queryParam(RangerRESTUtils.REST_PARAM_LAST_KNOWN_POLICY_VERSION, Long.toString(lastKnownVersion)) +// .queryParam(RangerRESTUtils.REST_PARAM_LAST_ACTIVATION_TIME, Long.toString(lastActivationTimeInMillis)) +// .queryParam(RangerRESTUtils.REST_PARAM_PLUGIN_ID, pluginId) +// .queryParam(RangerRESTUtils.REST_PARAM_CLUSTER_NAME, clusterName); +// return secureWebResource.accept(RangerRESTUtils.REST_MIME_TYPE_JSON).get(ClientResponse.class); +// } +// }; +// response = user.doAs(action); +// } else { +// if (LOG.isDebugEnabled()) { +// LOG.debug("Checking Service policy if updated with old api call"); +// } +// WebResource webResource = createWebResource(RangerRESTUtils.REST_URL_POLICY_GET_FOR_SERVICE_IF_UPDATED + serviceName) +// .queryParam(RangerRESTUtils.REST_PARAM_LAST_KNOWN_POLICY_VERSION, Long.toString(lastKnownVersion)) +// .queryParam(RangerRESTUtils.REST_PARAM_LAST_ACTIVATION_TIME, Long.toString(lastActivationTimeInMillis)) +// .queryParam(RangerRESTUtils.REST_PARAM_PLUGIN_ID, pluginId) +// .queryParam(RangerRESTUtils.REST_PARAM_CLUSTER_NAME, clusterName); +// response = webResource.accept(RangerRESTUtils.REST_MIME_TYPE_JSON).get(ClientResponse.class); +// } + + if (LOG.isDebugEnabled()) { LOG.debug("Checking Service policy if updated with old api call"); } WebResource webResource = createWebResource(RangerRESTUtils.REST_URL_POLICY_GET_FOR_SERVICE_IF_UPDATED + serviceName) @@ -129,7 +140,6 @@ public ClientResponse run() { .queryParam(RangerRESTUtils.REST_PARAM_PLUGIN_ID, pluginId) .queryParam(RangerRESTUtils.REST_PARAM_CLUSTER_NAME, clusterName); response = webResource.accept(RangerRESTUtils.REST_MIME_TYPE_JSON).get(ClientResponse.class); - } if (response == null || response.getStatus() == HttpServletResponse.SC_NOT_MODIFIED) { if (response == null) { From 0b334bbed7c343e8dd39137d3a0c0139b7adaf96 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 2 Apr 2019 15:45:40 +0800 Subject: [PATCH 14/35] row filter provides anlyzed --- .../authorizer/RangerSparkSQLExtension.scala | 5 ++-- .../sql/catalyst/optimizer/Authorizable.scala | 8 +------ ...a => RangerSparkAuthorizerExtension.scala} | 3 ++- ...la => RangerSparkRowFilterExtension.scala} | 24 +++++++++---------- 4 files changed, 17 insertions(+), 23 deletions(-) rename plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/{AuthorizerExtension.scala => RangerSparkAuthorizerExtension.scala} (91%) rename plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/{RangerSparkRowFilter.scala => RangerSparkRowFilterExtension.scala} (90%) diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala index f8988083ea..4045850b0f 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala @@ -18,10 +18,11 @@ package org.apache.ranger.authorization.spark.authorizer import org.apache.spark.sql.SparkSessionExtensions -import org.apache.spark.sql.catalyst.optimizer.AuthorizerExtension +import org.apache.spark.sql.catalyst.optimizer.{RangerSparkAuthorizerExtension, RangerSparkRowFilterExtension} class RangerSparkSQLExtension extends Extensions { override def apply(ext: SparkSessionExtensions): Unit = { - ext.injectOptimizerRule(AuthorizerExtension) + ext.injectOptimizerRule(RangerSparkAuthorizerExtension) + ext.injectOptimizerRule(RangerSparkRowFilterExtension) } } diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala index 189b6cf168..80c1dd4fc7 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.ql.plan.HiveOperation import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveAuthzContext, HiveOperationType} import org.apache.spark.sql.SparkSession @@ -31,8 +30,6 @@ import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand trait Authorizable extends Rule[LogicalPlan] { - private val logger = LogFactory.getLog(classOf[Authorizable]) - def spark: SparkSession /** @@ -53,10 +50,7 @@ trait Authorizable extends Rule[LogicalPlan] { RangerSparkAuthzImpl.checkPrivileges(spark, operationType, in, out, authzContext) case _ => } - - // Row level filtering - new RangerSparkRowFilter(spark).build(plan) - // TODO(Kent Yao) applying column masking + plan } /** diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/AuthorizerExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkAuthorizerExtension.scala similarity index 91% rename from plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/AuthorizerExtension.scala rename to plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkAuthorizerExtension.scala index 8ef8a103c6..0563ea3579 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/AuthorizerExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkAuthorizerExtension.scala @@ -26,4 +26,5 @@ import org.apache.spark.sql.catalyst.rules.Rule * * For Apache Spark 2.2.x and later */ -case class AuthorizerExtension(spark: SparkSession) extends Rule[LogicalPlan] with Authorizable +case class RangerSparkAuthorizerExtension(spark: SparkSession) + extends Rule[LogicalPlan] with Authorizable diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilter.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala similarity index 90% rename from plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilter.scala rename to plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala index 808f0ec0a3..732664d427 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilter.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala @@ -17,21 +17,20 @@ package org.apache.spark.sql.catalyst.optimizer -import scala.collection.JavaConverters._ - import org.apache.commons.lang.StringUtils import org.apache.hadoop.security.UserGroupInformation import org.apache.ranger.authorization.spark.authorizer._ import org.apache.ranger.plugin.policyengine.RangerAccessResult +import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.sql.AuthzUtils.getFieldVal import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.LogicalRelation -import org.apache.spark.sql.{Dataset, SparkSession} -class RangerSparkRowFilter(spark: SparkSession) { +import scala.collection.JavaConverters._ - import RangerSparkRowFilter._ +case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[LogicalPlan] { private lazy val sparkPlugin = RangerSparkPlugin.build().getOrCreate() @@ -64,12 +63,16 @@ class RangerSparkRowFilter(spark: SparkSession) { } } + private def isRowFilterEnabled(result: RangerAccessResult): Boolean = { + result != null && result.isRowFilterEnabled && StringUtils.isNotEmpty(result.getFilterExpr) + } + /** * Transform a spark logical plan to another plan with the row filer expressions * @param plan the original [[LogicalPlan]] * @return the logical plan with row filer expressions applied */ - def build(plan: LogicalPlan): LogicalPlan = { + override def apply(plan: LogicalPlan): LogicalPlan = { var newPlan = plan newPlan = plan transform { case h if h.nodeName == "HiveTableRelation" => @@ -81,12 +84,7 @@ class RangerSparkRowFilter(spark: SparkSession) { case l: LogicalRelation if l.catalogTable.isDefined => applyingRowFilterExpr(l, l.catalogTable.get) } - Dataset.ofRows(spark, newPlan).queryExecution.optimizedPlan - } -} - -object RangerSparkRowFilter { - def isRowFilterEnabled(result: RangerAccessResult): Boolean = { - result != null && result.isRowFilterEnabled && StringUtils.isNotEmpty(result.getFilterExpr) + newPlan.withNewChildren() + Dataset.ofRows(spark, newPlan).queryExecution.analyzed } } From 87de022d794ca7e427f7dfde983877561f5f085c Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 2 Apr 2019 15:48:03 +0800 Subject: [PATCH 15/35] typo --- .../sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala index 732664d427..f1604c7ab9 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala @@ -84,7 +84,6 @@ case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[Logic case l: LogicalRelation if l.catalogTable.isDefined => applyingRowFilterExpr(l, l.catalogTable.get) } - newPlan.withNewChildren() Dataset.ofRows(spark, newPlan).queryExecution.analyzed } } From 57640f0ff8bbefb7591e86ab05354c37df268ac0 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 2 Apr 2019 17:29:33 +0800 Subject: [PATCH 16/35] fix stack over flow --- .../authorizer/RangerSparkSQLExtension.scala | 2 ++ .../RangerSparkRowFilterExtension.scala | 32 +++++++++++-------- .../plans/logical/RangerSparkRowFilter.scala | 28 ++++++++++++++++ .../RangerSparkRowFilterStrategy.scala | 28 ++++++++++++++++ 4 files changed, 76 insertions(+), 14 deletions(-) create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/RangerSparkRowFilter.scala create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerSparkRowFilterStrategy.scala diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala index 4045850b0f..535322bd78 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala @@ -19,10 +19,12 @@ package org.apache.ranger.authorization.spark.authorizer import org.apache.spark.sql.SparkSessionExtensions import org.apache.spark.sql.catalyst.optimizer.{RangerSparkAuthorizerExtension, RangerSparkRowFilterExtension} +import org.apache.spark.sql.execution.RangerSparkRowFilterStrategy class RangerSparkSQLExtension extends Extensions { override def apply(ext: SparkSessionExtensions): Unit = { ext.injectOptimizerRule(RangerSparkAuthorizerExtension) ext.injectOptimizerRule(RangerSparkRowFilterExtension) + ext.injectPlannerStrategy(RangerSparkRowFilterStrategy) } } diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala index f1604c7ab9..b78573e2f4 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala @@ -21,10 +21,10 @@ import org.apache.commons.lang.StringUtils import org.apache.hadoop.security.UserGroupInformation import org.apache.ranger.authorization.spark.authorizer._ import org.apache.ranger.plugin.policyengine.RangerAccessResult -import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.sql.AuthzUtils.getFieldVal +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, RangerSparkRowFilter} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.LogicalRelation @@ -54,7 +54,7 @@ case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[Logic if (isRowFilterEnabled(result)) { val sql = s"select ${plan.output.map(_.name).mkString(",")} from ${table.qualifiedName}" + s" where ${result.getFilterExpr}" - spark.sessionState.sqlParser.parsePlan(sql) + RangerSparkRowFilter(spark.sessionState.sqlParser.parsePlan(sql)) } else { plan } @@ -73,17 +73,21 @@ case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[Logic * @return the logical plan with row filer expressions applied */ override def apply(plan: LogicalPlan): LogicalPlan = { - var newPlan = plan - newPlan = plan transform { - case h if h.nodeName == "HiveTableRelation" => - val ct = getFieldVal(h, "tableMeta").asInstanceOf[CatalogTable] - applyingRowFilterExpr(h, ct) - case m if m.nodeName == "MetastoreRelation" => - val ct = getFieldVal(m, "catalogTable").asInstanceOf[CatalogTable] - applyingRowFilterExpr(m, ct) - case l: LogicalRelation if l.catalogTable.isDefined => - applyingRowFilterExpr(l, l.catalogTable.get) + if (plan.find(_.isInstanceOf[RangerSparkRowFilter]).nonEmpty) { + plan + } else { + val lp = plan transform { + case h if h.nodeName == "HiveTableRelation" => + val ct = getFieldVal(h, "tableMeta").asInstanceOf[CatalogTable] + applyingRowFilterExpr(h, ct) + case m if m.nodeName == "MetastoreRelation" => + val ct = getFieldVal(m, "catalogTable").asInstanceOf[CatalogTable] + applyingRowFilterExpr(m, ct) + case l: LogicalRelation if l.catalogTable.isDefined => + applyingRowFilterExpr(l, l.catalogTable.get) + } + val analyzed = spark.sessionState.analyzer.execute(lp) + spark.sessionState.optimizer.execute(analyzed) } - Dataset.ofRows(spark, newPlan).queryExecution.analyzed } } diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/RangerSparkRowFilter.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/RangerSparkRowFilter.scala new file mode 100644 index 0000000000..bbc9324e4c --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/RangerSparkRowFilter.scala @@ -0,0 +1,28 @@ +/* + * 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.catalyst.plans.logical +import org.apache.spark.sql.catalyst.expressions.Attribute + +/** + * A wrapper for a transformed plan with row level filter applied, which will be removed during + * LogicalPlan -> PhysicalPlan + * + */ +case class RangerSparkRowFilter(child: LogicalPlan) extends UnaryNode { + override def output: Seq[Attribute] = child.output +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerSparkRowFilterStrategy.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerSparkRowFilterStrategy.scala new file mode 100644 index 0000000000..1b5df4aedf --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerSparkRowFilterStrategy.scala @@ -0,0 +1,28 @@ +/* + * 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 + +import org.apache.spark.sql.{SparkSession, Strategy} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, RangerSparkRowFilter} + +case class RangerSparkRowFilterStrategy(spark: SparkSession) extends Strategy { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case RangerSparkRowFilter(child) => planLater(child) :: Nil + case _ => Nil + } +} From 6e50e9d763dc186543a6a85aefa73ab4f81dbfdd Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 2 Apr 2019 17:45:18 +0800 Subject: [PATCH 17/35] not opt --- .../sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala index b78573e2f4..ecbc1c8b25 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala @@ -86,8 +86,7 @@ case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[Logic case l: LogicalRelation if l.catalogTable.isDefined => applyingRowFilterExpr(l, l.catalogTable.get) } - val analyzed = spark.sessionState.analyzer.execute(lp) - spark.sessionState.optimizer.execute(analyzed) + spark.sessionState.analyzer.execute(lp) } } } From f5a799a417d41a47b5c5e425dc1d9f6a11b76a9a Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 4 Apr 2019 10:22:22 +0800 Subject: [PATCH 18/35] spark source for tbl an col --- .../spark/authorizer/RangerSparkResource.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala index 1dd4c713f4..5c2f9036c8 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala @@ -42,13 +42,13 @@ class RangerSparkResource( setValue(KEY_DATABASE, databaseOrUrl.getOrElse("")) setValue(KEY_UDF, tableOrUdf) case COLUMN => - setValue(KEY_DATABASE, databaseOrUrl) + setValue(KEY_DATABASE, databaseOrUrl.getOrElse("")) setValue(KEY_TABLE, tableOrUdf) setValue(KEY_COLUMN, column) case TABLE | VIEW | PARTITION => - setValue(KEY_DATABASE, databaseOrUrl) + setValue(KEY_DATABASE, databaseOrUrl.getOrElse("")) setValue(KEY_TABLE, tableOrUdf) - case URI => setValue(KEY_URL, databaseOrUrl) + case URI => setValue(KEY_URL, databaseOrUrl.getOrElse("*")) case _ => } From 570faad090a48e3044c0dedc2cefe42009e12c1d Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 4 Apr 2019 10:41:58 +0800 Subject: [PATCH 19/35] typo --- .../authorization/spark/authorizer/RangerSparkResource.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala index 5c2f9036c8..b366d53f57 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala @@ -42,11 +42,11 @@ class RangerSparkResource( setValue(KEY_DATABASE, databaseOrUrl.getOrElse("")) setValue(KEY_UDF, tableOrUdf) case COLUMN => - setValue(KEY_DATABASE, databaseOrUrl.getOrElse("")) + setValue(KEY_DATABASE, databaseOrUrl.getOrElse("*")) setValue(KEY_TABLE, tableOrUdf) setValue(KEY_COLUMN, column) case TABLE | VIEW | PARTITION => - setValue(KEY_DATABASE, databaseOrUrl.getOrElse("")) + setValue(KEY_DATABASE, databaseOrUrl.getOrElse("*")) setValue(KEY_TABLE, tableOrUdf) case URI => setValue(KEY_URL, databaseOrUrl.getOrElse("*")) case _ => From b6232b683d7a00c78e3ca8f257e73480177200ff Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 4 Apr 2019 11:50:22 +0800 Subject: [PATCH 20/35] typo --- .../spark/authorizer/RangerSparkResource.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala index b366d53f57..f633555d1e 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala @@ -42,13 +42,13 @@ class RangerSparkResource( setValue(KEY_DATABASE, databaseOrUrl.getOrElse("")) setValue(KEY_UDF, tableOrUdf) case COLUMN => - setValue(KEY_DATABASE, databaseOrUrl.getOrElse("*")) + setValue(KEY_DATABASE, databaseOrUrl.getOrElse("")) setValue(KEY_TABLE, tableOrUdf) setValue(KEY_COLUMN, column) case TABLE | VIEW | PARTITION => - setValue(KEY_DATABASE, databaseOrUrl.getOrElse("*")) + setValue(KEY_DATABASE, databaseOrUrl.getOrElse("")) setValue(KEY_TABLE, tableOrUdf) - case URI => setValue(KEY_URL, databaseOrUrl.getOrElse("*")) + case URI => setValue(KEY_URL, databaseOrUrl.getOrElse("")) case _ => } From 39fec99812eff9cafda4dba5b44c896bbc61a46c Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 4 Apr 2019 12:02:46 +0800 Subject: [PATCH 21/35] wrong condition check pri --- .../authorization/spark/authorizer/RangerSparkAuthorizer.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala index de83a61fa3..6cc86f4de2 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala @@ -131,7 +131,7 @@ class RangerSparkAuthorizer( } } else { val result = sparkPlugin.isAccessAllowed(request, auditHandler) - if (result != null && result.getIsAllowed) { + if (result != null && !result.getIsAllowed) { throw new HiveAccessControlException(s"Permission denied: user [$user] does not" + s" have [${request.getSparkAccessType}] privilege on [${resource.getAsString}]") } From 3812f5f26c443bf57146ffdb9afc5b0ace523d01 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 4 Apr 2019 12:12:24 +0800 Subject: [PATCH 22/35] wrong condition check pri --- .../spark/authorizer/RangerSparkResource.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala index f633555d1e..b366d53f57 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala @@ -42,13 +42,13 @@ class RangerSparkResource( setValue(KEY_DATABASE, databaseOrUrl.getOrElse("")) setValue(KEY_UDF, tableOrUdf) case COLUMN => - setValue(KEY_DATABASE, databaseOrUrl.getOrElse("")) + setValue(KEY_DATABASE, databaseOrUrl.getOrElse("*")) setValue(KEY_TABLE, tableOrUdf) setValue(KEY_COLUMN, column) case TABLE | VIEW | PARTITION => - setValue(KEY_DATABASE, databaseOrUrl.getOrElse("")) + setValue(KEY_DATABASE, databaseOrUrl.getOrElse("*")) setValue(KEY_TABLE, tableOrUdf) - case URI => setValue(KEY_URL, databaseOrUrl.getOrElse("")) + case URI => setValue(KEY_URL, databaseOrUrl.getOrElse("*")) case _ => } From 7edaf9212841f6d4799c6561b4c9fc497d135305 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 4 Apr 2019 13:26:14 +0800 Subject: [PATCH 23/35] add a ranger spark optimizer to do basic logial plan optimization --- .../optimizer/RangerSparkOptimizer.scala | 34 +++++++++++++++++++ .../RangerSparkRowFilterExtension.scala | 5 ++- 2 files changed, 38 insertions(+), 1 deletion(-) create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkOptimizer.scala diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkOptimizer.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkOptimizer.scala new file mode 100644 index 0000000000..0ddb67aeb2 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkOptimizer.scala @@ -0,0 +1,34 @@ +/* + * 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.catalyst.optimizer + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.RuleExecutor + +class RangerSparkOptimizer(spark: SparkSession) extends RuleExecutor[LogicalPlan] { + + override def batches: Seq[Batch] = { + val optimizer = spark.sessionState.optimizer + val extRules = optimizer.extendedOperatorOptimizationRules + optimizer.batches.map { batch => + val ruleSet = batch.rules.toSet -- extRules + Batch(batch.name, FixedPoint(batch.strategy.maxIterations), ruleSet.toSeq: _*) + } + } +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala index ecbc1c8b25..6041764d79 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala @@ -34,6 +34,8 @@ case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[Logic private lazy val sparkPlugin = RangerSparkPlugin.build().getOrCreate() + private lazy val rangerSparkOptimizer = new RangerSparkOptimizer(spark) + /** * Transform a Relation to a parsed [[LogicalPlan]] with specified row filter expressions * @param plan the original [[LogicalPlan]] @@ -86,7 +88,8 @@ case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[Logic case l: LogicalRelation if l.catalogTable.isDefined => applyingRowFilterExpr(l, l.catalogTable.get) } - spark.sessionState.analyzer.execute(lp) + val analyzed = spark.sessionState.analyzer.execute(lp) + rangerSparkOptimizer.execute(analyzed) } } } From 273c3b8ae1d3b94b1710f6e46cbf90e5bfe5fe93 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 9 Apr 2019 16:17:12 +0800 Subject: [PATCH 24/35] add data masking, fix where clasue issue --- plugin-spark/pom.xml | 22 +++ .../authorizer/RangerSparkSQLExtension.scala | 7 +- .../RangerSparkMaskingExtension.scala | 143 ++++++++++++++++++ .../optimizer/RangerSparkOptimizer.scala | 3 + .../RangerSparkRowFilterExtension.scala | 47 ++++-- .../plans/logical/RangerSparkMasking.scala | 27 ++++ ...cala => RangerSparkPlanOmitStrategy.scala} | 9 +- 7 files changed, 237 insertions(+), 21 deletions(-) create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/RangerSparkMasking.scala rename plugin-spark/src/main/scala/org/apache/spark/sql/execution/{RangerSparkRowFilterStrategy.scala => RangerSparkPlanOmitStrategy.scala} (78%) diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml index 1b5c63b3b9..11a2659ae7 100644 --- a/plugin-spark/pom.xml +++ b/plugin-spark/pom.xml @@ -89,6 +89,17 @@ + + org.apache.hive + hive-exec + 2.3.2 + + + * + * + + + org.eclipse.persistence eclipselink @@ -283,9 +294,20 @@ org.codehaus.jackson:jackson-xc org.eclipse.persistence:eclipselink org.eclipse.persistence:javax.persistence + org.apache.hive:hive-exec org.noggit:noggit + + + org.apache.hive:hive-exec + + + org/apache/hadoop/hive/ql/udf/generic/**Mask** + org/apache/hadoop/hive/ql/udf/generic/**Transform** + + + com.sun.jersey diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala index 535322bd78..5d7b2d07fa 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkSQLExtension.scala @@ -18,13 +18,14 @@ package org.apache.ranger.authorization.spark.authorizer import org.apache.spark.sql.SparkSessionExtensions -import org.apache.spark.sql.catalyst.optimizer.{RangerSparkAuthorizerExtension, RangerSparkRowFilterExtension} -import org.apache.spark.sql.execution.RangerSparkRowFilterStrategy +import org.apache.spark.sql.catalyst.optimizer.{RangerSparkAuthorizerExtension, RangerSparkMaskingExtension, RangerSparkRowFilterExtension} +import org.apache.spark.sql.execution.RangerSparkPlanOmitStrategy class RangerSparkSQLExtension extends Extensions { override def apply(ext: SparkSessionExtensions): Unit = { ext.injectOptimizerRule(RangerSparkAuthorizerExtension) ext.injectOptimizerRule(RangerSparkRowFilterExtension) - ext.injectPlannerStrategy(RangerSparkRowFilterStrategy) + ext.injectOptimizerRule(RangerSparkMaskingExtension) + ext.injectPlannerStrategy(RangerSparkPlanOmitStrategy) } } diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala new file mode 100644 index 0000000000..5c2b4a05a6 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala @@ -0,0 +1,143 @@ +/* + * 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.catalyst.optimizer + +import org.apache.commons.lang3.StringUtils +import org.apache.hadoop.security.UserGroupInformation +import org.apache.ranger.authorization.spark.authorizer._ +import org.apache.ranger.plugin.model.RangerPolicy +import org.apache.ranger.plugin.policyengine.RangerAccessResult +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RangerSparkMasking} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.AuthzUtils.getFieldVal +import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTable} +import org.apache.spark.sql.catalyst.expressions.{Attribute, NamedExpression} +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.execution.datasources.LogicalRelation + +import scala.collection.JavaConverters._ + +/** + * An Apache Spark's [[Optimizer]] extension for column data masking. + */ +case class RangerSparkMaskingExtension(spark: SparkSession) extends Rule[LogicalPlan] { + import RangerPolicy._ + + // register all built-in masking udfs + Map("mask" -> "org.apache.hadoop.hive.ql.udf.generic.GenericUDFMask", + "mask_first_n" -> "org.apache.hadoop.hive.ql.udf.generic.GenericUDFMaskFirstN", + "mask_hash" -> "org.apache.hadoop.hive.ql.udf.generic.GenericUDFMaskHash", + "mask_last_n" -> "org.apache.hadoop.hive.ql.udf.generic.GenericUDFMaskLastN", + "mask_show_first_n" -> "org.apache.hadoop.hive.ql.udf.generic.GenericUDFMaskShowFirstN", + "mask_show_last_n" -> "org.apache.hadoop.hive.ql.udf.generic.GenericUDFMaskShowLastN") + .map(x => CatalogFunction(FunctionIdentifier(x._1), x._2, Seq.empty)) + .foreach(spark.sessionState.catalog.registerFunction(_, ignoreIfExists = true)) + + private lazy val sparkPlugin = RangerSparkPlugin.build().getOrCreate() + + /** + * Collecting transformers from Ranger data masking policies, and mapping the to the + * [[LogicalPlan]] output attributes. + * + * @param plan the original logical plan with a underlying catalog table + * @param table the catalog table + * @return a list of key-value pairs of original expression with its masking representation + */ + private def collectTransformers( + plan: LogicalPlan, table: CatalogTable): Seq[(Attribute, NamedExpression)] = { + val auditHandler = new RangerSparkAuditHandler() + val ugi = UserGroupInformation.getCurrentUser + val userName = ugi.getShortUserName + val groups = ugi.getGroupNames.toSet.asJava + try { + val identifier = table.identifier + import SparkObjectType._ + + plan.output.map { expr => + val resource = RangerSparkResource(COLUMN, identifier.database, identifier.table, expr.name) + val req = new RangerSparkAccessRequest(resource, userName, groups, COLUMN.toString, + SparkAccessType.SELECT, null, null, sparkPlugin.getClusterName) + (expr, sparkPlugin.evalDataMaskPolicies(req, auditHandler)) + }.filter(x => isMaskEnabled(x._2)).map { x => + val expr = x._1 + val result = x._2 + if (StringUtils.equalsIgnoreCase(result.getMaskType, MASK_TYPE_NULL)) { + val sql = s"SELECT NULL AS ${expr.name} FROM ${table.qualifiedName}" + (expr, spark.sessionState.sqlParser.parsePlan(sql)) + } else if (StringUtils.equalsIgnoreCase(result.getMaskType, MASK_TYPE_CUSTOM)) { + val maskVal = result.getMaskedValue + if (maskVal == null) { + val sql = s"SELECT NULL AS ${expr.name} FROM ${table.qualifiedName}" + (expr, spark.sessionState.sqlParser.parsePlan(sql)) + } else { + val sql = s"SELECT ${maskVal.replace("{col}", expr.name)} AS ${expr.name} FROM" + + s" ${table.qualifiedName}" + (expr, spark.sessionState.sqlParser.parsePlan(sql)) + } + } else if (result.getMaskTypeDef != null) { + val transformer = result.getMaskTypeDef.getTransformer + if (StringUtils.isNotEmpty(transformer)) { + val trans = transformer.replace("{col}", expr.name) + val sql = s"SELECT $trans AS ${expr.name} FROM ${table.qualifiedName}" + (expr, spark.sessionState.sqlParser.parsePlan(sql)) + } else { + (expr, null) + } + } else { + (expr, null) + } + }.filter(_._2 != null).map { kv => + kv._2 match { + case p: Project => (kv._1, p.projectList.head) + case _ => (kv._1, kv._2.output.head) + } + } + } catch { + case e: Exception => throw e + } + } + + private def isMaskEnabled(result: RangerAccessResult): Boolean = { + result != null && result.isMaskEnabled + } + + override def apply(plan: LogicalPlan): LogicalPlan = plan match { + case m: RangerSparkMasking => m // escape the optimize iteration if already masked + case fixed if fixed.find(_.isInstanceOf[RangerSparkMasking]).nonEmpty => fixed + case _ => + val transformers = plan.collectLeaves().flatMap { + case h if h.nodeName == "HiveTableRelation" => + val ct = getFieldVal(h, "tableMeta").asInstanceOf[CatalogTable] + collectTransformers(h, ct) + case m if m.nodeName == "MetastoreRelation" => + val ct = getFieldVal(m, "catalogTable").asInstanceOf[CatalogTable] + collectTransformers(m, ct) + case l: LogicalRelation if l.catalogTable.isDefined => + collectTransformers(l, l.catalogTable.get) + case _ => Seq.empty + }.toMap + val newOutput = plan.output.map(attr => transformers.getOrElse(attr, attr)) + if (newOutput.isEmpty) { + RangerSparkMasking(plan) + } else { + val analyzed = spark.sessionState.analyzer.execute(Project(newOutput, plan)) + RangerSparkMasking(analyzed) + } + } +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkOptimizer.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkOptimizer.scala index 0ddb67aeb2..18a7e1a4b7 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkOptimizer.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkOptimizer.scala @@ -21,6 +21,9 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor +/** + * An Optimizer without all `spark.sql.extensions` + */ class RangerSparkOptimizer(spark: SparkSession) extends RuleExecutor[LogicalPlan] { override def batches: Seq[Batch] = { diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala index 6041764d79..69ba810a07 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala @@ -24,12 +24,16 @@ import org.apache.ranger.plugin.policyengine.RangerAccessResult import org.apache.spark.sql.AuthzUtils.getFieldVal import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, RangerSparkRowFilter} +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, RangerSparkRowFilter, Subquery} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.datasources.LogicalRelation import scala.collection.JavaConverters._ +/** + * An Apache Spark's [[Optimizer]] extension for row level filtering. + */ case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[LogicalPlan] { private lazy val sparkPlugin = RangerSparkPlugin.build().getOrCreate() @@ -56,9 +60,20 @@ case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[Logic if (isRowFilterEnabled(result)) { val sql = s"select ${plan.output.map(_.name).mkString(",")} from ${table.qualifiedName}" + s" where ${result.getFilterExpr}" - RangerSparkRowFilter(spark.sessionState.sqlParser.parsePlan(sql)) + val parsed = spark.sessionState.sqlParser.parsePlan(sql) + + val parsedNew = parsed transform { + case Filter(condition, child) if !child.fastEquals(plan) => Filter(condition, plan) + } + val analyzed = spark.sessionState.analyzer.execute(parsedNew) + val optimized = analyzed transformAllExpressions { + case s: SubqueryExpression => + val Subquery(newPlan) = rangerSparkOptimizer.execute(Subquery(s.plan)) + s.withNewPlan(newPlan) + } + RangerSparkRowFilter(optimized) } else { - plan + RangerSparkRowFilter(plan) } } catch { case e: Exception => throw e @@ -74,22 +89,22 @@ case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[Logic * @param plan the original [[LogicalPlan]] * @return the logical plan with row filer expressions applied */ - override def apply(plan: LogicalPlan): LogicalPlan = { - if (plan.find(_.isInstanceOf[RangerSparkRowFilter]).nonEmpty) { - plan - } else { - val lp = plan transform { + override def apply(plan: LogicalPlan): LogicalPlan = plan match { + case rf: RangerSparkRowFilter => rf + case fixed if fixed.find(_.isInstanceOf[RangerSparkRowFilter]).nonEmpty => fixed + case _ => + val plansWithTables = plan.collectLeaves().map { case h if h.nodeName == "HiveTableRelation" => - val ct = getFieldVal(h, "tableMeta").asInstanceOf[CatalogTable] - applyingRowFilterExpr(h, ct) + (h, getFieldVal(h, "tableMeta").asInstanceOf[CatalogTable]) case m if m.nodeName == "MetastoreRelation" => - val ct = getFieldVal(m, "catalogTable").asInstanceOf[CatalogTable] - applyingRowFilterExpr(m, ct) + (m, getFieldVal(m, "catalogTable").asInstanceOf[CatalogTable]) case l: LogicalRelation if l.catalogTable.isDefined => - applyingRowFilterExpr(l, l.catalogTable.get) + (l, l.catalogTable.get) + case _ => null + }.filter(_ != null).map(lt => (lt._1, applyingRowFilterExpr(lt._1, lt._2))).toMap + + plan transformUp { + case p => plansWithTables.getOrElse(p, p) } - val analyzed = spark.sessionState.analyzer.execute(lp) - rangerSparkOptimizer.execute(analyzed) - } } } diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/RangerSparkMasking.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/RangerSparkMasking.scala new file mode 100644 index 0000000000..8ec1e2bb2b --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/RangerSparkMasking.scala @@ -0,0 +1,27 @@ +/* + * 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.catalyst.plans.logical + +import org.apache.spark.sql.catalyst.expressions.Attribute + +/** + * A marker [[LogicalPlan]] for column data masking + */ +case class RangerSparkMasking(child: LogicalPlan) extends UnaryNode { + override def output: Seq[Attribute] = child.output +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerSparkRowFilterStrategy.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerSparkPlanOmitStrategy.scala similarity index 78% rename from plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerSparkRowFilterStrategy.scala rename to plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerSparkPlanOmitStrategy.scala index 1b5df4aedf..4483b09953 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerSparkRowFilterStrategy.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerSparkPlanOmitStrategy.scala @@ -18,11 +18,16 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{SparkSession, Strategy} -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, RangerSparkRowFilter} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, RangerSparkMasking, RangerSparkRowFilter} -case class RangerSparkRowFilterStrategy(spark: SparkSession) extends Strategy { +/** + * An Apache Spark's [[Strategy]] extension for omitting marker for row level filtering and data + * masking. + */ +case class RangerSparkPlanOmitStrategy(spark: SparkSession) extends Strategy { override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case RangerSparkRowFilter(child) => planLater(child) :: Nil + case RangerSparkMasking(child) => planLater(child) :: Nil case _ => Nil } } From 05d1787c1891b6a0bf54036fbd65fe68bd924399 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 11 Apr 2019 10:13:06 +0800 Subject: [PATCH 25/35] fix scala.MatchError --- .../catalyst/optimizer/RangerSparkMaskingExtension.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala index 5c2b4a05a6..62bc37abb2 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala @@ -23,7 +23,7 @@ import org.apache.ranger.authorization.spark.authorizer._ import org.apache.ranger.plugin.model.RangerPolicy import org.apache.ranger.plugin.policyengine.RangerAccessResult import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RangerSparkMasking} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RangerSparkMasking, Subquery} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.AuthzUtils.getFieldVal import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTable} @@ -134,7 +134,10 @@ case class RangerSparkMaskingExtension(spark: SparkSession) extends Rule[Logical }.toMap val newOutput = plan.output.map(attr => transformers.getOrElse(attr, attr)) if (newOutput.isEmpty) { - RangerSparkMasking(plan) + plan match { + case Subquery(child) => Subquery(RangerSparkMasking(child)) + case _ => RangerSparkMasking(plan) + } } else { val analyzed = spark.sessionState.analyzer.execute(Project(newOutput, plan)) RangerSparkMasking(analyzed) From fd911a99bfd22e4e64d9638af38b984e6a00ef66 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 11 Apr 2019 10:53:27 +0800 Subject: [PATCH 26/35] fix scala.MatchError 2 --- .../optimizer/RangerSparkMaskingExtension.scala | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala index 62bc37abb2..23c07abf72 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala @@ -133,14 +133,10 @@ case class RangerSparkMaskingExtension(spark: SparkSession) extends Rule[Logical case _ => Seq.empty }.toMap val newOutput = plan.output.map(attr => transformers.getOrElse(attr, attr)) - if (newOutput.isEmpty) { - plan match { - case Subquery(child) => Subquery(RangerSparkMasking(child)) - case _ => RangerSparkMasking(plan) - } - } else { - val analyzed = spark.sessionState.analyzer.execute(Project(newOutput, plan)) - RangerSparkMasking(analyzed) + val masked = plan match { + case Subquery(child) => Subquery(Project(newOutput, RangerSparkMasking(child))) + case _ => Project(newOutput, RangerSparkMasking(plan)) } + spark.sessionState.analyzer.execute(masked) } } From 4d9c2ddf6bf6195c8d2e62c955b5e4a0e5db5e97 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 11 Apr 2019 11:16:37 +0800 Subject: [PATCH 27/35] fix Unexpected operator in scalar subquery: RangerSparkMasking --- .../optimizer/RangerSparkMaskingExtension.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala index 23c07abf72..e42c1b7dab 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala @@ -23,7 +23,7 @@ import org.apache.ranger.authorization.spark.authorizer._ import org.apache.ranger.plugin.model.RangerPolicy import org.apache.ranger.plugin.policyengine.RangerAccessResult import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RangerSparkMasking, Subquery} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.AuthzUtils.getFieldVal import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTable} @@ -133,9 +133,13 @@ case class RangerSparkMaskingExtension(spark: SparkSession) extends Rule[Logical case _ => Seq.empty }.toMap val newOutput = plan.output.map(attr => transformers.getOrElse(attr, attr)) - val masked = plan match { - case Subquery(child) => Subquery(Project(newOutput, RangerSparkMasking(child))) - case _ => Project(newOutput, RangerSparkMasking(plan)) + val planWithMasking = plan match { + case Subquery(child) => Subquery(Project(newOutput, child)) + case _ => Project(newOutput, plan) + } + + val masked = planWithMasking transform { + case l: LeafNode => RangerSparkMasking(l) } spark.sessionState.analyzer.execute(masked) } From 3b755ec9414a110057416d31e501f9b0cfbb15b5 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 11 Apr 2019 14:52:56 +0800 Subject: [PATCH 28/35] handling ctas-like commands --- plugin-spark/pom.xml | 2 +- .../RangerSparkMaskingExtension.scala | 26 +++++++++++--- .../RangerSparkRowFilterExtension.scala | 34 ++++++++++++++----- 3 files changed, 49 insertions(+), 13 deletions(-) diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml index 11a2659ae7..db5428c070 100644 --- a/plugin-spark/pom.xml +++ b/plugin-spark/pom.xml @@ -26,7 +26,7 @@ 2.11.8 2.11 2.2.6 - 2.2.1 + 2.3.2 provided 1.2.1.spark2 ranger_spark_project diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala index e42c1b7dab..78c0d051da 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala @@ -29,7 +29,9 @@ import org.apache.spark.sql.AuthzUtils.getFieldVal import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTable} import org.apache.spark.sql.catalyst.expressions.{Attribute, NamedExpression} import org.apache.spark.sql.catalyst.FunctionIdentifier -import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, CreateViewCommand, InsertIntoDataSourceDirCommand} +import org.apache.spark.sql.execution.datasources.{InsertIntoDataSourceCommand, InsertIntoHadoopFsRelationCommand, LogicalRelation, SaveIntoDataSourceCommand} +import org.apache.spark.sql.hive.execution.{CreateHiveTableAsSelectCommand, InsertIntoHiveDirCommand, InsertIntoHiveTable} import scala.collection.JavaConverters._ @@ -47,7 +49,7 @@ case class RangerSparkMaskingExtension(spark: SparkSession) extends Rule[Logical "mask_show_first_n" -> "org.apache.hadoop.hive.ql.udf.generic.GenericUDFMaskShowFirstN", "mask_show_last_n" -> "org.apache.hadoop.hive.ql.udf.generic.GenericUDFMaskShowLastN") .map(x => CatalogFunction(FunctionIdentifier(x._1), x._2, Seq.empty)) - .foreach(spark.sessionState.catalog.registerFunction(_, ignoreIfExists = true)) + .foreach(spark.sessionState.catalog.registerFunction(_, true)) private lazy val sparkPlugin = RangerSparkPlugin.build().getOrCreate() @@ -117,7 +119,7 @@ case class RangerSparkMaskingExtension(spark: SparkSession) extends Rule[Logical result != null && result.isMaskEnabled } - override def apply(plan: LogicalPlan): LogicalPlan = plan match { + private def doMasking(plan: LogicalPlan): LogicalPlan = plan match { case m: RangerSparkMasking => m // escape the optimize iteration if already masked case fixed if fixed.find(_.isInstanceOf[RangerSparkMasking]).nonEmpty => fixed case _ => @@ -138,9 +140,25 @@ case class RangerSparkMaskingExtension(spark: SparkSession) extends Rule[Logical case _ => Project(newOutput, plan) } - val masked = planWithMasking transform { + val masked = planWithMasking transformUp { case l: LeafNode => RangerSparkMasking(l) } spark.sessionState.analyzer.execute(masked) } + + override def apply(plan: LogicalPlan): LogicalPlan = plan match { + case c: Command => c match { + case c: CreateDataSourceTableAsSelectCommand => c.copy(query = doMasking(c.query)) + case c: CreateHiveTableAsSelectCommand => c.copy(query = doMasking(c.query)) + case c: CreateViewCommand => c.copy(child = doMasking(c.child)) + case i: InsertIntoDataSourceCommand => i.copy(query = doMasking(i.query)) + case i: InsertIntoDataSourceDirCommand => i.copy(query = doMasking(i.query)) + case i: InsertIntoHadoopFsRelationCommand => i.copy(query = doMasking(i.query)) + case i: InsertIntoHiveDirCommand => i.copy(query = doMasking(i.query)) + case i: InsertIntoHiveTable => i.copy(query = doMasking(i.query)) + case s: SaveIntoDataSourceCommand => s.copy(query = doMasking(s.query)) + case cmd => cmd + } + case other => doMasking(other) + } } diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala index 69ba810a07..fa2befeddf 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala @@ -25,9 +25,11 @@ import org.apache.spark.sql.AuthzUtils.getFieldVal import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.SubqueryExpression -import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, RangerSparkRowFilter, Subquery} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, CreateViewCommand, InsertIntoDataSourceDirCommand} +import org.apache.spark.sql.execution.datasources.{InsertIntoDataSourceCommand, InsertIntoHadoopFsRelationCommand, LogicalRelation, SaveIntoDataSourceCommand} +import org.apache.spark.sql.hive.execution.{CreateHiveTableAsSelectCommand, InsertIntoHiveDirCommand, InsertIntoHiveTable} import scala.collection.JavaConverters._ @@ -84,12 +86,7 @@ case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[Logic result != null && result.isRowFilterEnabled && StringUtils.isNotEmpty(result.getFilterExpr) } - /** - * Transform a spark logical plan to another plan with the row filer expressions - * @param plan the original [[LogicalPlan]] - * @return the logical plan with row filer expressions applied - */ - override def apply(plan: LogicalPlan): LogicalPlan = plan match { + private def doFiltering(plan: LogicalPlan): LogicalPlan = plan match { case rf: RangerSparkRowFilter => rf case fixed if fixed.find(_.isInstanceOf[RangerSparkRowFilter]).nonEmpty => fixed case _ => @@ -107,4 +104,25 @@ case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[Logic case p => plansWithTables.getOrElse(p, p) } } + + /** + * Transform a spark logical plan to another plan with the row filer expressions + * @param plan the original [[LogicalPlan]] + * @return the logical plan with row filer expressions applied + */ + override def apply(plan: LogicalPlan): LogicalPlan = plan match { + case c: Command => c match { + case c: CreateDataSourceTableAsSelectCommand => c.copy(query = doFiltering(c.query)) + case c: CreateHiveTableAsSelectCommand => c.copy(query = doFiltering(c.query)) + case c: CreateViewCommand => c.copy(child = doFiltering(c.child)) + case i: InsertIntoDataSourceCommand => i.copy(query = doFiltering(i.query)) + case i: InsertIntoDataSourceDirCommand => i.copy(query = doFiltering(i.query)) + case i: InsertIntoHadoopFsRelationCommand => i.copy(query = doFiltering(i.query)) + case i: InsertIntoHiveDirCommand => i.copy(query = doFiltering(i.query)) + case i: InsertIntoHiveTable => i.copy(query = doFiltering(i.query)) + case s: SaveIntoDataSourceCommand => s.copy(query = doFiltering(s.query)) + case cmd => cmd + } + case other => doFiltering(other) + } } From e5a2ce409d3d194259d632214d53e4e85fe9964d Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sun, 28 Apr 2019 22:52:51 +0800 Subject: [PATCH 29/35] add ut for all tpcds queries --- plugin-spark/.gitignore | 3 +- plugin-spark/pom.xml | 50 +- .../authorizer/RangerSparkAuthorizer.scala | 2 +- .../spark/authorizer/RangerSparkPlugin.scala | 16 +- .../RangerSparkMaskingExtension.scala | 130 +- .../RangerSparkRowFilterExtension.scala | 5 +- .../src/test/resources/data/files/kv1.txt | 500 +++++ .../src/test/resources/log4j.properties | 23 + .../test/resources/ranger-spark-security.xml | 45 + .../test/resources/sparkSql_hive_jenkins.json | 1897 +++++++++++++++++ plugin-spark/src/test/resources/tpcds/q1.sql | 19 + plugin-spark/src/test/resources/tpcds/q10.sql | 57 + plugin-spark/src/test/resources/tpcds/q11.sql | 68 + plugin-spark/src/test/resources/tpcds/q12.sql | 22 + plugin-spark/src/test/resources/tpcds/q13.sql | 49 + .../src/test/resources/tpcds/q14a.sql | 120 ++ .../src/test/resources/tpcds/q14b.sql | 95 + plugin-spark/src/test/resources/tpcds/q15.sql | 15 + plugin-spark/src/test/resources/tpcds/q16.sql | 23 + plugin-spark/src/test/resources/tpcds/q17.sql | 33 + plugin-spark/src/test/resources/tpcds/q18.sql | 28 + plugin-spark/src/test/resources/tpcds/q19.sql | 19 + plugin-spark/src/test/resources/tpcds/q2.sql | 81 + plugin-spark/src/test/resources/tpcds/q20.sql | 18 + plugin-spark/src/test/resources/tpcds/q21.sql | 25 + plugin-spark/src/test/resources/tpcds/q22.sql | 14 + .../src/test/resources/tpcds/q23a.sql | 53 + .../src/test/resources/tpcds/q23b.sql | 68 + .../src/test/resources/tpcds/q24a.sql | 34 + .../src/test/resources/tpcds/q24b.sql | 34 + plugin-spark/src/test/resources/tpcds/q25.sql | 33 + plugin-spark/src/test/resources/tpcds/q26.sql | 19 + plugin-spark/src/test/resources/tpcds/q27.sql | 21 + plugin-spark/src/test/resources/tpcds/q28.sql | 56 + plugin-spark/src/test/resources/tpcds/q29.sql | 32 + plugin-spark/src/test/resources/tpcds/q3.sql | 13 + plugin-spark/src/test/resources/tpcds/q30.sql | 35 + plugin-spark/src/test/resources/tpcds/q31.sql | 60 + plugin-spark/src/test/resources/tpcds/q32.sql | 15 + plugin-spark/src/test/resources/tpcds/q33.sql | 65 + plugin-spark/src/test/resources/tpcds/q34.sql | 32 + plugin-spark/src/test/resources/tpcds/q35.sql | 46 + plugin-spark/src/test/resources/tpcds/q36.sql | 26 + plugin-spark/src/test/resources/tpcds/q37.sql | 15 + plugin-spark/src/test/resources/tpcds/q38.sql | 30 + .../src/test/resources/tpcds/q39a.sql | 47 + .../src/test/resources/tpcds/q39b.sql | 48 + plugin-spark/src/test/resources/tpcds/q4.sql | 120 ++ plugin-spark/src/test/resources/tpcds/q40.sql | 25 + plugin-spark/src/test/resources/tpcds/q41.sql | 49 + plugin-spark/src/test/resources/tpcds/q42.sql | 18 + plugin-spark/src/test/resources/tpcds/q43.sql | 33 + plugin-spark/src/test/resources/tpcds/q44.sql | 46 + plugin-spark/src/test/resources/tpcds/q45.sql | 21 + plugin-spark/src/test/resources/tpcds/q46.sql | 32 + plugin-spark/src/test/resources/tpcds/q47.sql | 63 + plugin-spark/src/test/resources/tpcds/q48.sql | 63 + plugin-spark/src/test/resources/tpcds/q49.sql | 126 ++ plugin-spark/src/test/resources/tpcds/q5.sql | 131 ++ plugin-spark/src/test/resources/tpcds/q50.sql | 47 + plugin-spark/src/test/resources/tpcds/q51.sql | 55 + plugin-spark/src/test/resources/tpcds/q52.sql | 14 + plugin-spark/src/test/resources/tpcds/q53.sql | 30 + plugin-spark/src/test/resources/tpcds/q54.sql | 61 + plugin-spark/src/test/resources/tpcds/q55.sql | 13 + plugin-spark/src/test/resources/tpcds/q56.sql | 65 + plugin-spark/src/test/resources/tpcds/q57.sql | 56 + plugin-spark/src/test/resources/tpcds/q58.sql | 59 + plugin-spark/src/test/resources/tpcds/q59.sql | 75 + plugin-spark/src/test/resources/tpcds/q6.sql | 21 + plugin-spark/src/test/resources/tpcds/q60.sql | 62 + plugin-spark/src/test/resources/tpcds/q61.sql | 33 + plugin-spark/src/test/resources/tpcds/q62.sql | 35 + plugin-spark/src/test/resources/tpcds/q63.sql | 31 + plugin-spark/src/test/resources/tpcds/q64.sql | 92 + plugin-spark/src/test/resources/tpcds/q65.sql | 33 + plugin-spark/src/test/resources/tpcds/q66.sql | 240 +++ plugin-spark/src/test/resources/tpcds/q67.sql | 38 + plugin-spark/src/test/resources/tpcds/q68.sql | 34 + plugin-spark/src/test/resources/tpcds/q69.sql | 38 + plugin-spark/src/test/resources/tpcds/q7.sql | 19 + plugin-spark/src/test/resources/tpcds/q70.sql | 38 + plugin-spark/src/test/resources/tpcds/q71.sql | 44 + plugin-spark/src/test/resources/tpcds/q72.sql | 33 + plugin-spark/src/test/resources/tpcds/q73.sql | 30 + plugin-spark/src/test/resources/tpcds/q74.sql | 58 + plugin-spark/src/test/resources/tpcds/q75.sql | 76 + plugin-spark/src/test/resources/tpcds/q76.sql | 47 + plugin-spark/src/test/resources/tpcds/q77.sql | 100 + plugin-spark/src/test/resources/tpcds/q78.sql | 64 + plugin-spark/src/test/resources/tpcds/q79.sql | 27 + plugin-spark/src/test/resources/tpcds/q8.sql | 87 + plugin-spark/src/test/resources/tpcds/q80.sql | 94 + plugin-spark/src/test/resources/tpcds/q81.sql | 38 + plugin-spark/src/test/resources/tpcds/q82.sql | 15 + plugin-spark/src/test/resources/tpcds/q83.sql | 56 + plugin-spark/src/test/resources/tpcds/q84.sql | 19 + plugin-spark/src/test/resources/tpcds/q85.sql | 82 + plugin-spark/src/test/resources/tpcds/q86.sql | 24 + plugin-spark/src/test/resources/tpcds/q87.sql | 28 + plugin-spark/src/test/resources/tpcds/q88.sql | 122 ++ plugin-spark/src/test/resources/tpcds/q89.sql | 30 + plugin-spark/src/test/resources/tpcds/q9.sql | 48 + plugin-spark/src/test/resources/tpcds/q90.sql | 19 + plugin-spark/src/test/resources/tpcds/q91.sql | 23 + plugin-spark/src/test/resources/tpcds/q92.sql | 16 + plugin-spark/src/test/resources/tpcds/q93.sql | 19 + plugin-spark/src/test/resources/tpcds/q94.sql | 23 + plugin-spark/src/test/resources/tpcds/q95.sql | 29 + plugin-spark/src/test/resources/tpcds/q96.sql | 11 + plugin-spark/src/test/resources/tpcds/q97.sql | 30 + plugin-spark/src/test/resources/tpcds/q98.sql | 21 + plugin-spark/src/test/resources/tpcds/q99.sql | 34 + .../spark/RangerAdminClientImpl.scala | 56 + .../spark/SparkRangerAuthorizerTest.scala | 477 +++++ .../spark/sql/RangerSparkTestUtils.scala | 40 + .../RangerSparkRowFilterExtensionTest.scala | 49 + .../RangerSparkPlanOmitStrategyTest.scala | 44 + 118 files changed, 8013 insertions(+), 55 deletions(-) create mode 100644 plugin-spark/src/test/resources/data/files/kv1.txt create mode 100644 plugin-spark/src/test/resources/log4j.properties create mode 100644 plugin-spark/src/test/resources/ranger-spark-security.xml create mode 100644 plugin-spark/src/test/resources/sparkSql_hive_jenkins.json create mode 100755 plugin-spark/src/test/resources/tpcds/q1.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q10.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q11.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q12.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q13.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q14a.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q14b.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q15.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q16.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q17.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q18.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q19.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q2.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q20.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q21.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q22.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q23a.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q23b.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q24a.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q24b.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q25.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q26.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q27.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q28.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q29.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q3.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q30.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q31.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q32.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q33.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q34.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q35.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q36.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q37.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q38.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q39a.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q39b.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q4.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q40.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q41.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q42.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q43.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q44.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q45.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q46.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q47.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q48.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q49.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q5.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q50.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q51.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q52.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q53.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q54.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q55.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q56.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q57.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q58.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q59.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q6.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q60.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q61.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q62.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q63.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q64.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q65.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q66.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q67.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q68.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q69.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q7.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q70.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q71.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q72.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q73.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q74.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q75.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q76.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q77.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q78.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q79.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q8.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q80.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q81.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q82.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q83.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q84.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q85.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q86.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q87.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q88.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q89.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q9.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q90.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q91.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q92.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q93.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q94.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q95.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q96.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q97.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q98.sql create mode 100755 plugin-spark/src/test/resources/tpcds/q99.sql create mode 100644 plugin-spark/src/test/scala/org/apache/ranger/services/spark/RangerAdminClientImpl.scala create mode 100644 plugin-spark/src/test/scala/org/apache/ranger/services/spark/SparkRangerAuthorizerTest.scala create mode 100644 plugin-spark/src/test/scala/org/apache/spark/sql/RangerSparkTestUtils.scala create mode 100644 plugin-spark/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtensionTest.scala create mode 100644 plugin-spark/src/test/scala/org/apache/spark/sql/execution/RangerSparkPlanOmitStrategyTest.scala diff --git a/plugin-spark/.gitignore b/plugin-spark/.gitignore index 3db6e9adbf..eebb5db03c 100644 --- a/plugin-spark/.gitignore +++ b/plugin-spark/.gitignore @@ -1,2 +1,3 @@ target/ -dependency-reduced-pom.xml \ No newline at end of file +dependency-reduced-pom.xml +/derby.log diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml index db5428c070..d16225d2bf 100644 --- a/plugin-spark/pom.xml +++ b/plugin-spark/pom.xml @@ -215,12 +215,29 @@ - org.mockito - mockito-core + org.apache.spark + spark-hive_${scala.binary.version} + ${spark.version} + test-jar + + + + com.kstruct + gethostname4j + 0.0.2 + test + + target/classes + target/test-classes + + + ${project.basedir}/src/test/resources + + net.alchim31.maven @@ -329,6 +346,35 @@ + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.12.4 + + true + + + + + org.scalatest + scalatest-maven-plugin + 1.0 + + ${project.build.directory}/surefire-reports + . + TestSuite.txt + + + + test + + test + + + + diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala index 6cc86f4de2..4c1024bea3 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala @@ -51,7 +51,7 @@ class RangerSparkAuthorizer( Option(hiveAuthenticator.getUserName).map(UserGroupInformation.createRemoteUser).orNull } - private val sparkPlugin = RangerSparkPlugin.build().sessionContext(sessionContext).getOrCreate() + private val sparkPlugin = RangerSparkPlugin.build().getOrCreate() override def getVersion: HiveAuthorizer.VERSION = HiveAuthorizer.VERSION.V1 diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkPlugin.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkPlugin.scala index fa3d630035..2a42e8fc9a 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkPlugin.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkPlugin.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionC import org.apache.ranger.authorization.hadoop.config.RangerConfiguration import org.apache.ranger.plugin.service.RangerBasePlugin -class RangerSparkPlugin(appType: String) extends RangerBasePlugin("spark", appType) { +class RangerSparkPlugin private extends RangerBasePlugin("spark", "sparkSql") { import RangerSparkPlugin._ private val LOG = LogFactory.getLog(classOf[RangerSparkPlugin]) @@ -63,23 +63,11 @@ object RangerSparkPlugin { class Builder { - @volatile private var appType: String = "unknown" @volatile private var sparkPlugin: RangerSparkPlugin = _ - def sessionContext(sessionContext: HiveAuthzSessionContext): Builder = { - appType = Option(sessionContext).map(_.getClientType) match { - case Some(v) => v match { - case CLIENT_TYPE.HIVECLI => "hiveCLI" - case CLIENT_TYPE.HIVESERVER2 => "hiveServer2" - } - case _ => "unknown" - } - this - } - def getOrCreate(): RangerSparkPlugin = RangerSparkPlugin.synchronized { if (sparkPlugin == null) { - sparkPlugin = new RangerSparkPlugin(appType) + sparkPlugin = new RangerSparkPlugin sparkPlugin.init() sparkPlugin } else { diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala index 78c0d051da..b026af84d0 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala @@ -23,17 +23,17 @@ import org.apache.ranger.authorization.spark.authorizer._ import org.apache.ranger.plugin.model.RangerPolicy import org.apache.ranger.plugin.policyengine.RangerAccessResult import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTable, HiveTableRelation} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, ExprId, NamedExpression, SubqueryExpression} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.AuthzUtils.getFieldVal -import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTable} -import org.apache.spark.sql.catalyst.expressions.{Attribute, NamedExpression} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, CreateViewCommand, InsertIntoDataSourceDirCommand} import org.apache.spark.sql.execution.datasources.{InsertIntoDataSourceCommand, InsertIntoHadoopFsRelationCommand, LogicalRelation, SaveIntoDataSourceCommand} import org.apache.spark.sql.hive.execution.{CreateHiveTableAsSelectCommand, InsertIntoHiveDirCommand, InsertIntoHiveTable} import scala.collection.JavaConverters._ +import scala.collection.mutable /** * An Apache Spark's [[Optimizer]] extension for column data masking. @@ -52,6 +52,9 @@ case class RangerSparkMaskingExtension(spark: SparkSession) extends Rule[Logical .foreach(spark.sessionState.catalog.registerFunction(_, true)) private lazy val sparkPlugin = RangerSparkPlugin.build().getOrCreate() + private lazy val sqlParser = spark.sessionState.sqlParser + private lazy val analyzer = spark.sessionState.analyzer + private lazy val rangerSparkOptimizer = new RangerSparkOptimizer(spark) /** * Collecting transformers from Ranger data masking policies, and mapping the to the @@ -62,7 +65,9 @@ case class RangerSparkMaskingExtension(spark: SparkSession) extends Rule[Logical * @return a list of key-value pairs of original expression with its masking representation */ private def collectTransformers( - plan: LogicalPlan, table: CatalogTable): Seq[(Attribute, NamedExpression)] = { + plan: LogicalPlan, + table: CatalogTable, + aliases: mutable.Map[Alias, ExprId]): Map[ExprId, NamedExpression] = { val auditHandler = new RangerSparkAuditHandler() val ugi = UserGroupInformation.getCurrentUser val userName = ugi.getShortUserName @@ -71,45 +76,71 @@ case class RangerSparkMaskingExtension(spark: SparkSession) extends Rule[Logical val identifier = table.identifier import SparkObjectType._ - plan.output.map { expr => + val maskEnableResults = plan.output.map { expr => val resource = RangerSparkResource(COLUMN, identifier.database, identifier.table, expr.name) val req = new RangerSparkAccessRequest(resource, userName, groups, COLUMN.toString, SparkAccessType.SELECT, null, null, sparkPlugin.getClusterName) (expr, sparkPlugin.evalDataMaskPolicies(req, auditHandler)) - }.filter(x => isMaskEnabled(x._2)).map { x => - val expr = x._1 - val result = x._2 + }.filter(x => isMaskEnabled(x._2)) + + val originMaskers = maskEnableResults.map { case (expr, result) => if (StringUtils.equalsIgnoreCase(result.getMaskType, MASK_TYPE_NULL)) { val sql = s"SELECT NULL AS ${expr.name} FROM ${table.qualifiedName}" - (expr, spark.sessionState.sqlParser.parsePlan(sql)) + val plan = analyzer.execute(sqlParser.parsePlan(sql)) + (expr, plan) } else if (StringUtils.equalsIgnoreCase(result.getMaskType, MASK_TYPE_CUSTOM)) { val maskVal = result.getMaskedValue if (maskVal == null) { val sql = s"SELECT NULL AS ${expr.name} FROM ${table.qualifiedName}" - (expr, spark.sessionState.sqlParser.parsePlan(sql)) + val plan = analyzer.execute(sqlParser.parsePlan(sql)) + (expr, plan) } else { val sql = s"SELECT ${maskVal.replace("{col}", expr.name)} AS ${expr.name} FROM" + s" ${table.qualifiedName}" - (expr, spark.sessionState.sqlParser.parsePlan(sql)) + val plan = analyzer.execute(sqlParser.parsePlan(sql)) + (expr, plan) } } else if (result.getMaskTypeDef != null) { val transformer = result.getMaskTypeDef.getTransformer if (StringUtils.isNotEmpty(transformer)) { val trans = transformer.replace("{col}", expr.name) val sql = s"SELECT $trans AS ${expr.name} FROM ${table.qualifiedName}" - (expr, spark.sessionState.sqlParser.parsePlan(sql)) + val plan = analyzer.execute(sqlParser.parsePlan(sql)) + (expr, plan) } else { (expr, null) } } else { (expr, null) } - }.filter(_._2 != null).map { kv => - kv._2 match { - case p: Project => (kv._1, p.projectList.head) - case _ => (kv._1, kv._2.output.head) + }.filter(_._2 != null) + + val formedMaskers: Map[ExprId, Alias] = + originMaskers.map { case (expr, p) => (expr, p.asInstanceOf[Project].projectList.head) } + .map { case (expr, attr) => + val originalAlias = attr.asInstanceOf[Alias] + val newChild = originalAlias.child mapChildren { + case _: AttributeReference => expr + case o => o + } + val newAlias = originalAlias.copy(child = newChild)( + originalAlias.exprId, originalAlias.qualifier, originalAlias.explicitMetadata) + (expr.exprId, newAlias) + }.toMap + + val aliasedMaskers = new mutable.HashMap[ExprId, Alias]() + for ((alias, id) <- aliases if formedMaskers.contains(id)) { + val originalAlias = formedMaskers(id) + val newChild = originalAlias.child mapChildren { + case ar: AttributeReference => + ar.copy(name = alias.name)(alias.exprId, alias.qualifier) + case o => o } + val newAlias = originalAlias.copy(child = newChild, alias.name)( + originalAlias.exprId, originalAlias.qualifier, originalAlias.explicitMetadata) + aliasedMaskers.put(alias.exprId, newAlias) } + formedMaskers ++ aliasedMaskers } catch { case e: Exception => throw e } @@ -119,31 +150,62 @@ case class RangerSparkMaskingExtension(spark: SparkSession) extends Rule[Logical result != null && result.isMaskEnabled } + private def hasCatalogTable(plan: LogicalPlan): Boolean = plan match { + case _: HiveTableRelation => true + case l: LogicalRelation if l.catalogTable.isDefined => true + case _ => false + } + + private def collectAllAliases(plan: LogicalPlan): mutable.HashMap[Alias, ExprId] = { + val aliases = new mutable.HashMap[Alias, ExprId]() + plan.transformAllExpressions { + case a: Alias => + a.child match { + case ne: NamedExpression => + aliases.put(a, ne.exprId) + case _ => + } + a + } + aliases + } + + private def collectAllTransformers( + plan: LogicalPlan, aliases: mutable.Map[Alias, ExprId]): Map[ExprId, NamedExpression] = { + plan.collectLeaves().flatMap { + case h: HiveTableRelation => + collectTransformers(h, h.tableMeta, aliases) + case l: LogicalRelation if l.catalogTable.isDefined => + collectTransformers(l, l.catalogTable.get, aliases) + case _ => Seq.empty + }.toMap + } + private def doMasking(plan: LogicalPlan): LogicalPlan = plan match { + case s: Subquery => s case m: RangerSparkMasking => m // escape the optimize iteration if already masked case fixed if fixed.find(_.isInstanceOf[RangerSparkMasking]).nonEmpty => fixed case _ => - val transformers = plan.collectLeaves().flatMap { - case h if h.nodeName == "HiveTableRelation" => - val ct = getFieldVal(h, "tableMeta").asInstanceOf[CatalogTable] - collectTransformers(h, ct) - case m if m.nodeName == "MetastoreRelation" => - val ct = getFieldVal(m, "catalogTable").asInstanceOf[CatalogTable] - collectTransformers(m, ct) - case l: LogicalRelation if l.catalogTable.isDefined => - collectTransformers(l, l.catalogTable.get) - case _ => Seq.empty - }.toMap - val newOutput = plan.output.map(attr => transformers.getOrElse(attr, attr)) - val planWithMasking = plan match { - case Subquery(child) => Subquery(Project(newOutput, child)) - case _ => Project(newOutput, plan) + val aliases = collectAllAliases(plan) + val transformers = collectAllTransformers(plan, aliases) + val newPlan = + if (transformers.nonEmpty && plan.output.exists(o => transformers.get(o.exprId).nonEmpty)) { + val newOutput = plan.output.map(attr => transformers.getOrElse(attr.exprId, attr)) + Project(newOutput, plan) + } else { + plan + } + + val marked = newPlan transformUp { + case p if hasCatalogTable(p) => RangerSparkMasking(p) } - val masked = planWithMasking transformUp { - case l: LeafNode => RangerSparkMasking(l) + marked transformAllExpressions { + case s: SubqueryExpression => + val Subquery(newPlan) = + rangerSparkOptimizer.execute(Subquery(RangerSparkMasking(s.plan))) + s.withNewPlan(newPlan) } - spark.sessionState.analyzer.execute(masked) } override def apply(plan: LogicalPlan): LogicalPlan = plan match { diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala index fa2befeddf..0d4b4dc835 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala @@ -37,9 +37,7 @@ import scala.collection.JavaConverters._ * An Apache Spark's [[Optimizer]] extension for row level filtering. */ case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[LogicalPlan] { - private lazy val sparkPlugin = RangerSparkPlugin.build().getOrCreate() - private lazy val rangerSparkOptimizer = new RangerSparkOptimizer(spark) /** @@ -70,7 +68,8 @@ case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[Logic val analyzed = spark.sessionState.analyzer.execute(parsedNew) val optimized = analyzed transformAllExpressions { case s: SubqueryExpression => - val Subquery(newPlan) = rangerSparkOptimizer.execute(Subquery(s.plan)) + val Subquery(newPlan) = + rangerSparkOptimizer.execute(Subquery(RangerSparkRowFilter(s.plan))) s.withNewPlan(newPlan) } RangerSparkRowFilter(optimized) diff --git a/plugin-spark/src/test/resources/data/files/kv1.txt b/plugin-spark/src/test/resources/data/files/kv1.txt new file mode 100644 index 0000000000..9825414ecf --- /dev/null +++ b/plugin-spark/src/test/resources/data/files/kv1.txt @@ -0,0 +1,500 @@ +238val_238 +86val_86 +311val_311 +27val_27 +165val_165 +409val_409 +255val_255 +278val_278 +98val_98 +484val_484 +265val_265 +193val_193 +401val_401 +150val_150 +273val_273 +224val_224 +369val_369 +66val_66 +128val_128 +213val_213 +146val_146 +406val_406 +429val_429 +374val_374 +152val_152 +469val_469 +145val_145 +495val_495 +37val_37 +327val_327 +281val_281 +277val_277 +209val_209 +15val_15 +82val_82 +403val_403 +166val_166 +417val_417 +430val_430 +252val_252 +292val_292 +219val_219 +287val_287 +153val_153 +193val_193 +338val_338 +446val_446 +459val_459 +394val_394 +237val_237 +482val_482 +174val_174 +413val_413 +494val_494 +207val_207 +199val_199 +466val_466 +208val_208 +174val_174 +399val_399 +396val_396 +247val_247 +417val_417 +489val_489 +162val_162 +377val_377 +397val_397 +309val_309 +365val_365 +266val_266 +439val_439 +342val_342 +367val_367 +325val_325 +167val_167 +195val_195 +475val_475 +17val_17 +113val_113 +155val_155 +203val_203 +339val_339 +0val_0 +455val_455 +128val_128 +311val_311 +316val_316 +57val_57 +302val_302 +205val_205 +149val_149 +438val_438 +345val_345 +129val_129 +170val_170 +20val_20 +489val_489 +157val_157 +378val_378 +221val_221 +92val_92 +111val_111 +47val_47 +72val_72 +4val_4 +280val_280 +35val_35 +427val_427 +277val_277 +208val_208 +356val_356 +399val_399 +169val_169 +382val_382 +498val_498 +125val_125 +386val_386 +437val_437 +469val_469 +192val_192 +286val_286 +187val_187 +176val_176 +54val_54 +459val_459 +51val_51 +138val_138 +103val_103 +239val_239 +213val_213 +216val_216 +430val_430 +278val_278 +176val_176 +289val_289 +221val_221 +65val_65 +318val_318 +332val_332 +311val_311 +275val_275 +137val_137 +241val_241 +83val_83 +333val_333 +180val_180 +284val_284 +12val_12 +230val_230 +181val_181 +67val_67 +260val_260 +404val_404 +384val_384 +489val_489 +353val_353 +373val_373 +272val_272 +138val_138 +217val_217 +84val_84 +348val_348 +466val_466 +58val_58 +8val_8 +411val_411 +230val_230 +208val_208 +348val_348 +24val_24 +463val_463 +431val_431 +179val_179 +172val_172 +42val_42 +129val_129 +158val_158 +119val_119 +496val_496 +0val_0 +322val_322 +197val_197 +468val_468 +393val_393 +454val_454 +100val_100 +298val_298 +199val_199 +191val_191 +418val_418 +96val_96 +26val_26 +165val_165 +327val_327 +230val_230 +205val_205 +120val_120 +131val_131 +51val_51 +404val_404 +43val_43 +436val_436 +156val_156 +469val_469 +468val_468 +308val_308 +95val_95 +196val_196 +288val_288 +481val_481 +457val_457 +98val_98 +282val_282 +197val_197 +187val_187 +318val_318 +318val_318 +409val_409 +470val_470 +137val_137 +369val_369 +316val_316 +169val_169 +413val_413 +85val_85 +77val_77 +0val_0 +490val_490 +87val_87 +364val_364 +179val_179 +118val_118 +134val_134 +395val_395 +282val_282 +138val_138 +238val_238 +419val_419 +15val_15 +118val_118 +72val_72 +90val_90 +307val_307 +19val_19 +435val_435 +10val_10 +277val_277 +273val_273 +306val_306 +224val_224 +309val_309 +389val_389 +327val_327 +242val_242 +369val_369 +392val_392 +272val_272 +331val_331 +401val_401 +242val_242 +452val_452 +177val_177 +226val_226 +5val_5 +497val_497 +402val_402 +396val_396 +317val_317 +395val_395 +58val_58 +35val_35 +336val_336 +95val_95 +11val_11 +168val_168 +34val_34 +229val_229 +233val_233 +143val_143 +472val_472 +322val_322 +498val_498 +160val_160 +195val_195 +42val_42 +321val_321 +430val_430 +119val_119 +489val_489 +458val_458 +78val_78 +76val_76 +41val_41 +223val_223 +492val_492 +149val_149 +449val_449 +218val_218 +228val_228 +138val_138 +453val_453 +30val_30 +209val_209 +64val_64 +468val_468 +76val_76 +74val_74 +342val_342 +69val_69 +230val_230 +33val_33 +368val_368 +103val_103 +296val_296 +113val_113 +216val_216 +367val_367 +344val_344 +167val_167 +274val_274 +219val_219 +239val_239 +485val_485 +116val_116 +223val_223 +256val_256 +263val_263 +70val_70 +487val_487 +480val_480 +401val_401 +288val_288 +191val_191 +5val_5 +244val_244 +438val_438 +128val_128 +467val_467 +432val_432 +202val_202 +316val_316 +229val_229 +469val_469 +463val_463 +280val_280 +2val_2 +35val_35 +283val_283 +331val_331 +235val_235 +80val_80 +44val_44 +193val_193 +321val_321 +335val_335 +104val_104 +466val_466 +366val_366 +175val_175 +403val_403 +483val_483 +53val_53 +105val_105 +257val_257 +406val_406 +409val_409 +190val_190 +406val_406 +401val_401 +114val_114 +258val_258 +90val_90 +203val_203 +262val_262 +348val_348 +424val_424 +12val_12 +396val_396 +201val_201 +217val_217 +164val_164 +431val_431 +454val_454 +478val_478 +298val_298 +125val_125 +431val_431 +164val_164 +424val_424 +187val_187 +382val_382 +5val_5 +70val_70 +397val_397 +480val_480 +291val_291 +24val_24 +351val_351 +255val_255 +104val_104 +70val_70 +163val_163 +438val_438 +119val_119 +414val_414 +200val_200 +491val_491 +237val_237 +439val_439 +360val_360 +248val_248 +479val_479 +305val_305 +417val_417 +199val_199 +444val_444 +120val_120 +429val_429 +169val_169 +443val_443 +323val_323 +325val_325 +277val_277 +230val_230 +478val_478 +178val_178 +468val_468 +310val_310 +317val_317 +333val_333 +493val_493 +460val_460 +207val_207 +249val_249 +265val_265 +480val_480 +83val_83 +136val_136 +353val_353 +172val_172 +214val_214 +462val_462 +233val_233 +406val_406 +133val_133 +175val_175 +189val_189 +454val_454 +375val_375 +401val_401 +421val_421 +407val_407 +384val_384 +256val_256 +26val_26 +134val_134 +67val_67 +384val_384 +379val_379 +18val_18 +462val_462 +492val_492 +100val_100 +298val_298 +9val_9 +341val_341 +498val_498 +146val_146 +458val_458 +362val_362 +186val_186 +285val_285 +348val_348 +167val_167 +18val_18 +273val_273 +183val_183 +281val_281 +344val_344 +97val_97 +469val_469 +315val_315 +84val_84 +28val_28 +37val_37 +448val_448 +152val_152 +348val_348 +307val_307 +194val_194 +414val_414 +477val_477 +222val_222 +126val_126 +90val_90 +169val_169 +403val_403 +400val_400 +200val_200 +97val_97 diff --git a/plugin-spark/src/test/resources/log4j.properties b/plugin-spark/src/test/resources/log4j.properties new file mode 100644 index 0000000000..41535d66b2 --- /dev/null +++ b/plugin-spark/src/test/resources/log4j.properties @@ -0,0 +1,23 @@ +# +# 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. +# + +# Set everything to be logged to the console +log4j.rootCategory=FATAL, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n \ No newline at end of file diff --git a/plugin-spark/src/test/resources/ranger-spark-security.xml b/plugin-spark/src/test/resources/ranger-spark-security.xml new file mode 100644 index 0000000000..a19d424106 --- /dev/null +++ b/plugin-spark/src/test/resources/ranger-spark-security.xml @@ -0,0 +1,45 @@ + + + + + + + ranger.plugin.spark.service.name + hive_jenkins + + Name of the Ranger service containing policies for this SampleApp instance + + + + + ranger.plugin.spark.policy.source.impl + org.apache.ranger.services.spark.RangerAdminClientImpl + + Policy source. + + + + + ranger.plugin.spark.policy.cache.dir + target/test-classes + + Directory where Ranger policies are cached after successful retrieval from the source + + + + diff --git a/plugin-spark/src/test/resources/sparkSql_hive_jenkins.json b/plugin-spark/src/test/resources/sparkSql_hive_jenkins.json new file mode 100644 index 0000000000..42a9a03364 --- /dev/null +++ b/plugin-spark/src/test/resources/sparkSql_hive_jenkins.json @@ -0,0 +1,1897 @@ +{ + "serviceName": "hive_jenkins", + "serviceId": 1, + "policyVersion": 73, + "policyUpdateTime": "20190424-16:19:30.000-+0800", + "policies": [ + { + "service": "hive_jenkins", + "name": "all - url", + "policyType": 0, + "policyPriority": 0, + "description": "Policy for all - url", + "isAuditEnabled": true, + "resources": { + "url": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": true + } + }, + "policyItems": [ + { + "accesses": [ + { + "type": "select", + "isAllowed": true + }, + { + "type": "update", + "isAllowed": true + }, + { + "type": "create", + "isAllowed": true + }, + { + "type": "drop", + "isAllowed": true + }, + { + "type": "alter", + "isAllowed": true + }, + { + "type": "index", + "isAllowed": true + }, + { + "type": "lock", + "isAllowed": true + }, + { + "type": "all", + "isAllowed": true + }, + { + "type": "read", + "isAllowed": true + }, + { + "type": "write", + "isAllowed": true + } + ], + "users": [ + "admin" + ], + "groups": [], + "conditions": [], + "delegateAdmin": true + } + ], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [], + "id": 1, + "guid": "cf7e6725-492f-434f-bffe-6bb4e3147246", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "all - database, table, column", + "policyType": 0, + "policyPriority": 0, + "description": "Policy for all - database, table, column", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [ + { + "accesses": [ + { + "type": "select", + "isAllowed": true + }, + { + "type": "update", + "isAllowed": true + }, + { + "type": "create", + "isAllowed": true + }, + { + "type": "drop", + "isAllowed": true + }, + { + "type": "alter", + "isAllowed": true + }, + { + "type": "index", + "isAllowed": true + }, + { + "type": "lock", + "isAllowed": true + }, + { + "type": "all", + "isAllowed": true + }, + { + "type": "read", + "isAllowed": true + }, + { + "type": "write", + "isAllowed": true + } + ], + "users": [ + "admin" + ], + "groups": [], + "conditions": [], + "delegateAdmin": true + } + ], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [], + "id": 2, + "guid": "3b96138a-af4d-48bc-9544-58c5bfa1979b", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "all - database, udf", + "policyType": 0, + "policyPriority": 0, + "description": "Policy for all - database, udf", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + }, + "udf": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [ + { + "accesses": [ + { + "type": "select", + "isAllowed": true + }, + { + "type": "update", + "isAllowed": true + }, + { + "type": "create", + "isAllowed": true + }, + { + "type": "drop", + "isAllowed": true + }, + { + "type": "alter", + "isAllowed": true + }, + { + "type": "index", + "isAllowed": true + }, + { + "type": "lock", + "isAllowed": true + }, + { + "type": "all", + "isAllowed": true + }, + { + "type": "read", + "isAllowed": true + }, + { + "type": "write", + "isAllowed": true + } + ], + "users": [ + "admin" + ], + "groups": [], + "conditions": [], + "delegateAdmin": true + } + ], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [], + "id": 3, + "guid": "db08fbb0-61da-4f33-8144-ccd89816151d", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "test1", + "policyType": 2, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "src" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "key\u003c20" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "serviceType": "hive", + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 4, + "guid": "f588a9ed-f7b1-48f7-9d0d-c12cf2b9b7ed", + "isEnabled": true, + "version": 25 + }, + { + "service": "hive_jenkins", + "name": "default", + "policyType": 0, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [ + { + "accesses": [ + { + "type": "select", + "isAllowed": true + }, + { + "type": "update", + "isAllowed": true + }, + { + "type": "create", + "isAllowed": true + }, + { + "type": "drop", + "isAllowed": true + }, + { + "type": "alter", + "isAllowed": true + }, + { + "type": "index", + "isAllowed": true + }, + { + "type": "lock", + "isAllowed": true + }, + { + "type": "all", + "isAllowed": true + }, + { + "type": "read", + "isAllowed": true + }, + { + "type": "write", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [ + "public" + ], + "conditions": [], + "delegateAdmin": false + } + ], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 5, + "guid": "2db6099d-e4f1-41df-9d24-f2f47bed618e", + "isEnabled": true, + "version": 5 + }, + { + "service": "hive_jenkins", + "name": "src", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "value" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "src" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_SHOW_LAST_4" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 6, + "guid": "b1261fcc-b2cd-49f2-85e8-93f254f987ec", + "isEnabled": true, + "version": 9 + }, + { + "service": "hive_jenkins", + "name": "store_sales", + "policyType": 2, + "policyPriority": 0, + "description": "equality", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "store_sales" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "ss_sold_date_sk\u003d2451546" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 7, + "guid": "08fa307f-77fa-4586-83d0-21d0eb68b0fc", + "isEnabled": true, + "version": 4 + }, + { + "service": "hive_jenkins", + "name": "default", + "policyType": 0, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "*" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [ + { + "accesses": [ + { + "type": "select", + "isAllowed": true + }, + { + "type": "update", + "isAllowed": true + }, + { + "type": "create", + "isAllowed": true + }, + { + "type": "drop", + "isAllowed": true + }, + { + "type": "alter", + "isAllowed": true + }, + { + "type": "index", + "isAllowed": true + }, + { + "type": "lock", + "isAllowed": true + }, + { + "type": "all", + "isAllowed": true + }, + { + "type": "read", + "isAllowed": true + }, + { + "type": "write", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 8, + "guid": "cfd49756-2d80-492d-bd26-6f67d531f28c", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "catalog_page", + "policyType": 2, + "policyPriority": 0, + "description": "key in another table", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "catalog_page" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "cp_start_date_sk in (select d_date_sk from date_dim)" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 9, + "guid": "ec617d1b-b85d-434f-b9db-8ef0178620f1", + "isEnabled": true, + "version": 2 + }, + { + "service": "hive_jenkins", + "name": "call_center", + "policyType": 2, + "policyPriority": 0, + "description": "is not null", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "call_center" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "cc_name is not null" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 10, + "guid": "c8259509-61ae-48f8-867f-be8cac339764", + "isEnabled": true, + "version": 2 + }, + { + "service": "hive_jenkins", + "name": "catalog_returns", + "policyType": 2, + "policyPriority": 0, + "description": "or expression", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "catalog_returns" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "cr_item_sk is null or cr_item_sk \u003e\u003d0" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 11, + "guid": "58aa8789-799b-4ce7-820e-9ed625ff2206", + "isEnabled": true, + "version": 2 + }, + { + "service": "hive_jenkins", + "name": "date_dim", + "policyType": 2, + "policyPriority": 0, + "description": "AND and UDF", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "date_dim" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "d_date_sk\u003d0 and d_date\u003dcurrent_date()" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 12, + "guid": "cc7b3ede-e483-4ba9-9584-2907f3237df0", + "isEnabled": true, + "version": 2 + }, + { + "service": "hive_jenkins", + "name": "reason", + "policyType": 2, + "policyPriority": 0, + "description": "row filter expression with a key in the table itself", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "reason" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "r_reason_sk in (select r_reason_sk from reason)" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 14, + "guid": "4c8d06ae-73ea-4ff8-aedb-4aeae6865768", + "isEnabled": true, + "version": 2 + }, + { + "service": "hive_jenkins", + "name": "inventory", + "policyType": 2, + "policyPriority": 0, + "description": "scalar expression with the table itself", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "inventory" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "inv_item_sk\u003d(select count(1) from inventory)" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 15, + "guid": "1e3da1db-47f3-465e-a604-aaf3d3a8de8e", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "item_i_item_id", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "i_item_id" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "item" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_SHOW_LAST_4", + "valueExpr": "" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 16, + "guid": "3bf13c7b-14b7-40cf-a7ed-913a3e528a11", + "isEnabled": true, + "version": 3 + }, + { + "service": "hive_jenkins", + "name": "customer_address", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "ca_state" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "customer_address" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_SHOW_LAST_4", + "valueExpr": "" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 17, + "guid": "a047b76d-ea97-4893-b469-94cc944b3edc", + "isEnabled": true, + "version": 4 + }, + { + "service": "hive_jenkins", + "name": "customer", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "c_customer_id" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "customer" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 18, + "guid": "ac2d963e-635f-49a8-a96c-ded88f68e731", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "date_dim_2", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "d_year" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "date_dim" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_NULL" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 19, + "guid": "07e7df0d-2cf7-4630-b796-31798a4496d4", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "item_i_brand_id", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "i_brand_id" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "item" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_HASH" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 20, + "guid": "35b5e3f7-c9f0-42d1-9118-56dc37ff42f5", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "item_i_item_sk", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "i_item_sk" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "item" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_SHOW_FIRST_4" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 21, + "guid": "7e16c0ca-927a-4e95-b42e-c93b62cb6dfa", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "item_i_class_id", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "i_class_id" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "item" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_NULL" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 22, + "guid": "b7847238-3a14-4d56-8257-b8625a7f25a1", + "isEnabled": true, + "version": 1 + } + ], + "serviceDef": { + "name": "bob", + "implClass": "org.apache.ranger.services.hive.RangerServiceHive", + "label": "Hive Server2", + "description": "Hive Server2", + "options": { + "enableDenyAndExceptionsInPolicies": "true" + }, + "configs": [ + { + "itemId": 1, + "name": "username", + "type": "string", + "mandatory": true, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "Username" + }, + { + "itemId": 2, + "name": "password", + "type": "password", + "mandatory": true, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "Password" + }, + { + "itemId": 3, + "name": "jdbc.driverClassName", + "type": "string", + "mandatory": true, + "defaultValue": "org.apache.hive.jdbc.HiveDriver", + "validationRegEx": "", + "validationMessage": "", + "uiHint": "" + }, + { + "itemId": 4, + "name": "jdbc.url", + "type": "string", + "mandatory": true, + "defaultValue": "", + "validationRegEx": "", + "validationMessage": "", + "uiHint": "{\"TextFieldWithIcon\":true, \"info\": \"1.For Remote Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;:\u0026lt;port\u0026gt;\u003cbr\u003e2.For Embedded Mode (no host or port), eg.\u003cbr\u003ejdbc:hive2:///;initFile\u003d\u0026lt;file\u0026gt;\u003cbr\u003e3.For HTTP Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;:\u0026lt;port\u0026gt;/;\u003cbr\u003etransportMode\u003dhttp;httpPath\u003d\u0026lt;httpPath\u0026gt;\u003cbr\u003e4.For SSL Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;:\u0026lt;port\u0026gt;/;ssl\u003dtrue;\u003cbr\u003esslTrustStore\u003dtStore;trustStorePassword\u003dpw\u003cbr\u003e5.For ZooKeeper Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;/;serviceDiscoveryMode\u003d\u003cbr\u003ezooKeeper;zooKeeperNamespace\u003dhiveserver2\u003cbr\u003e6.For Kerberos Mode, eg.\u003cbr\u003ejdbc:hive2://\u0026lt;host\u0026gt;:\u0026lt;port\u0026gt;/;\u003cbr\u003eprincipal\u003dhive/domain@EXAMPLE.COM\u003cbr\u003e\"}" + }, + { + "itemId": 5, + "name": "commonNameForCertificate", + "type": "string", + "mandatory": false, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "Common Name for Certificate" + } + ], + "resources": [ + { + "itemId": 1, + "name": "database", + "type": "string", + "level": 10, + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": true, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "true", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "Hive Database", + "description": "Hive Database", + "accessTypeRestrictions": [], + "isValidLeaf": false + }, + { + "itemId": 5, + "name": "url", + "type": "string", + "level": 10, + "mandatory": true, + "lookupSupported": false, + "recursiveSupported": true, + "excludesSupported": false, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerPathResourceMatcher", + "matcherOptions": { + "wildCard": "true", + "ignoreCase": "false" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "URL", + "description": "URL", + "accessTypeRestrictions": [], + "isValidLeaf": true + }, + { + "itemId": 2, + "name": "table", + "type": "string", + "level": 20, + "parent": "database", + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": true, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "true", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "Hive Table", + "description": "Hive Table", + "accessTypeRestrictions": [], + "isValidLeaf": false + }, + { + "itemId": 3, + "name": "udf", + "type": "string", + "level": 20, + "parent": "database", + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": true, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "true", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "Hive UDF", + "description": "Hive UDF", + "accessTypeRestrictions": [], + "isValidLeaf": true + }, + { + "itemId": 4, + "name": "column", + "type": "string", + "level": 30, + "parent": "table", + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": true, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "true", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "", + "label": "Hive Column", + "description": "Hive Column", + "accessTypeRestrictions": [], + "isValidLeaf": true + } + ], + "accessTypes": [ + { + "itemId": 1, + "name": "select", + "label": "select", + "impliedGrants": [] + }, + { + "itemId": 2, + "name": "update", + "label": "update", + "impliedGrants": [] + }, + { + "itemId": 3, + "name": "create", + "label": "Create", + "impliedGrants": [] + }, + { + "itemId": 4, + "name": "drop", + "label": "Drop", + "impliedGrants": [] + }, + { + "itemId": 5, + "name": "alter", + "label": "Alter", + "impliedGrants": [] + }, + { + "itemId": 6, + "name": "index", + "label": "Index", + "impliedGrants": [] + }, + { + "itemId": 7, + "name": "lock", + "label": "Lock", + "impliedGrants": [] + }, + { + "itemId": 8, + "name": "all", + "label": "All", + "impliedGrants": [ + "select", + "update", + "create", + "drop", + "alter", + "index", + "lock", + "read", + "write" + ] + }, + { + "itemId": 9, + "name": "read", + "label": "Read", + "impliedGrants": [] + }, + { + "itemId": 10, + "name": "write", + "label": "Write", + "impliedGrants": [] + } + ], + "policyConditions": [], + "contextEnrichers": [], + "enums": [], + "dataMaskDef": { + "maskTypes": [ + { + "itemId": 1, + "name": "MASK", + "label": "Redact", + "description": "Replace lowercase with \u0027x\u0027, uppercase with \u0027X\u0027, digits with \u00270\u0027", + "transformer": "mask({col})", + "dataMaskOptions": {} + }, + { + "itemId": 2, + "name": "MASK_SHOW_LAST_4", + "label": "Partial mask: show last 4", + "description": "Show last 4 characters; replace rest with \u0027x\u0027", + "transformer": "mask_show_last_n({col}, 4, \u0027x\u0027, \u0027x\u0027, \u0027x\u0027, -1, \u00271\u0027)", + "dataMaskOptions": {} + }, + { + "itemId": 3, + "name": "MASK_SHOW_FIRST_4", + "label": "Partial mask: show first 4", + "description": "Show first 4 characters; replace rest with \u0027x\u0027", + "transformer": "mask_show_first_n({col}, 4, \u0027x\u0027, \u0027x\u0027, \u0027x\u0027, -1, \u00271\u0027)", + "dataMaskOptions": {} + }, + { + "itemId": 4, + "name": "MASK_HASH", + "label": "Hash", + "description": "Hash the value", + "transformer": "mask_hash({col})", + "dataMaskOptions": {} + }, + { + "itemId": 5, + "name": "MASK_NULL", + "label": "Nullify", + "description": "Replace with NULL", + "dataMaskOptions": {} + }, + { + "itemId": 6, + "name": "MASK_NONE", + "label": "Unmasked (retain original value)", + "description": "No masking", + "dataMaskOptions": {} + }, + { + "itemId": 12, + "name": "MASK_DATE_SHOW_YEAR", + "label": "Date: show only year", + "description": "Date: show only year", + "transformer": "mask({col}, \u0027x\u0027, \u0027x\u0027, \u0027x\u0027, -1, \u00271\u0027, 1, 0, -1)", + "dataMaskOptions": {} + }, + { + "itemId": 13, + "name": "CUSTOM", + "label": "Custom", + "description": "Custom", + "dataMaskOptions": {} + } + ], + "accessTypes": [ + { + "itemId": 1, + "name": "select", + "label": "select", + "impliedGrants": [] + } + ], + "resources": [ + { + "itemId": 1, + "name": "database", + "type": "string", + "level": 10, + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": false, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "false", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "{ \"singleValue\":true }", + "label": "Hive Database", + "description": "Hive Database", + "accessTypeRestrictions": [], + "isValidLeaf": false + }, + { + "itemId": 2, + "name": "table", + "type": "string", + "level": 20, + "parent": "database", + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": false, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "false", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "{ \"singleValue\":true }", + "label": "Hive Table", + "description": "Hive Table", + "accessTypeRestrictions": [], + "isValidLeaf": false + }, + { + "itemId": 4, + "name": "column", + "type": "string", + "level": 30, + "parent": "table", + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": false, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "false", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "{ \"singleValue\":true }", + "label": "Hive Column", + "description": "Hive Column", + "accessTypeRestrictions": [], + "isValidLeaf": true + } + ] + }, + "rowFilterDef": { + "accessTypes": [ + { + "itemId": 1, + "name": "select", + "label": "select", + "impliedGrants": [] + } + ], + "resources": [ + { + "itemId": 1, + "name": "database", + "type": "string", + "level": 10, + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": false, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "false", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "{ \"singleValue\":true }", + "label": "Hive Database", + "description": "Hive Database", + "accessTypeRestrictions": [], + "isValidLeaf": false + }, + { + "itemId": 2, + "name": "table", + "type": "string", + "level": 20, + "parent": "database", + "mandatory": true, + "lookupSupported": true, + "recursiveSupported": false, + "excludesSupported": false, + "matcher": "org.apache.ranger.plugin.resourcematcher.RangerDefaultResourceMatcher", + "matcherOptions": { + "wildCard": "false", + "ignoreCase": "true" + }, + "validationRegEx": "", + "validationMessage": "", + "uiHint": "{ \"singleValue\":true }", + "label": "Hive Table", + "description": "Hive Table", + "accessTypeRestrictions": [], + "isValidLeaf": true + } + ] + }, + "id": 3, + "guid": "3e1afb5a-184a-4e82-9d9c-87a5cacc243c", + "isEnabled": true, + "createTime": "20190401-20:14:36.000-+0800", + "updateTime": "20190401-20:14:36.000-+0800", + "version": 1 + }, + "auditMode": "audit-default" +} \ No newline at end of file diff --git a/plugin-spark/src/test/resources/tpcds/q1.sql b/plugin-spark/src/test/resources/tpcds/q1.sql new file mode 100755 index 0000000000..4d20faad8e --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q1.sql @@ -0,0 +1,19 @@ +WITH customer_total_return AS +( SELECT + sr_customer_sk AS ctr_customer_sk, + sr_store_sk AS ctr_store_sk, + sum(sr_return_amt) AS ctr_total_return + FROM store_returns, date_dim + WHERE sr_returned_date_sk = d_date_sk AND d_year = 2000 + GROUP BY sr_customer_sk, sr_store_sk) +SELECT c_customer_id +FROM customer_total_return ctr1, store, customer +WHERE ctr1.ctr_total_return > + (SELECT avg(ctr_total_return) * 1.2 + FROM customer_total_return ctr2 + WHERE ctr1.ctr_store_sk = ctr2.ctr_store_sk) + AND s_store_sk = ctr1.ctr_store_sk + AND s_state = 'TN' + AND ctr1.ctr_customer_sk = c_customer_sk +ORDER BY c_customer_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q10.sql b/plugin-spark/src/test/resources/tpcds/q10.sql new file mode 100755 index 0000000000..5500e1aea1 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q10.sql @@ -0,0 +1,57 @@ +SELECT + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3, + cd_dep_count, + count(*) cnt4, + cd_dep_employed_count, + count(*) cnt5, + cd_dep_college_count, + count(*) cnt6 +FROM + customer c, customer_address ca, customer_demographics +WHERE + c.c_current_addr_sk = ca.ca_address_sk AND + ca_county IN ('Rush County', 'Toole County', 'Jefferson County', + 'Dona Ana County', 'La Porte County') AND + cd_demo_sk = c.c_current_cdemo_sk AND + exists(SELECT * + FROM store_sales, date_dim + WHERE c.c_customer_sk = ss_customer_sk AND + ss_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_moy BETWEEN 1 AND 1 + 3) AND + (exists(SELECT * + FROM web_sales, date_dim + WHERE c.c_customer_sk = ws_bill_customer_sk AND + ws_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_moy BETWEEN 1 AND 1 + 3) OR + exists(SELECT * + FROM catalog_sales, date_dim + WHERE c.c_customer_sk = cs_ship_customer_sk AND + cs_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_moy BETWEEN 1 AND 1 + 3)) +GROUP BY cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count +ORDER BY cd_gender, + cd_marital_status, + cd_education_status, + cd_purchase_estimate, + cd_credit_rating, + cd_dep_count, + cd_dep_employed_count, + cd_dep_college_count +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q11.sql b/plugin-spark/src/test/resources/tpcds/q11.sql new file mode 100755 index 0000000000..3618fb14fa --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q11.sql @@ -0,0 +1,68 @@ +WITH year_total AS ( + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum(ss_ext_list_price - ss_ext_discount_amt) year_total, + 's' sale_type + FROM customer, store_sales, date_dim + WHERE c_customer_sk = ss_customer_sk + AND ss_sold_date_sk = d_date_sk + GROUP BY c_customer_id + , c_first_name + , c_last_name + , d_year + , c_preferred_cust_flag + , c_birth_country + , c_login + , c_email_address + , d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum(ws_ext_list_price - ws_ext_discount_amt) year_total, + 'w' sale_type + FROM customer, web_sales, date_dim + WHERE c_customer_sk = ws_bill_customer_sk + AND ws_sold_date_sk = d_date_sk + GROUP BY + c_customer_id, c_first_name, c_last_name, c_preferred_cust_flag, c_birth_country, + c_login, c_email_address, d_year) +SELECT t_s_secyear.customer_preferred_cust_flag +FROM year_total t_s_firstyear + , year_total t_s_secyear + , year_total t_w_firstyear + , year_total t_w_secyear +WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_secyear.customer_id + AND t_s_firstyear.customer_id = t_w_firstyear.customer_id + AND t_s_firstyear.sale_type = 's' + AND t_w_firstyear.sale_type = 'w' + AND t_s_secyear.sale_type = 's' + AND t_w_secyear.sale_type = 'w' + AND t_s_firstyear.dyear = 2001 + AND t_s_secyear.dyear = 2001 + 1 + AND t_w_firstyear.dyear = 2001 + AND t_w_secyear.dyear = 2001 + 1 + AND t_s_firstyear.year_total > 0 + AND t_w_firstyear.year_total > 0 + AND CASE WHEN t_w_firstyear.year_total > 0 + THEN t_w_secyear.year_total / t_w_firstyear.year_total + ELSE NULL END + > CASE WHEN t_s_firstyear.year_total > 0 + THEN t_s_secyear.year_total / t_s_firstyear.year_total + ELSE NULL END +ORDER BY t_s_secyear.customer_preferred_cust_flag +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q12.sql b/plugin-spark/src/test/resources/tpcds/q12.sql new file mode 100755 index 0000000000..0382737f5a --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q12.sql @@ -0,0 +1,22 @@ +SELECT + i_item_desc, + i_category, + i_class, + i_current_price, + sum(ws_ext_sales_price) AS itemrevenue, + sum(ws_ext_sales_price) * 100 / sum(sum(ws_ext_sales_price)) + OVER + (PARTITION BY i_class) AS revenueratio +FROM + web_sales, item, date_dim +WHERE + ws_item_sk = i_item_sk + AND i_category IN ('Sports', 'Books', 'Home') + AND ws_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('1999-02-22' AS DATE) + AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) +GROUP BY + i_item_id, i_item_desc, i_category, i_class, i_current_price +ORDER BY + i_category, i_class, i_item_id, i_item_desc, revenueratio +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q13.sql b/plugin-spark/src/test/resources/tpcds/q13.sql new file mode 100755 index 0000000000..32dc9e2609 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q13.sql @@ -0,0 +1,49 @@ +SELECT + avg(ss_quantity), + avg(ss_ext_sales_price), + avg(ss_ext_wholesale_cost), + sum(ss_ext_wholesale_cost) +FROM store_sales + , store + , customer_demographics + , household_demographics + , customer_address + , date_dim +WHERE s_store_sk = ss_store_sk + AND ss_sold_date_sk = d_date_sk AND d_year = 2001 + AND ((ss_hdemo_sk = hd_demo_sk + AND cd_demo_sk = ss_cdemo_sk + AND cd_marital_status = 'M' + AND cd_education_status = 'Advanced Degree' + AND ss_sales_price BETWEEN 100.00 AND 150.00 + AND hd_dep_count = 3 +) OR + (ss_hdemo_sk = hd_demo_sk + AND cd_demo_sk = ss_cdemo_sk + AND cd_marital_status = 'S' + AND cd_education_status = 'College' + AND ss_sales_price BETWEEN 50.00 AND 100.00 + AND hd_dep_count = 1 + ) OR + (ss_hdemo_sk = hd_demo_sk + AND cd_demo_sk = ss_cdemo_sk + AND cd_marital_status = 'W' + AND cd_education_status = '2 yr Degree' + AND ss_sales_price BETWEEN 150.00 AND 200.00 + AND hd_dep_count = 1 + )) + AND ((ss_addr_sk = ca_address_sk + AND ca_country = 'United States' + AND ca_state IN ('TX', 'OH', 'TX') + AND ss_net_profit BETWEEN 100 AND 200 +) OR + (ss_addr_sk = ca_address_sk + AND ca_country = 'United States' + AND ca_state IN ('OR', 'NM', 'KY') + AND ss_net_profit BETWEEN 150 AND 300 + ) OR + (ss_addr_sk = ca_address_sk + AND ca_country = 'United States' + AND ca_state IN ('VA', 'TX', 'MS') + AND ss_net_profit BETWEEN 50 AND 250 + )) diff --git a/plugin-spark/src/test/resources/tpcds/q14a.sql b/plugin-spark/src/test/resources/tpcds/q14a.sql new file mode 100755 index 0000000000..954ddd41be --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q14a.sql @@ -0,0 +1,120 @@ +WITH cross_items AS +(SELECT i_item_sk ss_item_sk + FROM item, + (SELECT + iss.i_brand_id brand_id, + iss.i_class_id class_id, + iss.i_category_id category_id + FROM store_sales, item iss, date_dim d1 + WHERE ss_item_sk = iss.i_item_sk + AND ss_sold_date_sk = d1.d_date_sk + AND d1.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + ics.i_brand_id, + ics.i_class_id, + ics.i_category_id + FROM catalog_sales, item ics, date_dim d2 + WHERE cs_item_sk = ics.i_item_sk + AND cs_sold_date_sk = d2.d_date_sk + AND d2.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + iws.i_brand_id, + iws.i_class_id, + iws.i_category_id + FROM web_sales, item iws, date_dim d3 + WHERE ws_item_sk = iws.i_item_sk + AND ws_sold_date_sk = d3.d_date_sk + AND d3.d_year BETWEEN 1999 AND 1999 + 2) x + WHERE i_brand_id = brand_id + AND i_class_id = class_id + AND i_category_id = category_id +), + avg_sales AS + (SELECT avg(quantity * list_price) average_sales + FROM ( + SELECT + ss_quantity quantity, + ss_list_price list_price + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk + AND d_year BETWEEN 1999 AND 2001 + UNION ALL + SELECT + cs_quantity quantity, + cs_list_price list_price + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk + AND d_year BETWEEN 1999 AND 1999 + 2 + UNION ALL + SELECT + ws_quantity quantity, + ws_list_price list_price + FROM web_sales, date_dim + WHERE ws_sold_date_sk = d_date_sk + AND d_year BETWEEN 1999 AND 1999 + 2) x) +SELECT + channel, + i_brand_id, + i_class_id, + i_category_id, + sum(sales), + sum(number_sales) +FROM ( + SELECT + 'store' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ss_quantity * ss_list_price) sales, + count(*) number_sales + FROM store_sales, item, date_dim + WHERE ss_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 1999 + 2 + AND d_moy = 11 + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales + FROM avg_sales) + UNION ALL + SELECT + 'catalog' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(cs_quantity * cs_list_price) sales, + count(*) number_sales + FROM catalog_sales, item, date_dim + WHERE cs_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 1999 + 2 + AND d_moy = 11 + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(cs_quantity * cs_list_price) > (SELECT average_sales FROM avg_sales) + UNION ALL + SELECT + 'web' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ws_quantity * ws_list_price) sales, + count(*) number_sales + FROM web_sales, item, date_dim + WHERE ws_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 1999 + 2 + AND d_moy = 11 + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ws_quantity * ws_list_price) > (SELECT average_sales + FROM avg_sales) + ) y +GROUP BY ROLLUP (channel, i_brand_id, i_class_id, i_category_id) +ORDER BY channel, i_brand_id, i_class_id, i_category_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q14b.sql b/plugin-spark/src/test/resources/tpcds/q14b.sql new file mode 100755 index 0000000000..929a8484bf --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q14b.sql @@ -0,0 +1,95 @@ +WITH cross_items AS +(SELECT i_item_sk ss_item_sk + FROM item, + (SELECT + iss.i_brand_id brand_id, + iss.i_class_id class_id, + iss.i_category_id category_id + FROM store_sales, item iss, date_dim d1 + WHERE ss_item_sk = iss.i_item_sk + AND ss_sold_date_sk = d1.d_date_sk + AND d1.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + ics.i_brand_id, + ics.i_class_id, + ics.i_category_id + FROM catalog_sales, item ics, date_dim d2 + WHERE cs_item_sk = ics.i_item_sk + AND cs_sold_date_sk = d2.d_date_sk + AND d2.d_year BETWEEN 1999 AND 1999 + 2 + INTERSECT + SELECT + iws.i_brand_id, + iws.i_class_id, + iws.i_category_id + FROM web_sales, item iws, date_dim d3 + WHERE ws_item_sk = iws.i_item_sk + AND ws_sold_date_sk = d3.d_date_sk + AND d3.d_year BETWEEN 1999 AND 1999 + 2) x + WHERE i_brand_id = brand_id + AND i_class_id = class_id + AND i_category_id = category_id +), + avg_sales AS + (SELECT avg(quantity * list_price) average_sales + FROM (SELECT + ss_quantity quantity, + ss_list_price list_price + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2 + UNION ALL + SELECT + cs_quantity quantity, + cs_list_price list_price + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2 + UNION ALL + SELECT + ws_quantity quantity, + ws_list_price list_price + FROM web_sales, date_dim + WHERE ws_sold_date_sk = d_date_sk AND d_year BETWEEN 1999 AND 1999 + 2) x) +SELECT * +FROM + (SELECT + 'store' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ss_quantity * ss_list_price) sales, + count(*) number_sales + FROM store_sales, item, date_dim + WHERE ss_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_year = 1999 + 1 AND d_moy = 12 AND d_dom = 11) + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales + FROM avg_sales)) this_year, + (SELECT + 'store' channel, + i_brand_id, + i_class_id, + i_category_id, + sum(ss_quantity * ss_list_price) sales, + count(*) number_sales + FROM store_sales, item, date_dim + WHERE ss_item_sk IN (SELECT ss_item_sk + FROM cross_items) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_year = 1999 AND d_moy = 12 AND d_dom = 11) + GROUP BY i_brand_id, i_class_id, i_category_id + HAVING sum(ss_quantity * ss_list_price) > (SELECT average_sales + FROM avg_sales)) last_year +WHERE this_year.i_brand_id = last_year.i_brand_id + AND this_year.i_class_id = last_year.i_class_id + AND this_year.i_category_id = last_year.i_category_id +ORDER BY this_year.channel, this_year.i_brand_id, this_year.i_class_id, this_year.i_category_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q15.sql b/plugin-spark/src/test/resources/tpcds/q15.sql new file mode 100755 index 0000000000..b8182e23b0 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q15.sql @@ -0,0 +1,15 @@ +SELECT + ca_zip, + sum(cs_sales_price) +FROM catalog_sales, customer, customer_address, date_dim +WHERE cs_bill_customer_sk = c_customer_sk + AND c_current_addr_sk = ca_address_sk + AND (substr(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', + '85392', '85460', '80348', '81792') + OR ca_state IN ('CA', 'WA', 'GA') + OR cs_sales_price > 500) + AND cs_sold_date_sk = d_date_sk + AND d_qoy = 2 AND d_year = 2001 +GROUP BY ca_zip +ORDER BY ca_zip +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q16.sql b/plugin-spark/src/test/resources/tpcds/q16.sql new file mode 100755 index 0000000000..732ad0d848 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q16.sql @@ -0,0 +1,23 @@ +SELECT + count(DISTINCT cs_order_number) AS `order count `, + sum(cs_ext_ship_cost) AS `total shipping cost `, + sum(cs_net_profit) AS `total net profit ` +FROM + catalog_sales cs1, date_dim, customer_address, call_center +WHERE + d_date BETWEEN '2002-02-01' AND (CAST('2002-02-01' AS DATE) + INTERVAL 60 days) + AND cs1.cs_ship_date_sk = d_date_sk + AND cs1.cs_ship_addr_sk = ca_address_sk + AND ca_state = 'GA' + AND cs1.cs_call_center_sk = cc_call_center_sk + AND cc_county IN + ('Williamson County', 'Williamson County', 'Williamson County', 'Williamson County', 'Williamson County') + AND EXISTS(SELECT * + FROM catalog_sales cs2 + WHERE cs1.cs_order_number = cs2.cs_order_number + AND cs1.cs_warehouse_sk <> cs2.cs_warehouse_sk) + AND NOT EXISTS(SELECT * + FROM catalog_returns cr1 + WHERE cs1.cs_order_number = cr1.cr_order_number) +ORDER BY count(DISTINCT cs_order_number) +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q17.sql b/plugin-spark/src/test/resources/tpcds/q17.sql new file mode 100755 index 0000000000..4d647f7956 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q17.sql @@ -0,0 +1,33 @@ +SELECT + i_item_id, + i_item_desc, + s_state, + count(ss_quantity) AS store_sales_quantitycount, + avg(ss_quantity) AS store_sales_quantityave, + stddev_samp(ss_quantity) AS store_sales_quantitystdev, + stddev_samp(ss_quantity) / avg(ss_quantity) AS store_sales_quantitycov, + count(sr_return_quantity) as_store_returns_quantitycount, + avg(sr_return_quantity) as_store_returns_quantityave, + stddev_samp(sr_return_quantity) as_store_returns_quantitystdev, + stddev_samp(sr_return_quantity) / avg(sr_return_quantity) AS store_returns_quantitycov, + count(cs_quantity) AS catalog_sales_quantitycount, + avg(cs_quantity) AS catalog_sales_quantityave, + stddev_samp(cs_quantity) / avg(cs_quantity) AS catalog_sales_quantitystdev, + stddev_samp(cs_quantity) / avg(cs_quantity) AS catalog_sales_quantitycov +FROM store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, date_dim d3, store, item +WHERE d1.d_quarter_name = '2001Q1' + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND ss_customer_sk = sr_customer_sk + AND ss_item_sk = sr_item_sk + AND ss_ticket_number = sr_ticket_number + AND sr_returned_date_sk = d2.d_date_sk + AND d2.d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3') + AND sr_customer_sk = cs_bill_customer_sk + AND sr_item_sk = cs_item_sk + AND cs_sold_date_sk = d3.d_date_sk + AND d3.d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3') +GROUP BY i_item_id, i_item_desc, s_state +ORDER BY i_item_id, i_item_desc, s_state +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q18.sql b/plugin-spark/src/test/resources/tpcds/q18.sql new file mode 100755 index 0000000000..4055c80fde --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q18.sql @@ -0,0 +1,28 @@ +SELECT + i_item_id, + ca_country, + ca_state, + ca_county, + avg(cast(cs_quantity AS DECIMAL(12, 2))) agg1, + avg(cast(cs_list_price AS DECIMAL(12, 2))) agg2, + avg(cast(cs_coupon_amt AS DECIMAL(12, 2))) agg3, + avg(cast(cs_sales_price AS DECIMAL(12, 2))) agg4, + avg(cast(cs_net_profit AS DECIMAL(12, 2))) agg5, + avg(cast(c_birth_year AS DECIMAL(12, 2))) agg6, + avg(cast(cd1.cd_dep_count AS DECIMAL(12, 2))) agg7 +FROM catalog_sales, customer_demographics cd1, + customer_demographics cd2, customer, customer_address, date_dim, item +WHERE cs_sold_date_sk = d_date_sk AND + cs_item_sk = i_item_sk AND + cs_bill_cdemo_sk = cd1.cd_demo_sk AND + cs_bill_customer_sk = c_customer_sk AND + cd1.cd_gender = 'F' AND + cd1.cd_education_status = 'Unknown' AND + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_addr_sk = ca_address_sk AND + c_birth_month IN (1, 6, 8, 9, 12, 2) AND + d_year = 1998 AND + ca_state IN ('MS', 'IN', 'ND', 'OK', 'NM', 'VA', 'MS') +GROUP BY ROLLUP (i_item_id, ca_country, ca_state, ca_county) +ORDER BY ca_country, ca_state, ca_county, i_item_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q19.sql b/plugin-spark/src/test/resources/tpcds/q19.sql new file mode 100755 index 0000000000..e38ab7f268 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q19.sql @@ -0,0 +1,19 @@ +SELECT + i_brand_id brand_id, + i_brand brand, + i_manufact_id, + i_manufact, + sum(ss_ext_sales_price) ext_price +FROM date_dim, store_sales, item, customer, customer_address, store +WHERE d_date_sk = ss_sold_date_sk + AND ss_item_sk = i_item_sk + AND i_manager_id = 8 + AND d_moy = 11 + AND d_year = 1998 + AND ss_customer_sk = c_customer_sk + AND c_current_addr_sk = ca_address_sk + AND substr(ca_zip, 1, 5) <> substr(s_zip, 1, 5) + AND ss_store_sk = s_store_sk +GROUP BY i_brand, i_brand_id, i_manufact_id, i_manufact +ORDER BY ext_price DESC, brand, brand_id, i_manufact_id, i_manufact +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q2.sql b/plugin-spark/src/test/resources/tpcds/q2.sql new file mode 100755 index 0000000000..52c0e90c46 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q2.sql @@ -0,0 +1,81 @@ +WITH wscs AS +( SELECT + sold_date_sk, + sales_price + FROM (SELECT + ws_sold_date_sk sold_date_sk, + ws_ext_sales_price sales_price + FROM web_sales) x + UNION ALL + (SELECT + cs_sold_date_sk sold_date_sk, + cs_ext_sales_price sales_price + FROM catalog_sales)), + wswscs AS + ( SELECT + d_week_seq, + sum(CASE WHEN (d_day_name = 'Sunday') + THEN sales_price + ELSE NULL END) + sun_sales, + sum(CASE WHEN (d_day_name = 'Monday') + THEN sales_price + ELSE NULL END) + mon_sales, + sum(CASE WHEN (d_day_name = 'Tuesday') + THEN sales_price + ELSE NULL END) + tue_sales, + sum(CASE WHEN (d_day_name = 'Wednesday') + THEN sales_price + ELSE NULL END) + wed_sales, + sum(CASE WHEN (d_day_name = 'Thursday') + THEN sales_price + ELSE NULL END) + thu_sales, + sum(CASE WHEN (d_day_name = 'Friday') + THEN sales_price + ELSE NULL END) + fri_sales, + sum(CASE WHEN (d_day_name = 'Saturday') + THEN sales_price + ELSE NULL END) + sat_sales + FROM wscs, date_dim + WHERE d_date_sk = sold_date_sk + GROUP BY d_week_seq) +SELECT + d_week_seq1, + round(sun_sales1 / sun_sales2, 2), + round(mon_sales1 / mon_sales2, 2), + round(tue_sales1 / tue_sales2, 2), + round(wed_sales1 / wed_sales2, 2), + round(thu_sales1 / thu_sales2, 2), + round(fri_sales1 / fri_sales2, 2), + round(sat_sales1 / sat_sales2, 2) +FROM + (SELECT + wswscs.d_week_seq d_week_seq1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 + FROM wswscs, date_dim + WHERE date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001) y, + (SELECT + wswscs.d_week_seq d_week_seq2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + FROM wswscs, date_dim + WHERE date_dim.d_week_seq = wswscs.d_week_seq AND d_year = 2001 + 1) z +WHERE d_week_seq1 = d_week_seq2 - 53 +ORDER BY d_week_seq1 diff --git a/plugin-spark/src/test/resources/tpcds/q20.sql b/plugin-spark/src/test/resources/tpcds/q20.sql new file mode 100755 index 0000000000..7ac6c7a75d --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q20.sql @@ -0,0 +1,18 @@ +SELECT + i_item_desc, + i_category, + i_class, + i_current_price, + sum(cs_ext_sales_price) AS itemrevenue, + sum(cs_ext_sales_price) * 100 / sum(sum(cs_ext_sales_price)) + OVER + (PARTITION BY i_class) AS revenueratio +FROM catalog_sales, item, date_dim +WHERE cs_item_sk = i_item_sk + AND i_category IN ('Sports', 'Books', 'Home') + AND cs_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('1999-02-22' AS DATE) +AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) +GROUP BY i_item_id, i_item_desc, i_category, i_class, i_current_price +ORDER BY i_category, i_class, i_item_id, i_item_desc, revenueratio +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q21.sql b/plugin-spark/src/test/resources/tpcds/q21.sql new file mode 100755 index 0000000000..550881143f --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q21.sql @@ -0,0 +1,25 @@ +SELECT * +FROM ( + SELECT + w_warehouse_name, + i_item_id, + sum(CASE WHEN (cast(d_date AS DATE) < cast('2000-03-11' AS DATE)) + THEN inv_quantity_on_hand + ELSE 0 END) AS inv_before, + sum(CASE WHEN (cast(d_date AS DATE) >= cast('2000-03-11' AS DATE)) + THEN inv_quantity_on_hand + ELSE 0 END) AS inv_after + FROM inventory, warehouse, item, date_dim + WHERE i_current_price BETWEEN 0.99 AND 1.49 + AND i_item_sk = inv_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND inv_date_sk = d_date_sk + AND d_date BETWEEN (cast('2000-03-11' AS DATE) - INTERVAL 30 days) + AND (cast('2000-03-11' AS DATE) + INTERVAL 30 days) + GROUP BY w_warehouse_name, i_item_id) x +WHERE (CASE WHEN inv_before > 0 + THEN inv_after / inv_before + ELSE NULL + END) BETWEEN 2.0 / 3.0 AND 3.0 / 2.0 +ORDER BY w_warehouse_name, i_item_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q22.sql b/plugin-spark/src/test/resources/tpcds/q22.sql new file mode 100755 index 0000000000..add3b41f7c --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q22.sql @@ -0,0 +1,14 @@ +SELECT + i_product_name, + i_brand, + i_class, + i_category, + avg(inv_quantity_on_hand) qoh +FROM inventory, date_dim, item, warehouse +WHERE inv_date_sk = d_date_sk + AND inv_item_sk = i_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 +GROUP BY ROLLUP (i_product_name, i_brand, i_class, i_category) +ORDER BY qoh, i_product_name, i_brand, i_class, i_category +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q23a.sql b/plugin-spark/src/test/resources/tpcds/q23a.sql new file mode 100755 index 0000000000..37791f6433 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q23a.sql @@ -0,0 +1,53 @@ +WITH frequent_ss_items AS +(SELECT + substr(i_item_desc, 1, 30) itemdesc, + i_item_sk item_sk, + d_date solddate, + count(*) cnt + FROM store_sales, date_dim, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY substr(i_item_desc, 1, 30), i_item_sk, d_date + HAVING count(*) > 4), + max_store_sales AS + (SELECT max(csales) tpcds_cmax + FROM (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) csales + FROM store_sales, customer, date_dim + WHERE ss_customer_sk = c_customer_sk + AND ss_sold_date_sk = d_date_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY c_customer_sk) x), + best_ss_customer AS + (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) ssales + FROM store_sales, customer + WHERE ss_customer_sk = c_customer_sk + GROUP BY c_customer_sk + HAVING sum(ss_quantity * ss_sales_price) > (50 / 100.0) * + (SELECT * + FROM max_store_sales)) +SELECT sum(sales) +FROM ((SELECT cs_quantity * cs_list_price sales +FROM catalog_sales, date_dim +WHERE d_year = 2000 + AND d_moy = 2 + AND cs_sold_date_sk = d_date_sk + AND cs_item_sk IN (SELECT item_sk +FROM frequent_ss_items) + AND cs_bill_customer_sk IN (SELECT c_customer_sk +FROM best_ss_customer)) + UNION ALL + (SELECT ws_quantity * ws_list_price sales + FROM web_sales, date_dim + WHERE d_year = 2000 + AND d_moy = 2 + AND ws_sold_date_sk = d_date_sk + AND ws_item_sk IN (SELECT item_sk + FROM frequent_ss_items) + AND ws_bill_customer_sk IN (SELECT c_customer_sk + FROM best_ss_customer))) y +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q23b.sql b/plugin-spark/src/test/resources/tpcds/q23b.sql new file mode 100755 index 0000000000..01150197af --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q23b.sql @@ -0,0 +1,68 @@ +WITH frequent_ss_items AS +(SELECT + substr(i_item_desc, 1, 30) itemdesc, + i_item_sk item_sk, + d_date solddate, + count(*) cnt + FROM store_sales, date_dim, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY substr(i_item_desc, 1, 30), i_item_sk, d_date + HAVING count(*) > 4), + max_store_sales AS + (SELECT max(csales) tpcds_cmax + FROM (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) csales + FROM store_sales, customer, date_dim + WHERE ss_customer_sk = c_customer_sk + AND ss_sold_date_sk = d_date_sk + AND d_year IN (2000, 2000 + 1, 2000 + 2, 2000 + 3) + GROUP BY c_customer_sk) x), + best_ss_customer AS + (SELECT + c_customer_sk, + sum(ss_quantity * ss_sales_price) ssales + FROM store_sales + , customer + WHERE ss_customer_sk = c_customer_sk + GROUP BY c_customer_sk + HAVING sum(ss_quantity * ss_sales_price) > (50 / 100.0) * + (SELECT * + FROM max_store_sales)) +SELECT + c_last_name, + c_first_name, + sales +FROM ((SELECT + c_last_name, + c_first_name, + sum(cs_quantity * cs_list_price) sales +FROM catalog_sales, customer, date_dim +WHERE d_year = 2000 + AND d_moy = 2 + AND cs_sold_date_sk = d_date_sk + AND cs_item_sk IN (SELECT item_sk +FROM frequent_ss_items) + AND cs_bill_customer_sk IN (SELECT c_customer_sk +FROM best_ss_customer) + AND cs_bill_customer_sk = c_customer_sk +GROUP BY c_last_name, c_first_name) + UNION ALL + (SELECT + c_last_name, + c_first_name, + sum(ws_quantity * ws_list_price) sales + FROM web_sales, customer, date_dim + WHERE d_year = 2000 + AND d_moy = 2 + AND ws_sold_date_sk = d_date_sk + AND ws_item_sk IN (SELECT item_sk + FROM frequent_ss_items) + AND ws_bill_customer_sk IN (SELECT c_customer_sk + FROM best_ss_customer) + AND ws_bill_customer_sk = c_customer_sk + GROUP BY c_last_name, c_first_name)) y +ORDER BY c_last_name, c_first_name, sales +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q24a.sql b/plugin-spark/src/test/resources/tpcds/q24a.sql new file mode 100755 index 0000000000..bcc1894866 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q24a.sql @@ -0,0 +1,34 @@ +WITH ssales AS +(SELECT + c_last_name, + c_first_name, + s_store_name, + ca_state, + s_state, + i_color, + i_current_price, + i_manager_id, + i_units, + i_size, + sum(ss_net_paid) netpaid + FROM store_sales, store_returns, store, item, customer, customer_address + WHERE ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk + AND ss_customer_sk = c_customer_sk + AND ss_item_sk = i_item_sk + AND ss_store_sk = s_store_sk + AND c_birth_country = upper(ca_country) + AND s_zip = ca_zip + AND s_market_id = 8 + GROUP BY c_last_name, c_first_name, s_store_name, ca_state, s_state, i_color, + i_current_price, i_manager_id, i_units, i_size) +SELECT + c_last_name, + c_first_name, + s_store_name, + sum(netpaid) paid +FROM ssales +WHERE i_color = 'pale' +GROUP BY c_last_name, c_first_name, s_store_name +HAVING sum(netpaid) > (SELECT 0.05 * avg(netpaid) +FROM ssales) diff --git a/plugin-spark/src/test/resources/tpcds/q24b.sql b/plugin-spark/src/test/resources/tpcds/q24b.sql new file mode 100755 index 0000000000..830eb670bc --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q24b.sql @@ -0,0 +1,34 @@ +WITH ssales AS +(SELECT + c_last_name, + c_first_name, + s_store_name, + ca_state, + s_state, + i_color, + i_current_price, + i_manager_id, + i_units, + i_size, + sum(ss_net_paid) netpaid + FROM store_sales, store_returns, store, item, customer, customer_address + WHERE ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk + AND ss_customer_sk = c_customer_sk + AND ss_item_sk = i_item_sk + AND ss_store_sk = s_store_sk + AND c_birth_country = upper(ca_country) + AND s_zip = ca_zip + AND s_market_id = 8 + GROUP BY c_last_name, c_first_name, s_store_name, ca_state, s_state, + i_color, i_current_price, i_manager_id, i_units, i_size) +SELECT + c_last_name, + c_first_name, + s_store_name, + sum(netpaid) paid +FROM ssales +WHERE i_color = 'chiffon' +GROUP BY c_last_name, c_first_name, s_store_name +HAVING sum(netpaid) > (SELECT 0.05 * avg(netpaid) +FROM ssales) diff --git a/plugin-spark/src/test/resources/tpcds/q25.sql b/plugin-spark/src/test/resources/tpcds/q25.sql new file mode 100755 index 0000000000..a4d78a3c56 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q25.sql @@ -0,0 +1,33 @@ +SELECT + i_item_id, + i_item_desc, + s_store_id, + s_store_name, + sum(ss_net_profit) AS store_sales_profit, + sum(sr_net_loss) AS store_returns_loss, + sum(cs_net_profit) AS catalog_sales_profit +FROM + store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, date_dim d3, + store, item +WHERE + d1.d_moy = 4 + AND d1.d_year = 2001 + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND ss_customer_sk = sr_customer_sk + AND ss_item_sk = sr_item_sk + AND ss_ticket_number = sr_ticket_number + AND sr_returned_date_sk = d2.d_date_sk + AND d2.d_moy BETWEEN 4 AND 10 + AND d2.d_year = 2001 + AND sr_customer_sk = cs_bill_customer_sk + AND sr_item_sk = cs_item_sk + AND cs_sold_date_sk = d3.d_date_sk + AND d3.d_moy BETWEEN 4 AND 10 + AND d3.d_year = 2001 +GROUP BY + i_item_id, i_item_desc, s_store_id, s_store_name +ORDER BY + i_item_id, i_item_desc, s_store_id, s_store_name +LIMIT 100 \ No newline at end of file diff --git a/plugin-spark/src/test/resources/tpcds/q26.sql b/plugin-spark/src/test/resources/tpcds/q26.sql new file mode 100755 index 0000000000..6d395a1d79 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q26.sql @@ -0,0 +1,19 @@ +SELECT + i_item_id, + avg(cs_quantity) agg1, + avg(cs_list_price) agg2, + avg(cs_coupon_amt) agg3, + avg(cs_sales_price) agg4 +FROM catalog_sales, customer_demographics, date_dim, item, promotion +WHERE cs_sold_date_sk = d_date_sk AND + cs_item_sk = i_item_sk AND + cs_bill_cdemo_sk = cd_demo_sk AND + cs_promo_sk = p_promo_sk AND + cd_gender = 'M' AND + cd_marital_status = 'S' AND + cd_education_status = 'College' AND + (p_channel_email = 'N' OR p_channel_event = 'N') AND + d_year = 2000 +GROUP BY i_item_id +ORDER BY i_item_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q27.sql b/plugin-spark/src/test/resources/tpcds/q27.sql new file mode 100755 index 0000000000..b0e2fd95fd --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q27.sql @@ -0,0 +1,21 @@ +SELECT + i_item_id, + s_state, + grouping(s_state) g_state, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 +FROM store_sales, customer_demographics, date_dim, store, item +WHERE ss_sold_date_sk = d_date_sk AND + ss_item_sk = i_item_sk AND + ss_store_sk = s_store_sk AND + ss_cdemo_sk = cd_demo_sk AND + cd_gender = 'M' AND + cd_marital_status = 'S' AND + cd_education_status = 'College' AND + d_year = 2002 AND + s_state IN ('TN', 'TN', 'TN', 'TN', 'TN', 'TN') +GROUP BY ROLLUP (i_item_id, s_state) +ORDER BY i_item_id, s_state +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q28.sql b/plugin-spark/src/test/resources/tpcds/q28.sql new file mode 100755 index 0000000000..f34c2bb0e3 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q28.sql @@ -0,0 +1,56 @@ +SELECT * +FROM (SELECT + avg(ss_list_price) B1_LP, + count(ss_list_price) B1_CNT, + count(DISTINCT ss_list_price) B1_CNTD +FROM store_sales +WHERE ss_quantity BETWEEN 0 AND 5 + AND (ss_list_price BETWEEN 8 AND 8 + 10 + OR ss_coupon_amt BETWEEN 459 AND 459 + 1000 + OR ss_wholesale_cost BETWEEN 57 AND 57 + 20)) B1, + (SELECT + avg(ss_list_price) B2_LP, + count(ss_list_price) B2_CNT, + count(DISTINCT ss_list_price) B2_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 6 AND 10 + AND (ss_list_price BETWEEN 90 AND 90 + 10 + OR ss_coupon_amt BETWEEN 2323 AND 2323 + 1000 + OR ss_wholesale_cost BETWEEN 31 AND 31 + 20)) B2, + (SELECT + avg(ss_list_price) B3_LP, + count(ss_list_price) B3_CNT, + count(DISTINCT ss_list_price) B3_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 11 AND 15 + AND (ss_list_price BETWEEN 142 AND 142 + 10 + OR ss_coupon_amt BETWEEN 12214 AND 12214 + 1000 + OR ss_wholesale_cost BETWEEN 79 AND 79 + 20)) B3, + (SELECT + avg(ss_list_price) B4_LP, + count(ss_list_price) B4_CNT, + count(DISTINCT ss_list_price) B4_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 16 AND 20 + AND (ss_list_price BETWEEN 135 AND 135 + 10 + OR ss_coupon_amt BETWEEN 6071 AND 6071 + 1000 + OR ss_wholesale_cost BETWEEN 38 AND 38 + 20)) B4, + (SELECT + avg(ss_list_price) B5_LP, + count(ss_list_price) B5_CNT, + count(DISTINCT ss_list_price) B5_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 25 + AND (ss_list_price BETWEEN 122 AND 122 + 10 + OR ss_coupon_amt BETWEEN 836 AND 836 + 1000 + OR ss_wholesale_cost BETWEEN 17 AND 17 + 20)) B5, + (SELECT + avg(ss_list_price) B6_LP, + count(ss_list_price) B6_CNT, + count(DISTINCT ss_list_price) B6_CNTD + FROM store_sales + WHERE ss_quantity BETWEEN 26 AND 30 + AND (ss_list_price BETWEEN 154 AND 154 + 10 + OR ss_coupon_amt BETWEEN 7326 AND 7326 + 1000 + OR ss_wholesale_cost BETWEEN 7 AND 7 + 20)) B6 +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q29.sql b/plugin-spark/src/test/resources/tpcds/q29.sql new file mode 100755 index 0000000000..3f1fd553f6 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q29.sql @@ -0,0 +1,32 @@ +SELECT + i_item_id, + i_item_desc, + s_store_id, + s_store_name, + sum(ss_quantity) AS store_sales_quantity, + sum(sr_return_quantity) AS store_returns_quantity, + sum(cs_quantity) AS catalog_sales_quantity +FROM + store_sales, store_returns, catalog_sales, date_dim d1, date_dim d2, + date_dim d3, store, item +WHERE + d1.d_moy = 9 + AND d1.d_year = 1999 + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND ss_customer_sk = sr_customer_sk + AND ss_item_sk = sr_item_sk + AND ss_ticket_number = sr_ticket_number + AND sr_returned_date_sk = d2.d_date_sk + AND d2.d_moy BETWEEN 9 AND 9 + 3 + AND d2.d_year = 1999 + AND sr_customer_sk = cs_bill_customer_sk + AND sr_item_sk = cs_item_sk + AND cs_sold_date_sk = d3.d_date_sk + AND d3.d_year IN (1999, 1999 + 1, 1999 + 2) +GROUP BY + i_item_id, i_item_desc, s_store_id, s_store_name +ORDER BY + i_item_id, i_item_desc, s_store_id, s_store_name +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q3.sql b/plugin-spark/src/test/resources/tpcds/q3.sql new file mode 100755 index 0000000000..181509df9d --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q3.sql @@ -0,0 +1,13 @@ +SELECT + dt.d_year, + item.i_brand_id brand_id, + item.i_brand brand, + SUM(ss_ext_sales_price) sum_agg +FROM date_dim dt, store_sales, item +WHERE dt.d_date_sk = store_sales.ss_sold_date_sk + AND store_sales.ss_item_sk = item.i_item_sk + AND item.i_manufact_id = 128 + AND dt.d_moy = 11 +GROUP BY dt.d_year, item.i_brand, item.i_brand_id +ORDER BY dt.d_year, sum_agg DESC, brand_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q30.sql b/plugin-spark/src/test/resources/tpcds/q30.sql new file mode 100755 index 0000000000..986bef566d --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q30.sql @@ -0,0 +1,35 @@ +WITH customer_total_return AS +(SELECT + wr_returning_customer_sk AS ctr_customer_sk, + ca_state AS ctr_state, + sum(wr_return_amt) AS ctr_total_return + FROM web_returns, date_dim, customer_address + WHERE wr_returned_date_sk = d_date_sk + AND d_year = 2002 + AND wr_returning_addr_sk = ca_address_sk + GROUP BY wr_returning_customer_sk, ca_state) +SELECT + c_customer_id, + c_salutation, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_day, + c_birth_month, + c_birth_year, + c_birth_country, + c_login, + c_email_address, + c_last_review_date, + ctr_total_return +FROM customer_total_return ctr1, customer_address, customer +WHERE ctr1.ctr_total_return > (SELECT avg(ctr_total_return) * 1.2 +FROM customer_total_return ctr2 +WHERE ctr1.ctr_state = ctr2.ctr_state) + AND ca_address_sk = c_current_addr_sk + AND ca_state = 'GA' + AND ctr1.ctr_customer_sk = c_customer_sk +ORDER BY c_customer_id, c_salutation, c_first_name, c_last_name, c_preferred_cust_flag + , c_birth_day, c_birth_month, c_birth_year, c_birth_country, c_login, c_email_address + , c_last_review_date, ctr_total_return +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q31.sql b/plugin-spark/src/test/resources/tpcds/q31.sql new file mode 100755 index 0000000000..3e543d5436 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q31.sql @@ -0,0 +1,60 @@ +WITH ss AS +(SELECT + ca_county, + d_qoy, + d_year, + sum(ss_ext_sales_price) AS store_sales + FROM store_sales, date_dim, customer_address + WHERE ss_sold_date_sk = d_date_sk + AND ss_addr_sk = ca_address_sk + GROUP BY ca_county, d_qoy, d_year), + ws AS + (SELECT + ca_county, + d_qoy, + d_year, + sum(ws_ext_sales_price) AS web_sales + FROM web_sales, date_dim, customer_address + WHERE ws_sold_date_sk = d_date_sk + AND ws_bill_addr_sk = ca_address_sk + GROUP BY ca_county, d_qoy, d_year) +SELECT + ss1.ca_county, + ss1.d_year, + ws2.web_sales / ws1.web_sales web_q1_q2_increase, + ss2.store_sales / ss1.store_sales store_q1_q2_increase, + ws3.web_sales / ws2.web_sales web_q2_q3_increase, + ss3.store_sales / ss2.store_sales store_q2_q3_increase +FROM + ss ss1, ss ss2, ss ss3, ws ws1, ws ws2, ws ws3 +WHERE + ss1.d_qoy = 1 + AND ss1.d_year = 2000 + AND ss1.ca_county = ss2.ca_county + AND ss2.d_qoy = 2 + AND ss2.d_year = 2000 + AND ss2.ca_county = ss3.ca_county + AND ss3.d_qoy = 3 + AND ss3.d_year = 2000 + AND ss1.ca_county = ws1.ca_county + AND ws1.d_qoy = 1 + AND ws1.d_year = 2000 + AND ws1.ca_county = ws2.ca_county + AND ws2.d_qoy = 2 + AND ws2.d_year = 2000 + AND ws1.ca_county = ws3.ca_county + AND ws3.d_qoy = 3 + AND ws3.d_year = 2000 + AND CASE WHEN ws1.web_sales > 0 + THEN ws2.web_sales / ws1.web_sales + ELSE NULL END + > CASE WHEN ss1.store_sales > 0 + THEN ss2.store_sales / ss1.store_sales + ELSE NULL END + AND CASE WHEN ws2.web_sales > 0 + THEN ws3.web_sales / ws2.web_sales + ELSE NULL END + > CASE WHEN ss2.store_sales > 0 + THEN ss3.store_sales / ss2.store_sales + ELSE NULL END +ORDER BY ss1.ca_county diff --git a/plugin-spark/src/test/resources/tpcds/q32.sql b/plugin-spark/src/test/resources/tpcds/q32.sql new file mode 100755 index 0000000000..1a907961e7 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q32.sql @@ -0,0 +1,15 @@ +SELECT 1 AS `excess discount amount ` +FROM + catalog_sales, item, date_dim +WHERE + i_manufact_id = 977 + AND i_item_sk = cs_item_sk + AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + interval 90 days) + AND d_date_sk = cs_sold_date_sk + AND cs_ext_discount_amt > ( + SELECT 1.3 * avg(cs_ext_discount_amt) + FROM catalog_sales, date_dim + WHERE cs_item_sk = i_item_sk + AND d_date BETWEEN '2000-01-27]' AND (cast('2000-01-27' AS DATE) + interval 90 days) + AND d_date_sk = cs_sold_date_sk) +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q33.sql b/plugin-spark/src/test/resources/tpcds/q33.sql new file mode 100755 index 0000000000..d24856aa5c --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q33.sql @@ -0,0 +1,65 @@ +WITH ss AS ( + SELECT + i_manufact_id, + sum(ss_ext_sales_price) total_sales + FROM + store_sales, date_dim, customer_address, item + WHERE + i_manufact_id IN (SELECT i_manufact_id + FROM item + WHERE i_category IN ('Electronics')) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 5 + AND ss_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_manufact_id), cs AS +(SELECT + i_manufact_id, + sum(cs_ext_sales_price) total_sales + FROM catalog_sales, date_dim, customer_address, item + WHERE + i_manufact_id IN ( + SELECT i_manufact_id + FROM item + WHERE + i_category IN ('Electronics')) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 5 + AND cs_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_manufact_id), + ws AS ( + SELECT + i_manufact_id, + sum(ws_ext_sales_price) total_sales + FROM + web_sales, date_dim, customer_address, item + WHERE + i_manufact_id IN (SELECT i_manufact_id + FROM item + WHERE i_category IN ('Electronics')) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 5 + AND ws_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_manufact_id) +SELECT + i_manufact_id, + sum(total_sales) total_sales +FROM (SELECT * + FROM ss + UNION ALL + SELECT * + FROM cs + UNION ALL + SELECT * + FROM ws) tmp1 +GROUP BY i_manufact_id +ORDER BY total_sales +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q34.sql b/plugin-spark/src/test/resources/tpcds/q34.sql new file mode 100755 index 0000000000..33396bf16e --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q34.sql @@ -0,0 +1,32 @@ +SELECT + c_last_name, + c_first_name, + c_salutation, + c_preferred_cust_flag, + ss_ticket_number, + cnt +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + count(*) cnt + FROM store_sales, date_dim, store, household_demographics + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND (date_dim.d_dom BETWEEN 1 AND 3 OR date_dim.d_dom BETWEEN 25 AND 28) + AND (household_demographics.hd_buy_potential = '>10000' OR + household_demographics.hd_buy_potential = 'unknown') + AND household_demographics.hd_vehicle_count > 0 + AND (CASE WHEN household_demographics.hd_vehicle_count > 0 + THEN household_demographics.hd_dep_count / household_demographics.hd_vehicle_count + ELSE NULL + END) > 1.2 + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_county IN + ('Williamson County', 'Williamson County', 'Williamson County', 'Williamson County', + 'Williamson County', 'Williamson County', 'Williamson County', 'Williamson County') + GROUP BY ss_ticket_number, ss_customer_sk) dn, customer +WHERE ss_customer_sk = c_customer_sk + AND cnt BETWEEN 15 AND 20 +ORDER BY c_last_name, c_first_name, c_salutation, c_preferred_cust_flag DESC diff --git a/plugin-spark/src/test/resources/tpcds/q35.sql b/plugin-spark/src/test/resources/tpcds/q35.sql new file mode 100755 index 0000000000..cfe4342d8b --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q35.sql @@ -0,0 +1,46 @@ +SELECT + ca_state, + cd_gender, + cd_marital_status, + count(*) cnt1, + min(cd_dep_count), + max(cd_dep_count), + avg(cd_dep_count), + cd_dep_employed_count, + count(*) cnt2, + min(cd_dep_employed_count), + max(cd_dep_employed_count), + avg(cd_dep_employed_count), + cd_dep_college_count, + count(*) cnt3, + min(cd_dep_college_count), + max(cd_dep_college_count), + avg(cd_dep_college_count) +FROM + customer c, customer_address ca, customer_demographics +WHERE + c.c_current_addr_sk = ca.ca_address_sk AND + cd_demo_sk = c.c_current_cdemo_sk AND + exists(SELECT * + FROM store_sales, date_dim + WHERE c.c_customer_sk = ss_customer_sk AND + ss_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_qoy < 4) AND + (exists(SELECT * + FROM web_sales, date_dim + WHERE c.c_customer_sk = ws_bill_customer_sk AND + ws_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_qoy < 4) OR + exists(SELECT * + FROM catalog_sales, date_dim + WHERE c.c_customer_sk = cs_ship_customer_sk AND + cs_sold_date_sk = d_date_sk AND + d_year = 2002 AND + d_qoy < 4)) +GROUP BY ca_state, cd_gender, cd_marital_status, cd_dep_count, + cd_dep_employed_count, cd_dep_college_count +ORDER BY ca_state, cd_gender, cd_marital_status, cd_dep_count, + cd_dep_employed_count, cd_dep_college_count +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q36.sql b/plugin-spark/src/test/resources/tpcds/q36.sql new file mode 100755 index 0000000000..a8f93df76a --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q36.sql @@ -0,0 +1,26 @@ +SELECT + sum(ss_net_profit) / sum(ss_ext_sales_price) AS gross_margin, + i_category, + i_class, + grouping(i_category) + grouping(i_class) AS lochierarchy, + rank() + OVER ( + PARTITION BY grouping(i_category) + grouping(i_class), + CASE WHEN grouping(i_class) = 0 + THEN i_category END + ORDER BY sum(ss_net_profit) / sum(ss_ext_sales_price) ASC) AS rank_within_parent +FROM + store_sales, date_dim d1, item, store +WHERE + d1.d_year = 2001 + AND d1.d_date_sk = ss_sold_date_sk + AND i_item_sk = ss_item_sk + AND s_store_sk = ss_store_sk + AND s_state IN ('TN', 'TN', 'TN', 'TN', 'TN', 'TN', 'TN', 'TN') +GROUP BY ROLLUP (i_category, i_class) +ORDER BY + lochierarchy DESC + , CASE WHEN lochierarchy = 0 + THEN i_category END + , rank_within_parent +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q37.sql b/plugin-spark/src/test/resources/tpcds/q37.sql new file mode 100755 index 0000000000..11b3821fa4 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q37.sql @@ -0,0 +1,15 @@ +SELECT + i_item_id, + i_item_desc, + i_current_price +FROM item, inventory, date_dim, catalog_sales +WHERE i_current_price BETWEEN 68 AND 68 + 30 + AND inv_item_sk = i_item_sk + AND d_date_sk = inv_date_sk + AND d_date BETWEEN cast('2000-02-01' AS DATE) AND (cast('2000-02-01' AS DATE) + INTERVAL 60 days) + AND i_manufact_id IN (677, 940, 694, 808) + AND inv_quantity_on_hand BETWEEN 100 AND 500 + AND cs_item_sk = i_item_sk +GROUP BY i_item_id, i_item_desc, i_current_price +ORDER BY i_item_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q38.sql b/plugin-spark/src/test/resources/tpcds/q38.sql new file mode 100755 index 0000000000..1c8d53ee2b --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q38.sql @@ -0,0 +1,30 @@ +SELECT count(*) +FROM ( + SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM store_sales, date_dim, customer + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + INTERSECT + SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM catalog_sales, date_dim, customer + WHERE catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + INTERSECT + SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM web_sales, date_dim, customer + WHERE web_sales.ws_sold_date_sk = date_dim.d_date_sk + AND web_sales.ws_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + ) hot_cust +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q39a.sql b/plugin-spark/src/test/resources/tpcds/q39a.sql new file mode 100755 index 0000000000..9fc4c1701c --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q39a.sql @@ -0,0 +1,47 @@ +WITH inv AS +(SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stdev, + mean, + CASE mean + WHEN 0 + THEN NULL + ELSE stdev / mean END cov + FROM (SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stddev_samp(inv_quantity_on_hand) stdev, + avg(inv_quantity_on_hand) mean + FROM inventory, item, warehouse, date_dim + WHERE inv_item_sk = i_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND inv_date_sk = d_date_sk + AND d_year = 2001 + GROUP BY w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo + WHERE CASE mean + WHEN 0 + THEN 0 + ELSE stdev / mean END > 1) +SELECT + inv1.w_warehouse_sk, + inv1.i_item_sk, + inv1.d_moy, + inv1.mean, + inv1.cov, + inv2.w_warehouse_sk, + inv2.i_item_sk, + inv2.d_moy, + inv2.mean, + inv2.cov +FROM inv inv1, inv inv2 +WHERE inv1.i_item_sk = inv2.i_item_sk + AND inv1.w_warehouse_sk = inv2.w_warehouse_sk + AND inv1.d_moy = 1 + AND inv2.d_moy = 1 + 1 +ORDER BY inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov + , inv2.d_moy, inv2.mean, inv2.cov diff --git a/plugin-spark/src/test/resources/tpcds/q39b.sql b/plugin-spark/src/test/resources/tpcds/q39b.sql new file mode 100755 index 0000000000..6f8493029f --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q39b.sql @@ -0,0 +1,48 @@ +WITH inv AS +(SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stdev, + mean, + CASE mean + WHEN 0 + THEN NULL + ELSE stdev / mean END cov + FROM (SELECT + w_warehouse_name, + w_warehouse_sk, + i_item_sk, + d_moy, + stddev_samp(inv_quantity_on_hand) stdev, + avg(inv_quantity_on_hand) mean + FROM inventory, item, warehouse, date_dim + WHERE inv_item_sk = i_item_sk + AND inv_warehouse_sk = w_warehouse_sk + AND inv_date_sk = d_date_sk + AND d_year = 2001 + GROUP BY w_warehouse_name, w_warehouse_sk, i_item_sk, d_moy) foo + WHERE CASE mean + WHEN 0 + THEN 0 + ELSE stdev / mean END > 1) +SELECT + inv1.w_warehouse_sk, + inv1.i_item_sk, + inv1.d_moy, + inv1.mean, + inv1.cov, + inv2.w_warehouse_sk, + inv2.i_item_sk, + inv2.d_moy, + inv2.mean, + inv2.cov +FROM inv inv1, inv inv2 +WHERE inv1.i_item_sk = inv2.i_item_sk + AND inv1.w_warehouse_sk = inv2.w_warehouse_sk + AND inv1.d_moy = 1 + AND inv2.d_moy = 1 + 1 + AND inv1.cov > 1.5 +ORDER BY inv1.w_warehouse_sk, inv1.i_item_sk, inv1.d_moy, inv1.mean, inv1.cov + , inv2.d_moy, inv2.mean, inv2.cov diff --git a/plugin-spark/src/test/resources/tpcds/q4.sql b/plugin-spark/src/test/resources/tpcds/q4.sql new file mode 100755 index 0000000000..b9f27fbc9a --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q4.sql @@ -0,0 +1,120 @@ +WITH year_total AS ( + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum(((ss_ext_list_price - ss_ext_wholesale_cost - ss_ext_discount_amt) + + ss_ext_sales_price) / 2) year_total, + 's' sale_type + FROM customer, store_sales, date_dim + WHERE c_customer_sk = ss_customer_sk AND ss_sold_date_sk = d_date_sk + GROUP BY c_customer_id, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_country, + c_login, + c_email_address, + d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum((((cs_ext_list_price - cs_ext_wholesale_cost - cs_ext_discount_amt) + + cs_ext_sales_price) / 2)) year_total, + 'c' sale_type + FROM customer, catalog_sales, date_dim + WHERE c_customer_sk = cs_bill_customer_sk AND cs_sold_date_sk = d_date_sk + GROUP BY c_customer_id, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_country, + c_login, + c_email_address, + d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + c_preferred_cust_flag customer_preferred_cust_flag, + c_birth_country customer_birth_country, + c_login customer_login, + c_email_address customer_email_address, + d_year dyear, + sum((((ws_ext_list_price - ws_ext_wholesale_cost - ws_ext_discount_amt) + ws_ext_sales_price) / + 2)) year_total, + 'w' sale_type + FROM customer, web_sales, date_dim + WHERE c_customer_sk = ws_bill_customer_sk AND ws_sold_date_sk = d_date_sk + GROUP BY c_customer_id, + c_first_name, + c_last_name, + c_preferred_cust_flag, + c_birth_country, + c_login, + c_email_address, + d_year) +SELECT + t_s_secyear.customer_id, + t_s_secyear.customer_first_name, + t_s_secyear.customer_last_name, + t_s_secyear.customer_preferred_cust_flag, + t_s_secyear.customer_birth_country, + t_s_secyear.customer_login, + t_s_secyear.customer_email_address +FROM year_total t_s_firstyear, year_total t_s_secyear, year_total t_c_firstyear, + year_total t_c_secyear, year_total t_w_firstyear, year_total t_w_secyear +WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id + AND t_s_firstyear.customer_id = t_c_secyear.customer_id + AND t_s_firstyear.customer_id = t_c_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_secyear.customer_id + AND t_s_firstyear.sale_type = 's' + AND t_c_firstyear.sale_type = 'c' + AND t_w_firstyear.sale_type = 'w' + AND t_s_secyear.sale_type = 's' + AND t_c_secyear.sale_type = 'c' + AND t_w_secyear.sale_type = 'w' + AND t_s_firstyear.dyear = 2001 + AND t_s_secyear.dyear = 2001 + 1 + AND t_c_firstyear.dyear = 2001 + AND t_c_secyear.dyear = 2001 + 1 + AND t_w_firstyear.dyear = 2001 + AND t_w_secyear.dyear = 2001 + 1 + AND t_s_firstyear.year_total > 0 + AND t_c_firstyear.year_total > 0 + AND t_w_firstyear.year_total > 0 + AND CASE WHEN t_c_firstyear.year_total > 0 + THEN t_c_secyear.year_total / t_c_firstyear.year_total + ELSE NULL END + > CASE WHEN t_s_firstyear.year_total > 0 + THEN t_s_secyear.year_total / t_s_firstyear.year_total + ELSE NULL END + AND CASE WHEN t_c_firstyear.year_total > 0 + THEN t_c_secyear.year_total / t_c_firstyear.year_total + ELSE NULL END + > CASE WHEN t_w_firstyear.year_total > 0 + THEN t_w_secyear.year_total / t_w_firstyear.year_total + ELSE NULL END +ORDER BY + t_s_secyear.customer_id, + t_s_secyear.customer_first_name, + t_s_secyear.customer_last_name, + t_s_secyear.customer_preferred_cust_flag, + t_s_secyear.customer_birth_country, + t_s_secyear.customer_login, + t_s_secyear.customer_email_address +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q40.sql b/plugin-spark/src/test/resources/tpcds/q40.sql new file mode 100755 index 0000000000..66d8b73ac1 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q40.sql @@ -0,0 +1,25 @@ +SELECT + w_state, + i_item_id, + sum(CASE WHEN (cast(d_date AS DATE) < cast('2000-03-11' AS DATE)) + THEN cs_sales_price - coalesce(cr_refunded_cash, 0) + ELSE 0 END) AS sales_before, + sum(CASE WHEN (cast(d_date AS DATE) >= cast('2000-03-11' AS DATE)) + THEN cs_sales_price - coalesce(cr_refunded_cash, 0) + ELSE 0 END) AS sales_after +FROM + catalog_sales + LEFT OUTER JOIN catalog_returns ON + (cs_order_number = cr_order_number + AND cs_item_sk = cr_item_sk) + , warehouse, item, date_dim +WHERE + i_current_price BETWEEN 0.99 AND 1.49 + AND i_item_sk = cs_item_sk + AND cs_warehouse_sk = w_warehouse_sk + AND cs_sold_date_sk = d_date_sk + AND d_date BETWEEN (cast('2000-03-11' AS DATE) - INTERVAL 30 days) + AND (cast('2000-03-11' AS DATE) + INTERVAL 30 days) +GROUP BY w_state, i_item_id +ORDER BY w_state, i_item_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q41.sql b/plugin-spark/src/test/resources/tpcds/q41.sql new file mode 100755 index 0000000000..25e317e0e2 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q41.sql @@ -0,0 +1,49 @@ +SELECT DISTINCT (i_product_name) +FROM item i1 +WHERE i_manufact_id BETWEEN 738 AND 738 + 40 + AND (SELECT count(*) AS item_cnt +FROM item +WHERE (i_manufact = i1.i_manufact AND + ((i_category = 'Women' AND + (i_color = 'powder' OR i_color = 'khaki') AND + (i_units = 'Ounce' OR i_units = 'Oz') AND + (i_size = 'medium' OR i_size = 'extra large') + ) OR + (i_category = 'Women' AND + (i_color = 'brown' OR i_color = 'honeydew') AND + (i_units = 'Bunch' OR i_units = 'Ton') AND + (i_size = 'N/A' OR i_size = 'small') + ) OR + (i_category = 'Men' AND + (i_color = 'floral' OR i_color = 'deep') AND + (i_units = 'N/A' OR i_units = 'Dozen') AND + (i_size = 'petite' OR i_size = 'large') + ) OR + (i_category = 'Men' AND + (i_color = 'light' OR i_color = 'cornflower') AND + (i_units = 'Box' OR i_units = 'Pound') AND + (i_size = 'medium' OR i_size = 'extra large') + ))) OR + (i_manufact = i1.i_manufact AND + ((i_category = 'Women' AND + (i_color = 'midnight' OR i_color = 'snow') AND + (i_units = 'Pallet' OR i_units = 'Gross') AND + (i_size = 'medium' OR i_size = 'extra large') + ) OR + (i_category = 'Women' AND + (i_color = 'cyan' OR i_color = 'papaya') AND + (i_units = 'Cup' OR i_units = 'Dram') AND + (i_size = 'N/A' OR i_size = 'small') + ) OR + (i_category = 'Men' AND + (i_color = 'orange' OR i_color = 'frosted') AND + (i_units = 'Each' OR i_units = 'Tbl') AND + (i_size = 'petite' OR i_size = 'large') + ) OR + (i_category = 'Men' AND + (i_color = 'forest' OR i_color = 'ghost') AND + (i_units = 'Lb' OR i_units = 'Bundle') AND + (i_size = 'medium' OR i_size = 'extra large') + )))) > 0 +ORDER BY i_product_name +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q42.sql b/plugin-spark/src/test/resources/tpcds/q42.sql new file mode 100755 index 0000000000..4d2e71760d --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q42.sql @@ -0,0 +1,18 @@ +SELECT + dt.d_year, + item.i_category_id, + item.i_category, + sum(ss_ext_sales_price) +FROM date_dim dt, store_sales, item +WHERE dt.d_date_sk = store_sales.ss_sold_date_sk + AND store_sales.ss_item_sk = item.i_item_sk + AND item.i_manager_id = 1 + AND dt.d_moy = 11 + AND dt.d_year = 2000 +GROUP BY dt.d_year + , item.i_category_id + , item.i_category +ORDER BY sum(ss_ext_sales_price) DESC, dt.d_year + , item.i_category_id + , item.i_category +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q43.sql b/plugin-spark/src/test/resources/tpcds/q43.sql new file mode 100755 index 0000000000..45411772c1 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q43.sql @@ -0,0 +1,33 @@ +SELECT + s_store_name, + s_store_id, + sum(CASE WHEN (d_day_name = 'Sunday') + THEN ss_sales_price + ELSE NULL END) sun_sales, + sum(CASE WHEN (d_day_name = 'Monday') + THEN ss_sales_price + ELSE NULL END) mon_sales, + sum(CASE WHEN (d_day_name = 'Tuesday') + THEN ss_sales_price + ELSE NULL END) tue_sales, + sum(CASE WHEN (d_day_name = 'Wednesday') + THEN ss_sales_price + ELSE NULL END) wed_sales, + sum(CASE WHEN (d_day_name = 'Thursday') + THEN ss_sales_price + ELSE NULL END) thu_sales, + sum(CASE WHEN (d_day_name = 'Friday') + THEN ss_sales_price + ELSE NULL END) fri_sales, + sum(CASE WHEN (d_day_name = 'Saturday') + THEN ss_sales_price + ELSE NULL END) sat_sales +FROM date_dim, store_sales, store +WHERE d_date_sk = ss_sold_date_sk AND + s_store_sk = ss_store_sk AND + s_gmt_offset = -5 AND + d_year = 2000 +GROUP BY s_store_name, s_store_id +ORDER BY s_store_name, s_store_id, sun_sales, mon_sales, tue_sales, wed_sales, + thu_sales, fri_sales, sat_sales +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q44.sql b/plugin-spark/src/test/resources/tpcds/q44.sql new file mode 100755 index 0000000000..379e604788 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q44.sql @@ -0,0 +1,46 @@ +SELECT + asceding.rnk, + i1.i_product_name best_performing, + i2.i_product_name worst_performing +FROM (SELECT * +FROM (SELECT + item_sk, + rank() + OVER ( + ORDER BY rank_col ASC) rnk +FROM (SELECT + ss_item_sk item_sk, + avg(ss_net_profit) rank_col +FROM store_sales ss1 +WHERE ss_store_sk = 4 +GROUP BY ss_item_sk +HAVING avg(ss_net_profit) > 0.9 * (SELECT avg(ss_net_profit) rank_col +FROM store_sales +WHERE ss_store_sk = 4 + AND ss_addr_sk IS NULL +GROUP BY ss_store_sk)) V1) V11 +WHERE rnk < 11) asceding, + (SELECT * + FROM (SELECT + item_sk, + rank() + OVER ( + ORDER BY rank_col DESC) rnk + FROM (SELECT + ss_item_sk item_sk, + avg(ss_net_profit) rank_col + FROM store_sales ss1 + WHERE ss_store_sk = 4 + GROUP BY ss_item_sk + HAVING avg(ss_net_profit) > 0.9 * (SELECT avg(ss_net_profit) rank_col + FROM store_sales + WHERE ss_store_sk = 4 + AND ss_addr_sk IS NULL + GROUP BY ss_store_sk)) V2) V21 + WHERE rnk < 11) descending, + item i1, item i2 +WHERE asceding.rnk = descending.rnk + AND i1.i_item_sk = asceding.item_sk + AND i2.i_item_sk = descending.item_sk +ORDER BY asceding.rnk +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q45.sql b/plugin-spark/src/test/resources/tpcds/q45.sql new file mode 100755 index 0000000000..907438f196 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q45.sql @@ -0,0 +1,21 @@ +SELECT + ca_zip, + ca_city, + sum(ws_sales_price) +FROM web_sales, customer, customer_address, date_dim, item +WHERE ws_bill_customer_sk = c_customer_sk + AND c_current_addr_sk = ca_address_sk + AND ws_item_sk = i_item_sk + AND (substr(ca_zip, 1, 5) IN + ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') + OR + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_item_sk IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29) + ) +) + AND ws_sold_date_sk = d_date_sk + AND d_qoy = 2 AND d_year = 2001 +GROUP BY ca_zip, ca_city +ORDER BY ca_zip, ca_city +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q46.sql b/plugin-spark/src/test/resources/tpcds/q46.sql new file mode 100755 index 0000000000..0911677dff --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q46.sql @@ -0,0 +1,32 @@ +SELECT + c_last_name, + c_first_name, + ca_city, + bought_city, + ss_ticket_number, + amt, + profit +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + ca_city bought_city, + sum(ss_coupon_amt) amt, + sum(ss_net_profit) profit + FROM store_sales, date_dim, store, household_demographics, customer_address + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND store_sales.ss_addr_sk = customer_address.ca_address_sk + AND (household_demographics.hd_dep_count = 4 OR + household_demographics.hd_vehicle_count = 3) + AND date_dim.d_dow IN (6, 0) + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_city IN ('Fairview', 'Midway', 'Fairview', 'Fairview', 'Fairview') + GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city) dn, customer, + customer_address current_addr +WHERE ss_customer_sk = c_customer_sk + AND customer.c_current_addr_sk = current_addr.ca_address_sk + AND current_addr.ca_city <> bought_city +ORDER BY c_last_name, c_first_name, ca_city, bought_city, ss_ticket_number +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q47.sql b/plugin-spark/src/test/resources/tpcds/q47.sql new file mode 100755 index 0000000000..cfc37a4cec --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q47.sql @@ -0,0 +1,63 @@ +WITH v1 AS ( + SELECT + i_category, + i_brand, + s_store_name, + s_company_name, + d_year, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER + (PARTITION BY i_category, i_brand, + s_store_name, s_company_name, d_year) + avg_monthly_sales, + rank() + OVER + (PARTITION BY i_category, i_brand, + s_store_name, s_company_name + ORDER BY d_year, d_moy) rn + FROM item, store_sales, date_dim, store + WHERE ss_item_sk = i_item_sk AND + ss_sold_date_sk = d_date_sk AND + ss_store_sk = s_store_sk AND + ( + d_year = 1999 OR + (d_year = 1999 - 1 AND d_moy = 12) OR + (d_year = 1999 + 1 AND d_moy = 1) + ) + GROUP BY i_category, i_brand, + s_store_name, s_company_name, + d_year, d_moy), + v2 AS ( + SELECT + v1.i_category, + v1.i_brand, + v1.s_store_name, + v1.s_company_name, + v1.d_year, + v1.d_moy, + v1.avg_monthly_sales, + v1.sum_sales, + v1_lag.sum_sales psum, + v1_lead.sum_sales nsum + FROM v1, v1 v1_lag, v1 v1_lead + WHERE v1.i_category = v1_lag.i_category AND + v1.i_category = v1_lead.i_category AND + v1.i_brand = v1_lag.i_brand AND + v1.i_brand = v1_lead.i_brand AND + v1.s_store_name = v1_lag.s_store_name AND + v1.s_store_name = v1_lead.s_store_name AND + v1.s_company_name = v1_lag.s_company_name AND + v1.s_company_name = v1_lead.s_company_name AND + v1.rn = v1_lag.rn + 1 AND + v1.rn = v1_lead.rn - 1) +SELECT * +FROM v2 +WHERE d_year = 1999 AND + avg_monthly_sales > 0 AND + CASE WHEN avg_monthly_sales > 0 + THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales + ELSE NULL END > 0.1 +ORDER BY sum_sales - avg_monthly_sales, 3 +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q48.sql b/plugin-spark/src/test/resources/tpcds/q48.sql new file mode 100755 index 0000000000..fdb9f38e29 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q48.sql @@ -0,0 +1,63 @@ +SELECT sum(ss_quantity) +FROM store_sales, store, customer_demographics, customer_address, date_dim +WHERE s_store_sk = ss_store_sk + AND ss_sold_date_sk = d_date_sk AND d_year = 2001 + AND + ( + ( + cd_demo_sk = ss_cdemo_sk + AND + cd_marital_status = 'M' + AND + cd_education_status = '4 yr Degree' + AND + ss_sales_price BETWEEN 100.00 AND 150.00 + ) + OR + ( + cd_demo_sk = ss_cdemo_sk + AND + cd_marital_status = 'D' + AND + cd_education_status = '2 yr Degree' + AND + ss_sales_price BETWEEN 50.00 AND 100.00 + ) + OR + ( + cd_demo_sk = ss_cdemo_sk + AND + cd_marital_status = 'S' + AND + cd_education_status = 'College' + AND + ss_sales_price BETWEEN 150.00 AND 200.00 + ) + ) + AND + ( + ( + ss_addr_sk = ca_address_sk + AND + ca_country = 'United States' + AND + ca_state IN ('CO', 'OH', 'TX') + AND ss_net_profit BETWEEN 0 AND 2000 + ) + OR + (ss_addr_sk = ca_address_sk + AND + ca_country = 'United States' + AND + ca_state IN ('OR', 'MN', 'KY') + AND ss_net_profit BETWEEN 150 AND 3000 + ) + OR + (ss_addr_sk = ca_address_sk + AND + ca_country = 'United States' + AND + ca_state IN ('VA', 'CA', 'MS') + AND ss_net_profit BETWEEN 50 AND 25000 + ) + ) diff --git a/plugin-spark/src/test/resources/tpcds/q49.sql b/plugin-spark/src/test/resources/tpcds/q49.sql new file mode 100755 index 0000000000..9568d8b92d --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q49.sql @@ -0,0 +1,126 @@ +SELECT + 'web' AS channel, + web.item, + web.return_ratio, + web.return_rank, + web.currency_rank +FROM ( + SELECT + item, + return_ratio, + currency_ratio, + rank() + OVER ( + ORDER BY return_ratio) AS return_rank, + rank() + OVER ( + ORDER BY currency_ratio) AS currency_rank + FROM + (SELECT + ws.ws_item_sk AS item, + (cast(sum(coalesce(wr.wr_return_quantity, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(ws.ws_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, + (cast(sum(coalesce(wr.wr_return_amt, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(ws.ws_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio + FROM + web_sales ws LEFT OUTER JOIN web_returns wr + ON (ws.ws_order_number = wr.wr_order_number AND + ws.ws_item_sk = wr.wr_item_sk) + , date_dim + WHERE + wr.wr_return_amt > 10000 + AND ws.ws_net_profit > 1 + AND ws.ws_net_paid > 0 + AND ws.ws_quantity > 0 + AND ws_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 12 + GROUP BY ws.ws_item_sk + ) in_web + ) web +WHERE (web.return_rank <= 10 OR web.currency_rank <= 10) +UNION +SELECT + 'catalog' AS channel, + catalog.item, + catalog.return_ratio, + catalog.return_rank, + catalog.currency_rank +FROM ( + SELECT + item, + return_ratio, + currency_ratio, + rank() + OVER ( + ORDER BY return_ratio) AS return_rank, + rank() + OVER ( + ORDER BY currency_ratio) AS currency_rank + FROM + (SELECT + cs.cs_item_sk AS item, + (cast(sum(coalesce(cr.cr_return_quantity, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(cs.cs_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, + (cast(sum(coalesce(cr.cr_return_amount, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(cs.cs_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio + FROM + catalog_sales cs LEFT OUTER JOIN catalog_returns cr + ON (cs.cs_order_number = cr.cr_order_number AND + cs.cs_item_sk = cr.cr_item_sk) + , date_dim + WHERE + cr.cr_return_amount > 10000 + AND cs.cs_net_profit > 1 + AND cs.cs_net_paid > 0 + AND cs.cs_quantity > 0 + AND cs_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 12 + GROUP BY cs.cs_item_sk + ) in_cat + ) catalog +WHERE (catalog.return_rank <= 10 OR catalog.currency_rank <= 10) +UNION +SELECT + 'store' AS channel, + store.item, + store.return_ratio, + store.return_rank, + store.currency_rank +FROM ( + SELECT + item, + return_ratio, + currency_ratio, + rank() + OVER ( + ORDER BY return_ratio) AS return_rank, + rank() + OVER ( + ORDER BY currency_ratio) AS currency_rank + FROM + (SELECT + sts.ss_item_sk AS item, + (cast(sum(coalesce(sr.sr_return_quantity, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(sts.ss_quantity, 0)) AS DECIMAL(15, 4))) AS return_ratio, + (cast(sum(coalesce(sr.sr_return_amt, 0)) AS DECIMAL(15, 4)) / + cast(sum(coalesce(sts.ss_net_paid, 0)) AS DECIMAL(15, 4))) AS currency_ratio + FROM + store_sales sts LEFT OUTER JOIN store_returns sr + ON (sts.ss_ticket_number = sr.sr_ticket_number AND sts.ss_item_sk = sr.sr_item_sk) + , date_dim + WHERE + sr.sr_return_amt > 10000 + AND sts.ss_net_profit > 1 + AND sts.ss_net_paid > 0 + AND sts.ss_quantity > 0 + AND ss_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 12 + GROUP BY sts.ss_item_sk + ) in_store + ) store +WHERE (store.return_rank <= 10 OR store.currency_rank <= 10) +ORDER BY 1, 4, 5 +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q5.sql b/plugin-spark/src/test/resources/tpcds/q5.sql new file mode 100755 index 0000000000..b87cf3a448 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q5.sql @@ -0,0 +1,131 @@ +WITH ssr AS +( SELECT + s_store_id, + sum(sales_price) AS sales, + sum(profit) AS profit, + sum(return_amt) AS RETURNS, + sum(net_loss) AS profit_loss + FROM + (SELECT + ss_store_sk AS store_sk, + ss_sold_date_sk AS date_sk, + ss_ext_sales_price AS sales_price, + ss_net_profit AS profit, + cast(0 AS DECIMAL(7, 2)) AS return_amt, + cast(0 AS DECIMAL(7, 2)) AS net_loss + FROM store_sales + UNION ALL + SELECT + sr_store_sk AS store_sk, + sr_returned_date_sk AS date_sk, + cast(0 AS DECIMAL(7, 2)) AS sales_price, + cast(0 AS DECIMAL(7, 2)) AS profit, + sr_return_amt AS return_amt, + sr_net_loss AS net_loss + FROM store_returns) + salesreturns, date_dim, store + WHERE date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) + AND store_sk = s_store_sk + GROUP BY s_store_id), + csr AS + ( SELECT + cp_catalog_page_id, + sum(sales_price) AS sales, + sum(profit) AS profit, + sum(return_amt) AS RETURNS, + sum(net_loss) AS profit_loss + FROM + (SELECT + cs_catalog_page_sk AS page_sk, + cs_sold_date_sk AS date_sk, + cs_ext_sales_price AS sales_price, + cs_net_profit AS profit, + cast(0 AS DECIMAL(7, 2)) AS return_amt, + cast(0 AS DECIMAL(7, 2)) AS net_loss + FROM catalog_sales + UNION ALL + SELECT + cr_catalog_page_sk AS page_sk, + cr_returned_date_sk AS date_sk, + cast(0 AS DECIMAL(7, 2)) AS sales_price, + cast(0 AS DECIMAL(7, 2)) AS profit, + cr_return_amount AS return_amt, + cr_net_loss AS net_loss + FROM catalog_returns + ) salesreturns, date_dim, catalog_page + WHERE date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) + AND page_sk = cp_catalog_page_sk + GROUP BY cp_catalog_page_id) + , + wsr AS + ( SELECT + web_site_id, + sum(sales_price) AS sales, + sum(profit) AS profit, + sum(return_amt) AS RETURNS, + sum(net_loss) AS profit_loss + FROM + (SELECT + ws_web_site_sk AS wsr_web_site_sk, + ws_sold_date_sk AS date_sk, + ws_ext_sales_price AS sales_price, + ws_net_profit AS profit, + cast(0 AS DECIMAL(7, 2)) AS return_amt, + cast(0 AS DECIMAL(7, 2)) AS net_loss + FROM web_sales + UNION ALL + SELECT + ws_web_site_sk AS wsr_web_site_sk, + wr_returned_date_sk AS date_sk, + cast(0 AS DECIMAL(7, 2)) AS sales_price, + cast(0 AS DECIMAL(7, 2)) AS profit, + wr_return_amt AS return_amt, + wr_net_loss AS net_loss + FROM web_returns + LEFT OUTER JOIN web_sales ON + (wr_item_sk = ws_item_sk + AND wr_order_number = ws_order_number) + ) salesreturns, date_dim, web_site + WHERE date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND ((cast('2000-08-23' AS DATE) + INTERVAL 14 days)) + AND wsr_web_site_sk = web_site_sk + GROUP BY web_site_id) +SELECT + channel, + id, + sum(sales) AS sales, + sum(returns) AS returns, + sum(profit) AS profit +FROM + (SELECT + 'store channel' AS channel, + concat('store', s_store_id) AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM ssr + UNION ALL + SELECT + 'catalog channel' AS channel, + concat('catalog_page', cp_catalog_page_id) AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM csr + UNION ALL + SELECT + 'web channel' AS channel, + concat('web_site', web_site_id) AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM wsr + ) x +GROUP BY ROLLUP (channel, id) +ORDER BY channel, id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q50.sql b/plugin-spark/src/test/resources/tpcds/q50.sql new file mode 100755 index 0000000000..f1d4b15449 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q50.sql @@ -0,0 +1,47 @@ +SELECT + s_store_name, + s_company_id, + s_street_number, + s_street_name, + s_street_type, + s_suite_number, + s_city, + s_county, + s_state, + s_zip, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk <= 30) + THEN 1 + ELSE 0 END) AS `30 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 30) AND + (sr_returned_date_sk - ss_sold_date_sk <= 60) + THEN 1 + ELSE 0 END) AS `31 - 60 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 60) AND + (sr_returned_date_sk - ss_sold_date_sk <= 90) + THEN 1 + ELSE 0 END) AS `61 - 90 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 90) AND + (sr_returned_date_sk - ss_sold_date_sk <= 120) + THEN 1 + ELSE 0 END) AS `91 - 120 days `, + sum(CASE WHEN (sr_returned_date_sk - ss_sold_date_sk > 120) + THEN 1 + ELSE 0 END) AS `>120 days ` +FROM + store_sales, store_returns, store, date_dim d1, date_dim d2 +WHERE + d2.d_year = 2001 + AND d2.d_moy = 8 + AND ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk + AND ss_sold_date_sk = d1.d_date_sk + AND sr_returned_date_sk = d2.d_date_sk + AND ss_customer_sk = sr_customer_sk + AND ss_store_sk = s_store_sk +GROUP BY + s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, + s_suite_number, s_city, s_county, s_state, s_zip +ORDER BY + s_store_name, s_company_id, s_street_number, s_street_name, s_street_type, + s_suite_number, s_city, s_county, s_state, s_zip +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q51.sql b/plugin-spark/src/test/resources/tpcds/q51.sql new file mode 100755 index 0000000000..62b003eb67 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q51.sql @@ -0,0 +1,55 @@ +WITH web_v1 AS ( + SELECT + ws_item_sk item_sk, + d_date, + sum(sum(ws_sales_price)) + OVER (PARTITION BY ws_item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) cume_sales + FROM web_sales, date_dim + WHERE ws_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + AND ws_item_sk IS NOT NULL + GROUP BY ws_item_sk, d_date), + store_v1 AS ( + SELECT + ss_item_sk item_sk, + d_date, + sum(sum(ss_sales_price)) + OVER (PARTITION BY ss_item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) cume_sales + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + AND ss_item_sk IS NOT NULL + GROUP BY ss_item_sk, d_date) +SELECT * +FROM (SELECT + item_sk, + d_date, + web_sales, + store_sales, + max(web_sales) + OVER (PARTITION BY item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) web_cumulative, + max(store_sales) + OVER (PARTITION BY item_sk + ORDER BY d_date + ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) store_cumulative +FROM (SELECT + CASE WHEN web.item_sk IS NOT NULL + THEN web.item_sk + ELSE store.item_sk END item_sk, + CASE WHEN web.d_date IS NOT NULL + THEN web.d_date + ELSE store.d_date END d_date, + web.cume_sales web_sales, + store.cume_sales store_sales +FROM web_v1 web FULL OUTER JOIN store_v1 store ON (web.item_sk = store.item_sk + AND web.d_date = store.d_date) + ) x) y +WHERE web_cumulative > store_cumulative +ORDER BY item_sk, d_date +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q52.sql b/plugin-spark/src/test/resources/tpcds/q52.sql new file mode 100755 index 0000000000..467d1ae050 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q52.sql @@ -0,0 +1,14 @@ +SELECT + dt.d_year, + item.i_brand_id brand_id, + item.i_brand brand, + sum(ss_ext_sales_price) ext_price +FROM date_dim dt, store_sales, item +WHERE dt.d_date_sk = store_sales.ss_sold_date_sk + AND store_sales.ss_item_sk = item.i_item_sk + AND item.i_manager_id = 1 + AND dt.d_moy = 11 + AND dt.d_year = 2000 +GROUP BY dt.d_year, item.i_brand, item.i_brand_id +ORDER BY dt.d_year, ext_price DESC, brand_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q53.sql b/plugin-spark/src/test/resources/tpcds/q53.sql new file mode 100755 index 0000000000..b42c68dcf8 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q53.sql @@ -0,0 +1,30 @@ +SELECT * +FROM + (SELECT + i_manufact_id, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER (PARTITION BY i_manufact_id) avg_quarterly_sales + FROM item, store_sales, date_dim, store + WHERE ss_item_sk = i_item_sk AND + ss_sold_date_sk = d_date_sk AND + ss_store_sk = s_store_sk AND + d_month_seq IN (1200, 1200 + 1, 1200 + 2, 1200 + 3, 1200 + 4, 1200 + 5, 1200 + 6, + 1200 + 7, 1200 + 8, 1200 + 9, 1200 + 10, 1200 + 11) AND + ((i_category IN ('Books', 'Children', 'Electronics') AND + i_class IN ('personal', 'portable', 'reference', 'self-help') AND + i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', + 'exportiunivamalg #9', 'scholaramalgamalg #9')) + OR + (i_category IN ('Women', 'Music', 'Men') AND + i_class IN ('accessories', 'classical', 'fragrances', 'pants') AND + i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', + 'importoamalg #1'))) + GROUP BY i_manufact_id, d_qoy) tmp1 +WHERE CASE WHEN avg_quarterly_sales > 0 + THEN abs(sum_sales - avg_quarterly_sales) / avg_quarterly_sales + ELSE NULL END > 0.1 +ORDER BY avg_quarterly_sales, + sum_sales, + i_manufact_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q54.sql b/plugin-spark/src/test/resources/tpcds/q54.sql new file mode 100755 index 0000000000..897237fb6e --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q54.sql @@ -0,0 +1,61 @@ +WITH my_customers AS ( + SELECT DISTINCT + c_customer_sk, + c_current_addr_sk + FROM + (SELECT + cs_sold_date_sk sold_date_sk, + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + FROM catalog_sales + UNION ALL + SELECT + ws_sold_date_sk sold_date_sk, + ws_bill_customer_sk customer_sk, + ws_item_sk item_sk + FROM web_sales + ) cs_or_ws_sales, + item, + date_dim, + customer + WHERE sold_date_sk = d_date_sk + AND item_sk = i_item_sk + AND i_category = 'Women' + AND i_class = 'maternity' + AND c_customer_sk = cs_or_ws_sales.customer_sk + AND d_moy = 12 + AND d_year = 1998 +) + , my_revenue AS ( + SELECT + c_customer_sk, + sum(ss_ext_sales_price) AS revenue + FROM my_customers, + store_sales, + customer_address, + store, + date_dim + WHERE c_current_addr_sk = ca_address_sk + AND ca_county = s_county + AND ca_state = s_state + AND ss_sold_date_sk = d_date_sk + AND c_customer_sk = ss_customer_sk + AND d_month_seq BETWEEN (SELECT DISTINCT d_month_seq + 1 + FROM date_dim + WHERE d_year = 1998 AND d_moy = 12) + AND (SELECT DISTINCT d_month_seq + 3 + FROM date_dim + WHERE d_year = 1998 AND d_moy = 12) + GROUP BY c_customer_sk +) + , segments AS +(SELECT cast((revenue / 50) AS INT) AS segment + FROM my_revenue) +SELECT + segment, + count(*) AS num_customers, + segment * 50 AS segment_base +FROM segments +GROUP BY segment +ORDER BY segment, num_customers +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q55.sql b/plugin-spark/src/test/resources/tpcds/q55.sql new file mode 100755 index 0000000000..bc5d888c9a --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q55.sql @@ -0,0 +1,13 @@ +SELECT + i_brand_id brand_id, + i_brand brand, + sum(ss_ext_sales_price) ext_price +FROM date_dim, store_sales, item +WHERE d_date_sk = ss_sold_date_sk + AND ss_item_sk = i_item_sk + AND i_manager_id = 28 + AND d_moy = 11 + AND d_year = 1999 +GROUP BY i_brand, i_brand_id +ORDER BY ext_price DESC, brand_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q56.sql b/plugin-spark/src/test/resources/tpcds/q56.sql new file mode 100755 index 0000000000..2fa1738dcf --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q56.sql @@ -0,0 +1,65 @@ +WITH ss AS ( + SELECT + i_item_id, + sum(ss_ext_sales_price) total_sales + FROM + store_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_color IN ('slate', 'blanched', 'burnished')) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 2 + AND ss_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + cs AS ( + SELECT + i_item_id, + sum(cs_ext_sales_price) total_sales + FROM + catalog_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_color IN ('slate', 'blanched', 'burnished')) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 2 + AND cs_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + ws AS ( + SELECT + i_item_id, + sum(ws_ext_sales_price) total_sales + FROM + web_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_color IN ('slate', 'blanched', 'burnished')) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 2001 + AND d_moy = 2 + AND ws_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id) +SELECT + i_item_id, + sum(total_sales) total_sales +FROM (SELECT * + FROM ss + UNION ALL + SELECT * + FROM cs + UNION ALL + SELECT * + FROM ws) tmp1 +GROUP BY i_item_id +ORDER BY total_sales +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q57.sql b/plugin-spark/src/test/resources/tpcds/q57.sql new file mode 100755 index 0000000000..cf70d4b905 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q57.sql @@ -0,0 +1,56 @@ +WITH v1 AS ( + SELECT + i_category, + i_brand, + cc_name, + d_year, + d_moy, + sum(cs_sales_price) sum_sales, + avg(sum(cs_sales_price)) + OVER + (PARTITION BY i_category, i_brand, cc_name, d_year) + avg_monthly_sales, + rank() + OVER + (PARTITION BY i_category, i_brand, cc_name + ORDER BY d_year, d_moy) rn + FROM item, catalog_sales, date_dim, call_center + WHERE cs_item_sk = i_item_sk AND + cs_sold_date_sk = d_date_sk AND + cc_call_center_sk = cs_call_center_sk AND + ( + d_year = 1999 OR + (d_year = 1999 - 1 AND d_moy = 12) OR + (d_year = 1999 + 1 AND d_moy = 1) + ) + GROUP BY i_category, i_brand, + cc_name, d_year, d_moy), + v2 AS ( + SELECT + v1.i_category, + v1.i_brand, + v1.cc_name, + v1.d_year, + v1.d_moy, + v1.avg_monthly_sales, + v1.sum_sales, + v1_lag.sum_sales psum, + v1_lead.sum_sales nsum + FROM v1, v1 v1_lag, v1 v1_lead + WHERE v1.i_category = v1_lag.i_category AND + v1.i_category = v1_lead.i_category AND + v1.i_brand = v1_lag.i_brand AND + v1.i_brand = v1_lead.i_brand AND + v1.cc_name = v1_lag.cc_name AND + v1.cc_name = v1_lead.cc_name AND + v1.rn = v1_lag.rn + 1 AND + v1.rn = v1_lead.rn - 1) +SELECT * +FROM v2 +WHERE d_year = 1999 AND + avg_monthly_sales > 0 AND + CASE WHEN avg_monthly_sales > 0 + THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales + ELSE NULL END > 0.1 +ORDER BY sum_sales - avg_monthly_sales, 3 +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q58.sql b/plugin-spark/src/test/resources/tpcds/q58.sql new file mode 100755 index 0000000000..5f63f33dc9 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q58.sql @@ -0,0 +1,59 @@ +WITH ss_items AS +(SELECT + i_item_id item_id, + sum(ss_ext_sales_price) ss_item_rev + FROM store_sales, item, date_dim + WHERE ss_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_date = '2000-01-03')) + AND ss_sold_date_sk = d_date_sk + GROUP BY i_item_id), + cs_items AS + (SELECT + i_item_id item_id, + sum(cs_ext_sales_price) cs_item_rev + FROM catalog_sales, item, date_dim + WHERE cs_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_date = '2000-01-03')) + AND cs_sold_date_sk = d_date_sk + GROUP BY i_item_id), + ws_items AS + (SELECT + i_item_id item_id, + sum(ws_ext_sales_price) ws_item_rev + FROM web_sales, item, date_dim + WHERE ws_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq = (SELECT d_week_seq + FROM date_dim + WHERE d_date = '2000-01-03')) + AND ws_sold_date_sk = d_date_sk + GROUP BY i_item_id) +SELECT + ss_items.item_id, + ss_item_rev, + ss_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 ss_dev, + cs_item_rev, + cs_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 cs_dev, + ws_item_rev, + ws_item_rev / (ss_item_rev + cs_item_rev + ws_item_rev) / 3 * 100 ws_dev, + (ss_item_rev + cs_item_rev + ws_item_rev) / 3 average +FROM ss_items, cs_items, ws_items +WHERE ss_items.item_id = cs_items.item_id + AND ss_items.item_id = ws_items.item_id + AND ss_item_rev BETWEEN 0.9 * cs_item_rev AND 1.1 * cs_item_rev + AND ss_item_rev BETWEEN 0.9 * ws_item_rev AND 1.1 * ws_item_rev + AND cs_item_rev BETWEEN 0.9 * ss_item_rev AND 1.1 * ss_item_rev + AND cs_item_rev BETWEEN 0.9 * ws_item_rev AND 1.1 * ws_item_rev + AND ws_item_rev BETWEEN 0.9 * ss_item_rev AND 1.1 * ss_item_rev + AND ws_item_rev BETWEEN 0.9 * cs_item_rev AND 1.1 * cs_item_rev +ORDER BY item_id, ss_item_rev +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q59.sql b/plugin-spark/src/test/resources/tpcds/q59.sql new file mode 100755 index 0000000000..3cef202768 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q59.sql @@ -0,0 +1,75 @@ +WITH wss AS +(SELECT + d_week_seq, + ss_store_sk, + sum(CASE WHEN (d_day_name = 'Sunday') + THEN ss_sales_price + ELSE NULL END) sun_sales, + sum(CASE WHEN (d_day_name = 'Monday') + THEN ss_sales_price + ELSE NULL END) mon_sales, + sum(CASE WHEN (d_day_name = 'Tuesday') + THEN ss_sales_price + ELSE NULL END) tue_sales, + sum(CASE WHEN (d_day_name = 'Wednesday') + THEN ss_sales_price + ELSE NULL END) wed_sales, + sum(CASE WHEN (d_day_name = 'Thursday') + THEN ss_sales_price + ELSE NULL END) thu_sales, + sum(CASE WHEN (d_day_name = 'Friday') + THEN ss_sales_price + ELSE NULL END) fri_sales, + sum(CASE WHEN (d_day_name = 'Saturday') + THEN ss_sales_price + ELSE NULL END) sat_sales + FROM store_sales, date_dim + WHERE d_date_sk = ss_sold_date_sk + GROUP BY d_week_seq, ss_store_sk +) +SELECT + s_store_name1, + s_store_id1, + d_week_seq1, + sun_sales1 / sun_sales2, + mon_sales1 / mon_sales2, + tue_sales1 / tue_sales2, + wed_sales1 / wed_sales2, + thu_sales1 / thu_sales2, + fri_sales1 / fri_sales2, + sat_sales1 / sat_sales2 +FROM + (SELECT + s_store_name s_store_name1, + wss.d_week_seq d_week_seq1, + s_store_id s_store_id1, + sun_sales sun_sales1, + mon_sales mon_sales1, + tue_sales tue_sales1, + wed_sales wed_sales1, + thu_sales thu_sales1, + fri_sales fri_sales1, + sat_sales sat_sales1 + FROM wss, store, date_dim d + WHERE d.d_week_seq = wss.d_week_seq AND + ss_store_sk = s_store_sk AND + d_month_seq BETWEEN 1212 AND 1212 + 11) y, + (SELECT + s_store_name s_store_name2, + wss.d_week_seq d_week_seq2, + s_store_id s_store_id2, + sun_sales sun_sales2, + mon_sales mon_sales2, + tue_sales tue_sales2, + wed_sales wed_sales2, + thu_sales thu_sales2, + fri_sales fri_sales2, + sat_sales sat_sales2 + FROM wss, store, date_dim d + WHERE d.d_week_seq = wss.d_week_seq AND + ss_store_sk = s_store_sk AND + d_month_seq BETWEEN 1212 + 12 AND 1212 + 23) x +WHERE s_store_id1 = s_store_id2 + AND d_week_seq1 = d_week_seq2 - 52 +ORDER BY s_store_name1, s_store_id1, d_week_seq1 +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q6.sql b/plugin-spark/src/test/resources/tpcds/q6.sql new file mode 100755 index 0000000000..f0f5cf05ae --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q6.sql @@ -0,0 +1,21 @@ +SELECT + a.ca_state state, + count(*) cnt +FROM + customer_address a, customer c, store_sales s, date_dim d, item i +WHERE a.ca_address_sk = c.c_current_addr_sk + AND c.c_customer_sk = s.ss_customer_sk + AND s.ss_sold_date_sk = d.d_date_sk + AND s.ss_item_sk = i.i_item_sk + AND d.d_month_seq = + (SELECT DISTINCT (d_month_seq) + FROM date_dim + WHERE d_year = 2000 AND d_moy = 1) + AND i.i_current_price > 1.2 * + (SELECT avg(j.i_current_price) + FROM item j + WHERE j.i_category = i.i_category) +GROUP BY a.ca_state +HAVING count(*) >= 10 +ORDER BY cnt +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q60.sql b/plugin-spark/src/test/resources/tpcds/q60.sql new file mode 100755 index 0000000000..41b963f44b --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q60.sql @@ -0,0 +1,62 @@ +WITH ss AS ( + SELECT + i_item_id, + sum(ss_ext_sales_price) total_sales + FROM store_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_category IN ('Music')) + AND ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 9 + AND ss_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + cs AS ( + SELECT + i_item_id, + sum(cs_ext_sales_price) total_sales + FROM catalog_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_category IN ('Music')) + AND cs_item_sk = i_item_sk + AND cs_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 9 + AND cs_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id), + ws AS ( + SELECT + i_item_id, + sum(ws_ext_sales_price) total_sales + FROM web_sales, date_dim, customer_address, item + WHERE + i_item_id IN (SELECT i_item_id + FROM item + WHERE i_category IN ('Music')) + AND ws_item_sk = i_item_sk + AND ws_sold_date_sk = d_date_sk + AND d_year = 1998 + AND d_moy = 9 + AND ws_bill_addr_sk = ca_address_sk + AND ca_gmt_offset = -5 + GROUP BY i_item_id) +SELECT + i_item_id, + sum(total_sales) total_sales +FROM (SELECT * + FROM ss + UNION ALL + SELECT * + FROM cs + UNION ALL + SELECT * + FROM ws) tmp1 +GROUP BY i_item_id +ORDER BY i_item_id, total_sales +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q61.sql b/plugin-spark/src/test/resources/tpcds/q61.sql new file mode 100755 index 0000000000..b0a872b4b8 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q61.sql @@ -0,0 +1,33 @@ +SELECT + promotions, + total, + cast(promotions AS DECIMAL(15, 4)) / cast(total AS DECIMAL(15, 4)) * 100 +FROM + (SELECT sum(ss_ext_sales_price) promotions + FROM store_sales, store, promotion, date_dim, customer, customer_address, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_store_sk = s_store_sk + AND ss_promo_sk = p_promo_sk + AND ss_customer_sk = c_customer_sk + AND ca_address_sk = c_current_addr_sk + AND ss_item_sk = i_item_sk + AND ca_gmt_offset = -5 + AND i_category = 'Jewelry' + AND (p_channel_dmail = 'Y' OR p_channel_email = 'Y' OR p_channel_tv = 'Y') + AND s_gmt_offset = -5 + AND d_year = 1998 + AND d_moy = 11) promotional_sales, + (SELECT sum(ss_ext_sales_price) total + FROM store_sales, store, date_dim, customer, customer_address, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_store_sk = s_store_sk + AND ss_customer_sk = c_customer_sk + AND ca_address_sk = c_current_addr_sk + AND ss_item_sk = i_item_sk + AND ca_gmt_offset = -5 + AND i_category = 'Jewelry' + AND s_gmt_offset = -5 + AND d_year = 1998 + AND d_moy = 11) all_sales +ORDER BY promotions, total +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q62.sql b/plugin-spark/src/test/resources/tpcds/q62.sql new file mode 100755 index 0000000000..8a414f154b --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q62.sql @@ -0,0 +1,35 @@ +SELECT + substr(w_warehouse_name, 1, 20), + sm_type, + web_name, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk <= 30) + THEN 1 + ELSE 0 END) AS `30 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 30) AND + (ws_ship_date_sk - ws_sold_date_sk <= 60) + THEN 1 + ELSE 0 END) AS `31 - 60 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 60) AND + (ws_ship_date_sk - ws_sold_date_sk <= 90) + THEN 1 + ELSE 0 END) AS `61 - 90 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 90) AND + (ws_ship_date_sk - ws_sold_date_sk <= 120) + THEN 1 + ELSE 0 END) AS `91 - 120 days `, + sum(CASE WHEN (ws_ship_date_sk - ws_sold_date_sk > 120) + THEN 1 + ELSE 0 END) AS `>120 days ` +FROM + web_sales, warehouse, ship_mode, web_site, date_dim +WHERE + d_month_seq BETWEEN 1200 AND 1200 + 11 + AND ws_ship_date_sk = d_date_sk + AND ws_warehouse_sk = w_warehouse_sk + AND ws_ship_mode_sk = sm_ship_mode_sk + AND ws_web_site_sk = web_site_sk +GROUP BY + substr(w_warehouse_name, 1, 20), sm_type, web_name +ORDER BY + substr(w_warehouse_name, 1, 20), sm_type, web_name +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q63.sql b/plugin-spark/src/test/resources/tpcds/q63.sql new file mode 100755 index 0000000000..ef6867e0a9 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q63.sql @@ -0,0 +1,31 @@ +SELECT * +FROM (SELECT + i_manager_id, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER (PARTITION BY i_manager_id) avg_monthly_sales +FROM item + , store_sales + , date_dim + , store +WHERE ss_item_sk = i_item_sk + AND ss_sold_date_sk = d_date_sk + AND ss_store_sk = s_store_sk + AND d_month_seq IN (1200, 1200 + 1, 1200 + 2, 1200 + 3, 1200 + 4, 1200 + 5, 1200 + 6, 1200 + 7, + 1200 + 8, 1200 + 9, 1200 + 10, 1200 + 11) + AND ((i_category IN ('Books', 'Children', 'Electronics') + AND i_class IN ('personal', 'portable', 'refernece', 'self-help') + AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', + 'exportiunivamalg #9', 'scholaramalgamalg #9')) + OR (i_category IN ('Women', 'Music', 'Men') + AND i_class IN ('accessories', 'classical', 'fragrances', 'pants') + AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', + 'importoamalg #1'))) +GROUP BY i_manager_id, d_moy) tmp1 +WHERE CASE WHEN avg_monthly_sales > 0 + THEN abs(sum_sales - avg_monthly_sales) / avg_monthly_sales + ELSE NULL END > 0.1 +ORDER BY i_manager_id + , avg_monthly_sales + , sum_sales +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q64.sql b/plugin-spark/src/test/resources/tpcds/q64.sql new file mode 100755 index 0000000000..8ec1d31b61 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q64.sql @@ -0,0 +1,92 @@ +WITH cs_ui AS +(SELECT + cs_item_sk, + sum(cs_ext_list_price) AS sale, + sum(cr_refunded_cash + cr_reversed_charge + cr_store_credit) AS refund + FROM catalog_sales + , catalog_returns + WHERE cs_item_sk = cr_item_sk + AND cs_order_number = cr_order_number + GROUP BY cs_item_sk + HAVING sum(cs_ext_list_price) > 2 * sum(cr_refunded_cash + cr_reversed_charge + cr_store_credit)), + cross_sales AS + (SELECT + i_product_name product_name, + i_item_sk item_sk, + s_store_name store_name, + s_zip store_zip, + ad1.ca_street_number b_street_number, + ad1.ca_street_name b_streen_name, + ad1.ca_city b_city, + ad1.ca_zip b_zip, + ad2.ca_street_number c_street_number, + ad2.ca_street_name c_street_name, + ad2.ca_city c_city, + ad2.ca_zip c_zip, + d1.d_year AS syear, + d2.d_year AS fsyear, + d3.d_year s2year, + count(*) cnt, + sum(ss_wholesale_cost) s1, + sum(ss_list_price) s2, + sum(ss_coupon_amt) s3 + FROM store_sales, store_returns, cs_ui, date_dim d1, date_dim d2, date_dim d3, + store, customer, customer_demographics cd1, customer_demographics cd2, + promotion, household_demographics hd1, household_demographics hd2, + customer_address ad1, customer_address ad2, income_band ib1, income_band ib2, item + WHERE ss_store_sk = s_store_sk AND + ss_sold_date_sk = d1.d_date_sk AND + ss_customer_sk = c_customer_sk AND + ss_cdemo_sk = cd1.cd_demo_sk AND + ss_hdemo_sk = hd1.hd_demo_sk AND + ss_addr_sk = ad1.ca_address_sk AND + ss_item_sk = i_item_sk AND + ss_item_sk = sr_item_sk AND + ss_ticket_number = sr_ticket_number AND + ss_item_sk = cs_ui.cs_item_sk AND + c_current_cdemo_sk = cd2.cd_demo_sk AND + c_current_hdemo_sk = hd2.hd_demo_sk AND + c_current_addr_sk = ad2.ca_address_sk AND + c_first_sales_date_sk = d2.d_date_sk AND + c_first_shipto_date_sk = d3.d_date_sk AND + ss_promo_sk = p_promo_sk AND + hd1.hd_income_band_sk = ib1.ib_income_band_sk AND + hd2.hd_income_band_sk = ib2.ib_income_band_sk AND + cd1.cd_marital_status <> cd2.cd_marital_status AND + i_color IN ('purple', 'burlywood', 'indian', 'spring', 'floral', 'medium') AND + i_current_price BETWEEN 64 AND 64 + 10 AND + i_current_price BETWEEN 64 + 1 AND 64 + 15 + GROUP BY i_product_name, i_item_sk, s_store_name, s_zip, ad1.ca_street_number, + ad1.ca_street_name, ad1.ca_city, ad1.ca_zip, ad2.ca_street_number, + ad2.ca_street_name, ad2.ca_city, ad2.ca_zip, d1.d_year, d2.d_year, d3.d_year + ) +SELECT + cs1.product_name, + cs1.store_name, + cs1.store_zip, + cs1.b_street_number, + cs1.b_streen_name, + cs1.b_city, + cs1.b_zip, + cs1.c_street_number, + cs1.c_street_name, + cs1.c_city, + cs1.c_zip, + cs1.syear, + cs1.cnt, + cs1.s1, + cs1.s2, + cs1.s3, + cs2.s1, + cs2.s2, + cs2.s3, + cs2.syear, + cs2.cnt +FROM cross_sales cs1, cross_sales cs2 +WHERE cs1.item_sk = cs2.item_sk AND + cs1.syear = 1999 AND + cs2.syear = 1999 + 1 AND + cs2.cnt <= cs1.cnt AND + cs1.store_name = cs2.store_name AND + cs1.store_zip = cs2.store_zip +ORDER BY cs1.product_name, cs1.store_name, cs2.cnt diff --git a/plugin-spark/src/test/resources/tpcds/q65.sql b/plugin-spark/src/test/resources/tpcds/q65.sql new file mode 100755 index 0000000000..aad04be1bc --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q65.sql @@ -0,0 +1,33 @@ +SELECT + s_store_name, + i_item_desc, + sc.revenue, + i_current_price, + i_wholesale_cost, + i_brand +FROM store, item, + (SELECT + ss_store_sk, + avg(revenue) AS ave + FROM + (SELECT + ss_store_sk, + ss_item_sk, + sum(ss_sales_price) AS revenue + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk AND d_month_seq BETWEEN 1176 AND 1176 + 11 + GROUP BY ss_store_sk, ss_item_sk) sa + GROUP BY ss_store_sk) sb, + (SELECT + ss_store_sk, + ss_item_sk, + sum(ss_sales_price) AS revenue + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk AND d_month_seq BETWEEN 1176 AND 1176 + 11 + GROUP BY ss_store_sk, ss_item_sk) sc +WHERE sb.ss_store_sk = sc.ss_store_sk AND + sc.revenue <= 0.1 * sb.ave AND + s_store_sk = sc.ss_store_sk AND + i_item_sk = sc.ss_item_sk +ORDER BY s_store_name, i_item_desc +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q66.sql b/plugin-spark/src/test/resources/tpcds/q66.sql new file mode 100755 index 0000000000..f826b41643 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q66.sql @@ -0,0 +1,240 @@ +SELECT + w_warehouse_name, + w_warehouse_sq_ft, + w_city, + w_county, + w_state, + w_country, + ship_carriers, + year, + sum(jan_sales) AS jan_sales, + sum(feb_sales) AS feb_sales, + sum(mar_sales) AS mar_sales, + sum(apr_sales) AS apr_sales, + sum(may_sales) AS may_sales, + sum(jun_sales) AS jun_sales, + sum(jul_sales) AS jul_sales, + sum(aug_sales) AS aug_sales, + sum(sep_sales) AS sep_sales, + sum(oct_sales) AS oct_sales, + sum(nov_sales) AS nov_sales, + sum(dec_sales) AS dec_sales, + sum(jan_sales / w_warehouse_sq_ft) AS jan_sales_per_sq_foot, + sum(feb_sales / w_warehouse_sq_ft) AS feb_sales_per_sq_foot, + sum(mar_sales / w_warehouse_sq_ft) AS mar_sales_per_sq_foot, + sum(apr_sales / w_warehouse_sq_ft) AS apr_sales_per_sq_foot, + sum(may_sales / w_warehouse_sq_ft) AS may_sales_per_sq_foot, + sum(jun_sales / w_warehouse_sq_ft) AS jun_sales_per_sq_foot, + sum(jul_sales / w_warehouse_sq_ft) AS jul_sales_per_sq_foot, + sum(aug_sales / w_warehouse_sq_ft) AS aug_sales_per_sq_foot, + sum(sep_sales / w_warehouse_sq_ft) AS sep_sales_per_sq_foot, + sum(oct_sales / w_warehouse_sq_ft) AS oct_sales_per_sq_foot, + sum(nov_sales / w_warehouse_sq_ft) AS nov_sales_per_sq_foot, + sum(dec_sales / w_warehouse_sq_ft) AS dec_sales_per_sq_foot, + sum(jan_net) AS jan_net, + sum(feb_net) AS feb_net, + sum(mar_net) AS mar_net, + sum(apr_net) AS apr_net, + sum(may_net) AS may_net, + sum(jun_net) AS jun_net, + sum(jul_net) AS jul_net, + sum(aug_net) AS aug_net, + sum(sep_net) AS sep_net, + sum(oct_net) AS oct_net, + sum(nov_net) AS nov_net, + sum(dec_net) AS dec_net +FROM ( + (SELECT + w_warehouse_name, + w_warehouse_sq_ft, + w_city, + w_county, + w_state, + w_country, + concat('DHL', ',', 'BARIAN') AS ship_carriers, + d_year AS year, + sum(CASE WHEN d_moy = 1 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS jan_sales, + sum(CASE WHEN d_moy = 2 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS feb_sales, + sum(CASE WHEN d_moy = 3 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS mar_sales, + sum(CASE WHEN d_moy = 4 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS apr_sales, + sum(CASE WHEN d_moy = 5 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS may_sales, + sum(CASE WHEN d_moy = 6 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS jun_sales, + sum(CASE WHEN d_moy = 7 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS jul_sales, + sum(CASE WHEN d_moy = 8 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS aug_sales, + sum(CASE WHEN d_moy = 9 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS sep_sales, + sum(CASE WHEN d_moy = 10 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS oct_sales, + sum(CASE WHEN d_moy = 11 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS nov_sales, + sum(CASE WHEN d_moy = 12 + THEN ws_ext_sales_price * ws_quantity + ELSE 0 END) AS dec_sales, + sum(CASE WHEN d_moy = 1 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS jan_net, + sum(CASE WHEN d_moy = 2 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS feb_net, + sum(CASE WHEN d_moy = 3 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS mar_net, + sum(CASE WHEN d_moy = 4 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS apr_net, + sum(CASE WHEN d_moy = 5 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS may_net, + sum(CASE WHEN d_moy = 6 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS jun_net, + sum(CASE WHEN d_moy = 7 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS jul_net, + sum(CASE WHEN d_moy = 8 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS aug_net, + sum(CASE WHEN d_moy = 9 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS sep_net, + sum(CASE WHEN d_moy = 10 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS oct_net, + sum(CASE WHEN d_moy = 11 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS nov_net, + sum(CASE WHEN d_moy = 12 + THEN ws_net_paid * ws_quantity + ELSE 0 END) AS dec_net + FROM + web_sales, warehouse, date_dim, time_dim, ship_mode + WHERE + ws_warehouse_sk = w_warehouse_sk + AND ws_sold_date_sk = d_date_sk + AND ws_sold_time_sk = t_time_sk + AND ws_ship_mode_sk = sm_ship_mode_sk + AND d_year = 2001 + AND t_time BETWEEN 30838 AND 30838 + 28800 + AND sm_carrier IN ('DHL', 'BARIAN') + GROUP BY + w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, d_year) + UNION ALL + (SELECT + w_warehouse_name, + w_warehouse_sq_ft, + w_city, + w_county, + w_state, + w_country, + concat('DHL', ',', 'BARIAN') AS ship_carriers, + d_year AS year, + sum(CASE WHEN d_moy = 1 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS jan_sales, + sum(CASE WHEN d_moy = 2 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS feb_sales, + sum(CASE WHEN d_moy = 3 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS mar_sales, + sum(CASE WHEN d_moy = 4 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS apr_sales, + sum(CASE WHEN d_moy = 5 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS may_sales, + sum(CASE WHEN d_moy = 6 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS jun_sales, + sum(CASE WHEN d_moy = 7 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS jul_sales, + sum(CASE WHEN d_moy = 8 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS aug_sales, + sum(CASE WHEN d_moy = 9 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS sep_sales, + sum(CASE WHEN d_moy = 10 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS oct_sales, + sum(CASE WHEN d_moy = 11 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS nov_sales, + sum(CASE WHEN d_moy = 12 + THEN cs_sales_price * cs_quantity + ELSE 0 END) AS dec_sales, + sum(CASE WHEN d_moy = 1 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS jan_net, + sum(CASE WHEN d_moy = 2 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS feb_net, + sum(CASE WHEN d_moy = 3 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS mar_net, + sum(CASE WHEN d_moy = 4 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS apr_net, + sum(CASE WHEN d_moy = 5 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS may_net, + sum(CASE WHEN d_moy = 6 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS jun_net, + sum(CASE WHEN d_moy = 7 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS jul_net, + sum(CASE WHEN d_moy = 8 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS aug_net, + sum(CASE WHEN d_moy = 9 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS sep_net, + sum(CASE WHEN d_moy = 10 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS oct_net, + sum(CASE WHEN d_moy = 11 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS nov_net, + sum(CASE WHEN d_moy = 12 + THEN cs_net_paid_inc_tax * cs_quantity + ELSE 0 END) AS dec_net + FROM + catalog_sales, warehouse, date_dim, time_dim, ship_mode + WHERE + cs_warehouse_sk = w_warehouse_sk + AND cs_sold_date_sk = d_date_sk + AND cs_sold_time_sk = t_time_sk + AND cs_ship_mode_sk = sm_ship_mode_sk + AND d_year = 2001 + AND t_time BETWEEN 30838 AND 30838 + 28800 + AND sm_carrier IN ('DHL', 'BARIAN') + GROUP BY + w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, d_year + ) + ) x +GROUP BY + w_warehouse_name, w_warehouse_sq_ft, w_city, w_county, w_state, w_country, + ship_carriers, year +ORDER BY w_warehouse_name +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q67.sql b/plugin-spark/src/test/resources/tpcds/q67.sql new file mode 100755 index 0000000000..f66e2252bd --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q67.sql @@ -0,0 +1,38 @@ +SELECT * +FROM + (SELECT + i_category, + i_class, + i_brand, + i_product_name, + d_year, + d_qoy, + d_moy, + s_store_id, + sumsales, + rank() + OVER (PARTITION BY i_category + ORDER BY sumsales DESC) rk + FROM + (SELECT + i_category, + i_class, + i_brand, + i_product_name, + d_year, + d_qoy, + d_moy, + s_store_id, + sum(coalesce(ss_sales_price * ss_quantity, 0)) sumsales + FROM store_sales, date_dim, store, item + WHERE ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + AND ss_store_sk = s_store_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + GROUP BY ROLLUP (i_category, i_class, i_brand, i_product_name, d_year, d_qoy, + d_moy, s_store_id)) dw1) dw2 +WHERE rk <= 100 +ORDER BY + i_category, i_class, i_brand, i_product_name, d_year, + d_qoy, d_moy, s_store_id, sumsales, rk +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q68.sql b/plugin-spark/src/test/resources/tpcds/q68.sql new file mode 100755 index 0000000000..adb8a7189d --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q68.sql @@ -0,0 +1,34 @@ +SELECT + c_last_name, + c_first_name, + ca_city, + bought_city, + ss_ticket_number, + extended_price, + extended_tax, + list_price +FROM (SELECT + ss_ticket_number, + ss_customer_sk, + ca_city bought_city, + sum(ss_ext_sales_price) extended_price, + sum(ss_ext_list_price) list_price, + sum(ss_ext_tax) extended_tax +FROM store_sales, date_dim, store, household_demographics, customer_address +WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND store_sales.ss_addr_sk = customer_address.ca_address_sk + AND date_dim.d_dom BETWEEN 1 AND 2 + AND (household_demographics.hd_dep_count = 4 OR + household_demographics.hd_vehicle_count = 3) + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_city IN ('Midway', 'Fairview') +GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, ca_city) dn, + customer, + customer_address current_addr +WHERE ss_customer_sk = c_customer_sk + AND customer.c_current_addr_sk = current_addr.ca_address_sk + AND current_addr.ca_city <> bought_city +ORDER BY c_last_name, ss_ticket_number +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q69.sql b/plugin-spark/src/test/resources/tpcds/q69.sql new file mode 100755 index 0000000000..1f0ee64f56 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q69.sql @@ -0,0 +1,38 @@ +SELECT + cd_gender, + cd_marital_status, + cd_education_status, + count(*) cnt1, + cd_purchase_estimate, + count(*) cnt2, + cd_credit_rating, + count(*) cnt3 +FROM + customer c, customer_address ca, customer_demographics +WHERE + c.c_current_addr_sk = ca.ca_address_sk AND + ca_state IN ('KY', 'GA', 'NM') AND + cd_demo_sk = c.c_current_cdemo_sk AND + exists(SELECT * + FROM store_sales, date_dim + WHERE c.c_customer_sk = ss_customer_sk AND + ss_sold_date_sk = d_date_sk AND + d_year = 2001 AND + d_moy BETWEEN 4 AND 4 + 2) AND + (NOT exists(SELECT * + FROM web_sales, date_dim + WHERE c.c_customer_sk = ws_bill_customer_sk AND + ws_sold_date_sk = d_date_sk AND + d_year = 2001 AND + d_moy BETWEEN 4 AND 4 + 2) AND + NOT exists(SELECT * + FROM catalog_sales, date_dim + WHERE c.c_customer_sk = cs_ship_customer_sk AND + cs_sold_date_sk = d_date_sk AND + d_year = 2001 AND + d_moy BETWEEN 4 AND 4 + 2)) +GROUP BY cd_gender, cd_marital_status, cd_education_status, + cd_purchase_estimate, cd_credit_rating +ORDER BY cd_gender, cd_marital_status, cd_education_status, + cd_purchase_estimate, cd_credit_rating +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q7.sql b/plugin-spark/src/test/resources/tpcds/q7.sql new file mode 100755 index 0000000000..6630a00548 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q7.sql @@ -0,0 +1,19 @@ +SELECT + i_item_id, + avg(ss_quantity) agg1, + avg(ss_list_price) agg2, + avg(ss_coupon_amt) agg3, + avg(ss_sales_price) agg4 +FROM store_sales, customer_demographics, date_dim, item, promotion +WHERE ss_sold_date_sk = d_date_sk AND + ss_item_sk = i_item_sk AND + ss_cdemo_sk = cd_demo_sk AND + ss_promo_sk = p_promo_sk AND + cd_gender = 'M' AND + cd_marital_status = 'S' AND + cd_education_status = 'College' AND + (p_channel_email = 'N' OR p_channel_event = 'N') AND + d_year = 2000 +GROUP BY i_item_id +ORDER BY i_item_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q70.sql b/plugin-spark/src/test/resources/tpcds/q70.sql new file mode 100755 index 0000000000..625011b212 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q70.sql @@ -0,0 +1,38 @@ +SELECT + sum(ss_net_profit) AS total_sum, + s_state, + s_county, + grouping(s_state) + grouping(s_county) AS lochierarchy, + rank() + OVER ( + PARTITION BY grouping(s_state) + grouping(s_county), + CASE WHEN grouping(s_county) = 0 + THEN s_state END + ORDER BY sum(ss_net_profit) DESC) AS rank_within_parent +FROM + store_sales, date_dim d1, store +WHERE + d1.d_month_seq BETWEEN 1200 AND 1200 + 11 + AND d1.d_date_sk = ss_sold_date_sk + AND s_store_sk = ss_store_sk + AND s_state IN + (SELECT s_state + FROM + (SELECT + s_state AS s_state, + rank() + OVER (PARTITION BY s_state + ORDER BY sum(ss_net_profit) DESC) AS ranking + FROM store_sales, store, date_dim + WHERE d_month_seq BETWEEN 1200 AND 1200 + 11 + AND d_date_sk = ss_sold_date_sk + AND s_store_sk = ss_store_sk + GROUP BY s_state) tmp1 + WHERE ranking <= 5) +GROUP BY ROLLUP (s_state, s_county) +ORDER BY + lochierarchy DESC + , CASE WHEN lochierarchy = 0 + THEN s_state END + , rank_within_parent +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q71.sql b/plugin-spark/src/test/resources/tpcds/q71.sql new file mode 100755 index 0000000000..8d724b9244 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q71.sql @@ -0,0 +1,44 @@ +SELECT + i_brand_id brand_id, + i_brand brand, + t_hour, + t_minute, + sum(ext_price) ext_price +FROM item, + (SELECT + ws_ext_sales_price AS ext_price, + ws_sold_date_sk AS sold_date_sk, + ws_item_sk AS sold_item_sk, + ws_sold_time_sk AS time_sk + FROM web_sales, date_dim + WHERE d_date_sk = ws_sold_date_sk + AND d_moy = 11 + AND d_year = 1999 + UNION ALL + SELECT + cs_ext_sales_price AS ext_price, + cs_sold_date_sk AS sold_date_sk, + cs_item_sk AS sold_item_sk, + cs_sold_time_sk AS time_sk + FROM catalog_sales, date_dim + WHERE d_date_sk = cs_sold_date_sk + AND d_moy = 11 + AND d_year = 1999 + UNION ALL + SELECT + ss_ext_sales_price AS ext_price, + ss_sold_date_sk AS sold_date_sk, + ss_item_sk AS sold_item_sk, + ss_sold_time_sk AS time_sk + FROM store_sales, date_dim + WHERE d_date_sk = ss_sold_date_sk + AND d_moy = 11 + AND d_year = 1999 + ) AS tmp, time_dim +WHERE + sold_item_sk = i_item_sk + AND i_manager_id = 1 + AND time_sk = t_time_sk + AND (t_meal_time = 'breakfast' OR t_meal_time = 'dinner') +GROUP BY i_brand, i_brand_id, t_hour, t_minute +ORDER BY ext_price DESC, brand_id diff --git a/plugin-spark/src/test/resources/tpcds/q72.sql b/plugin-spark/src/test/resources/tpcds/q72.sql new file mode 100755 index 0000000000..99b3eee54a --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q72.sql @@ -0,0 +1,33 @@ +SELECT + i_item_desc, + w_warehouse_name, + d1.d_week_seq, + count(CASE WHEN p_promo_sk IS NULL + THEN 1 + ELSE 0 END) no_promo, + count(CASE WHEN p_promo_sk IS NOT NULL + THEN 1 + ELSE 0 END) promo, + count(*) total_cnt +FROM catalog_sales + JOIN inventory ON (cs_item_sk = inv_item_sk) + JOIN warehouse ON (w_warehouse_sk = inv_warehouse_sk) + JOIN item ON (i_item_sk = cs_item_sk) + JOIN customer_demographics ON (cs_bill_cdemo_sk = cd_demo_sk) + JOIN household_demographics ON (cs_bill_hdemo_sk = hd_demo_sk) + JOIN date_dim d1 ON (cs_sold_date_sk = d1.d_date_sk) + JOIN date_dim d2 ON (inv_date_sk = d2.d_date_sk) + JOIN date_dim d3 ON (cs_ship_date_sk = d3.d_date_sk) + LEFT OUTER JOIN promotion ON (cs_promo_sk = p_promo_sk) + LEFT OUTER JOIN catalog_returns ON (cr_item_sk = cs_item_sk AND cr_order_number = cs_order_number) +WHERE d1.d_week_seq = d2.d_week_seq + AND inv_quantity_on_hand < cs_quantity + AND d3.d_date > (cast(d1.d_date AS DATE) + interval 5 days) + AND hd_buy_potential = '>10000' + AND d1.d_year = 1999 + AND hd_buy_potential = '>10000' + AND cd_marital_status = 'D' + AND d1.d_year = 1999 +GROUP BY i_item_desc, w_warehouse_name, d1.d_week_seq +ORDER BY total_cnt DESC, i_item_desc, w_warehouse_name, d_week_seq +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q73.sql b/plugin-spark/src/test/resources/tpcds/q73.sql new file mode 100755 index 0000000000..881be2e902 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q73.sql @@ -0,0 +1,30 @@ +SELECT + c_last_name, + c_first_name, + c_salutation, + c_preferred_cust_flag, + ss_ticket_number, + cnt +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + count(*) cnt + FROM store_sales, date_dim, store, household_demographics + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND date_dim.d_dom BETWEEN 1 AND 2 + AND (household_demographics.hd_buy_potential = '>10000' OR + household_demographics.hd_buy_potential = 'unknown') + AND household_demographics.hd_vehicle_count > 0 + AND CASE WHEN household_demographics.hd_vehicle_count > 0 + THEN + household_demographics.hd_dep_count / household_demographics.hd_vehicle_count + ELSE NULL END > 1 + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_county IN ('Williamson County', 'Franklin Parish', 'Bronx County', 'Orange County') + GROUP BY ss_ticket_number, ss_customer_sk) dj, customer +WHERE ss_customer_sk = c_customer_sk + AND cnt BETWEEN 1 AND 5 +ORDER BY cnt DESC diff --git a/plugin-spark/src/test/resources/tpcds/q74.sql b/plugin-spark/src/test/resources/tpcds/q74.sql new file mode 100755 index 0000000000..154b26d680 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q74.sql @@ -0,0 +1,58 @@ +WITH year_total AS ( + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + d_year AS year, + sum(ss_net_paid) year_total, + 's' sale_type + FROM + customer, store_sales, date_dim + WHERE c_customer_sk = ss_customer_sk + AND ss_sold_date_sk = d_date_sk + AND d_year IN (2001, 2001 + 1) + GROUP BY + c_customer_id, c_first_name, c_last_name, d_year + UNION ALL + SELECT + c_customer_id customer_id, + c_first_name customer_first_name, + c_last_name customer_last_name, + d_year AS year, + sum(ws_net_paid) year_total, + 'w' sale_type + FROM + customer, web_sales, date_dim + WHERE c_customer_sk = ws_bill_customer_sk + AND ws_sold_date_sk = d_date_sk + AND d_year IN (2001, 2001 + 1) + GROUP BY + c_customer_id, c_first_name, c_last_name, d_year) +SELECT + t_s_secyear.customer_id, + t_s_secyear.customer_first_name, + t_s_secyear.customer_last_name +FROM + year_total t_s_firstyear, year_total t_s_secyear, + year_total t_w_firstyear, year_total t_w_secyear +WHERE t_s_secyear.customer_id = t_s_firstyear.customer_id + AND t_s_firstyear.customer_id = t_w_secyear.customer_id + AND t_s_firstyear.customer_id = t_w_firstyear.customer_id + AND t_s_firstyear.sale_type = 's' + AND t_w_firstyear.sale_type = 'w' + AND t_s_secyear.sale_type = 's' + AND t_w_secyear.sale_type = 'w' + AND t_s_firstyear.year = 2001 + AND t_s_secyear.year = 2001 + 1 + AND t_w_firstyear.year = 2001 + AND t_w_secyear.year = 2001 + 1 + AND t_s_firstyear.year_total > 0 + AND t_w_firstyear.year_total > 0 + AND CASE WHEN t_w_firstyear.year_total > 0 + THEN t_w_secyear.year_total / t_w_firstyear.year_total + ELSE NULL END + > CASE WHEN t_s_firstyear.year_total > 0 + THEN t_s_secyear.year_total / t_s_firstyear.year_total + ELSE NULL END +ORDER BY 1, 1, 1 +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q75.sql b/plugin-spark/src/test/resources/tpcds/q75.sql new file mode 100755 index 0000000000..2a143232b5 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q75.sql @@ -0,0 +1,76 @@ +WITH all_sales AS ( + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + SUM(sales_cnt) AS sales_cnt, + SUM(sales_amt) AS sales_amt + FROM ( + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + cs_quantity - COALESCE(cr_return_quantity, 0) AS sales_cnt, + cs_ext_sales_price - COALESCE(cr_return_amount, 0.0) AS sales_amt + FROM catalog_sales + JOIN item ON i_item_sk = cs_item_sk + JOIN date_dim ON d_date_sk = cs_sold_date_sk + LEFT JOIN catalog_returns ON (cs_order_number = cr_order_number + AND cs_item_sk = cr_item_sk) + WHERE i_category = 'Books' + UNION + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + ss_quantity - COALESCE(sr_return_quantity, 0) AS sales_cnt, + ss_ext_sales_price - COALESCE(sr_return_amt, 0.0) AS sales_amt + FROM store_sales + JOIN item ON i_item_sk = ss_item_sk + JOIN date_dim ON d_date_sk = ss_sold_date_sk + LEFT JOIN store_returns ON (ss_ticket_number = sr_ticket_number + AND ss_item_sk = sr_item_sk) + WHERE i_category = 'Books' + UNION + SELECT + d_year, + i_brand_id, + i_class_id, + i_category_id, + i_manufact_id, + ws_quantity - COALESCE(wr_return_quantity, 0) AS sales_cnt, + ws_ext_sales_price - COALESCE(wr_return_amt, 0.0) AS sales_amt + FROM web_sales + JOIN item ON i_item_sk = ws_item_sk + JOIN date_dim ON d_date_sk = ws_sold_date_sk + LEFT JOIN web_returns ON (ws_order_number = wr_order_number + AND ws_item_sk = wr_item_sk) + WHERE i_category = 'Books') sales_detail + GROUP BY d_year, i_brand_id, i_class_id, i_category_id, i_manufact_id) +SELECT + prev_yr.d_year AS prev_year, + curr_yr.d_year AS year, + curr_yr.i_brand_id, + curr_yr.i_class_id, + curr_yr.i_category_id, + curr_yr.i_manufact_id, + prev_yr.sales_cnt AS prev_yr_cnt, + curr_yr.sales_cnt AS curr_yr_cnt, + curr_yr.sales_cnt - prev_yr.sales_cnt AS sales_cnt_diff, + curr_yr.sales_amt - prev_yr.sales_amt AS sales_amt_diff +FROM all_sales curr_yr, all_sales prev_yr +WHERE curr_yr.i_brand_id = prev_yr.i_brand_id + AND curr_yr.i_class_id = prev_yr.i_class_id + AND curr_yr.i_category_id = prev_yr.i_category_id + AND curr_yr.i_manufact_id = prev_yr.i_manufact_id + AND curr_yr.d_year = 2002 + AND prev_yr.d_year = 2002 - 1 + AND CAST(curr_yr.sales_cnt AS DECIMAL(17, 2)) / CAST(prev_yr.sales_cnt AS DECIMAL(17, 2)) < 0.9 +ORDER BY sales_cnt_diff +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q76.sql b/plugin-spark/src/test/resources/tpcds/q76.sql new file mode 100755 index 0000000000..815fa922be --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q76.sql @@ -0,0 +1,47 @@ +SELECT + channel, + col_name, + d_year, + d_qoy, + i_category, + COUNT(*) sales_cnt, + SUM(ext_sales_price) sales_amt +FROM ( + SELECT + 'store' AS channel, + ss_store_sk col_name, + d_year, + d_qoy, + i_category, + ss_ext_sales_price ext_sales_price + FROM store_sales, item, date_dim + WHERE ss_store_sk IS NULL + AND ss_sold_date_sk = d_date_sk + AND ss_item_sk = i_item_sk + UNION ALL + SELECT + 'web' AS channel, + ws_ship_customer_sk col_name, + d_year, + d_qoy, + i_category, + ws_ext_sales_price ext_sales_price + FROM web_sales, item, date_dim + WHERE ws_ship_customer_sk IS NULL + AND ws_sold_date_sk = d_date_sk + AND ws_item_sk = i_item_sk + UNION ALL + SELECT + 'catalog' AS channel, + cs_ship_addr_sk col_name, + d_year, + d_qoy, + i_category, + cs_ext_sales_price ext_sales_price + FROM catalog_sales, item, date_dim + WHERE cs_ship_addr_sk IS NULL + AND cs_sold_date_sk = d_date_sk + AND cs_item_sk = i_item_sk) foo +GROUP BY channel, col_name, d_year, d_qoy, i_category +ORDER BY channel, col_name, d_year, d_qoy, i_category +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q77.sql b/plugin-spark/src/test/resources/tpcds/q77.sql new file mode 100755 index 0000000000..a69df9fbcd --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q77.sql @@ -0,0 +1,100 @@ +WITH ss AS +(SELECT + s_store_sk, + sum(ss_ext_sales_price) AS sales, + sum(ss_net_profit) AS profit + FROM store_sales, date_dim, store + WHERE ss_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND ss_store_sk = s_store_sk + GROUP BY s_store_sk), + sr AS + (SELECT + s_store_sk, + sum(sr_return_amt) AS returns, + sum(sr_net_loss) AS profit_loss + FROM store_returns, date_dim, store + WHERE sr_returned_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND sr_store_sk = s_store_sk + GROUP BY s_store_sk), + cs AS + (SELECT + cs_call_center_sk, + sum(cs_ext_sales_price) AS sales, + sum(cs_net_profit) AS profit + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + GROUP BY cs_call_center_sk), + cr AS + (SELECT + sum(cr_return_amount) AS returns, + sum(cr_net_loss) AS profit_loss + FROM catalog_returns, date_dim + WHERE cr_returned_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days)), + ws AS + (SELECT + wp_web_page_sk, + sum(ws_ext_sales_price) AS sales, + sum(ws_net_profit) AS profit + FROM web_sales, date_dim, web_page + WHERE ws_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND ws_web_page_sk = wp_web_page_sk + GROUP BY wp_web_page_sk), + wr AS + (SELECT + wp_web_page_sk, + sum(wr_return_amt) AS returns, + sum(wr_net_loss) AS profit_loss + FROM web_returns, date_dim, web_page + WHERE wr_returned_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-03' AS DATE) AND + (cast('2000-08-03' AS DATE) + INTERVAL 30 days) + AND wr_web_page_sk = wp_web_page_sk + GROUP BY wp_web_page_sk) +SELECT + channel, + id, + sum(sales) AS sales, + sum(returns) AS returns, + sum(profit) AS profit +FROM + (SELECT + 'store channel' AS channel, + ss.s_store_sk AS id, + sales, + coalesce(returns, 0) AS returns, + (profit - coalesce(profit_loss, 0)) AS profit + FROM ss + LEFT JOIN sr + ON ss.s_store_sk = sr.s_store_sk + UNION ALL + SELECT + 'catalog channel' AS channel, + cs_call_center_sk AS id, + sales, + returns, + (profit - profit_loss) AS profit + FROM cs, cr + UNION ALL + SELECT + 'web channel' AS channel, + ws.wp_web_page_sk AS id, + sales, + coalesce(returns, 0) returns, + (profit - coalesce(profit_loss, 0)) AS profit + FROM ws + LEFT JOIN wr + ON ws.wp_web_page_sk = wr.wp_web_page_sk + ) x +GROUP BY ROLLUP (channel, id) +ORDER BY channel, id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q78.sql b/plugin-spark/src/test/resources/tpcds/q78.sql new file mode 100755 index 0000000000..07b0940e26 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q78.sql @@ -0,0 +1,64 @@ +WITH ws AS +(SELECT + d_year AS ws_sold_year, + ws_item_sk, + ws_bill_customer_sk ws_customer_sk, + sum(ws_quantity) ws_qty, + sum(ws_wholesale_cost) ws_wc, + sum(ws_sales_price) ws_sp + FROM web_sales + LEFT JOIN web_returns ON wr_order_number = ws_order_number AND ws_item_sk = wr_item_sk + JOIN date_dim ON ws_sold_date_sk = d_date_sk + WHERE wr_order_number IS NULL + GROUP BY d_year, ws_item_sk, ws_bill_customer_sk +), + cs AS + (SELECT + d_year AS cs_sold_year, + cs_item_sk, + cs_bill_customer_sk cs_customer_sk, + sum(cs_quantity) cs_qty, + sum(cs_wholesale_cost) cs_wc, + sum(cs_sales_price) cs_sp + FROM catalog_sales + LEFT JOIN catalog_returns ON cr_order_number = cs_order_number AND cs_item_sk = cr_item_sk + JOIN date_dim ON cs_sold_date_sk = d_date_sk + WHERE cr_order_number IS NULL + GROUP BY d_year, cs_item_sk, cs_bill_customer_sk + ), + ss AS + (SELECT + d_year AS ss_sold_year, + ss_item_sk, + ss_customer_sk, + sum(ss_quantity) ss_qty, + sum(ss_wholesale_cost) ss_wc, + sum(ss_sales_price) ss_sp + FROM store_sales + LEFT JOIN store_returns ON sr_ticket_number = ss_ticket_number AND ss_item_sk = sr_item_sk + JOIN date_dim ON ss_sold_date_sk = d_date_sk + WHERE sr_ticket_number IS NULL + GROUP BY d_year, ss_item_sk, ss_customer_sk + ) +SELECT + round(ss_qty / (coalesce(ws_qty + cs_qty, 1)), 2) ratio, + ss_qty store_qty, + ss_wc store_wholesale_cost, + ss_sp store_sales_price, + coalesce(ws_qty, 0) + coalesce(cs_qty, 0) other_chan_qty, + coalesce(ws_wc, 0) + coalesce(cs_wc, 0) other_chan_wholesale_cost, + coalesce(ws_sp, 0) + coalesce(cs_sp, 0) other_chan_sales_price +FROM ss + LEFT JOIN ws + ON (ws_sold_year = ss_sold_year AND ws_item_sk = ss_item_sk AND ws_customer_sk = ss_customer_sk) + LEFT JOIN cs + ON (cs_sold_year = ss_sold_year AND cs_item_sk = ss_item_sk AND cs_customer_sk = ss_customer_sk) +WHERE coalesce(ws_qty, 0) > 0 AND coalesce(cs_qty, 0) > 0 AND ss_sold_year = 2000 +ORDER BY + ratio, + ss_qty DESC, ss_wc DESC, ss_sp DESC, + other_chan_qty, + other_chan_wholesale_cost, + other_chan_sales_price, + round(ss_qty / (coalesce(ws_qty + cs_qty, 1)), 2) +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q79.sql b/plugin-spark/src/test/resources/tpcds/q79.sql new file mode 100755 index 0000000000..08f86dc203 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q79.sql @@ -0,0 +1,27 @@ +SELECT + c_last_name, + c_first_name, + substr(s_city, 1, 30), + ss_ticket_number, + amt, + profit +FROM + (SELECT + ss_ticket_number, + ss_customer_sk, + store.s_city, + sum(ss_coupon_amt) amt, + sum(ss_net_profit) profit + FROM store_sales, date_dim, store, household_demographics + WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_store_sk = store.s_store_sk + AND store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk + AND (household_demographics.hd_dep_count = 6 OR + household_demographics.hd_vehicle_count > 2) + AND date_dim.d_dow = 1 + AND date_dim.d_year IN (1999, 1999 + 1, 1999 + 2) + AND store.s_number_employees BETWEEN 200 AND 295 + GROUP BY ss_ticket_number, ss_customer_sk, ss_addr_sk, store.s_city) ms, customer +WHERE ss_customer_sk = c_customer_sk +ORDER BY c_last_name, c_first_name, substr(s_city, 1, 30), profit +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q8.sql b/plugin-spark/src/test/resources/tpcds/q8.sql new file mode 100755 index 0000000000..497725111f --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q8.sql @@ -0,0 +1,87 @@ +SELECT + s_store_name, + sum(ss_net_profit) +FROM store_sales, date_dim, store, + (SELECT ca_zip + FROM ( + (SELECT substr(ca_zip, 1, 5) ca_zip + FROM customer_address + WHERE substr(ca_zip, 1, 5) IN ( + '24128','76232','65084','87816','83926','77556','20548', + '26231','43848','15126','91137','61265','98294','25782', + '17920','18426','98235','40081','84093','28577','55565', + '17183','54601','67897','22752','86284','18376','38607', + '45200','21756','29741','96765','23932','89360','29839', + '25989','28898','91068','72550','10390','18845','47770', + '82636','41367','76638','86198','81312','37126','39192', + '88424','72175','81426','53672','10445','42666','66864', + '66708','41248','48583','82276','18842','78890','49448', + '14089','38122','34425','79077','19849','43285','39861', + '66162','77610','13695','99543','83444','83041','12305', + '57665','68341','25003','57834','62878','49130','81096', + '18840','27700','23470','50412','21195','16021','76107', + '71954','68309','18119','98359','64544','10336','86379', + '27068','39736','98569','28915','24206','56529','57647', + '54917','42961','91110','63981','14922','36420','23006', + '67467','32754','30903','20260','31671','51798','72325', + '85816','68621','13955','36446','41766','68806','16725', + '15146','22744','35850','88086','51649','18270','52867', + '39972','96976','63792','11376','94898','13595','10516', + '90225','58943','39371','94945','28587','96576','57855', + '28488','26105','83933','25858','34322','44438','73171', + '30122','34102','22685','71256','78451','54364','13354', + '45375','40558','56458','28286','45266','47305','69399', + '83921','26233','11101','15371','69913','35942','15882', + '25631','24610','44165','99076','33786','70738','26653', + '14328','72305','62496','22152','10144','64147','48425', + '14663','21076','18799','30450','63089','81019','68893', + '24996','51200','51211','45692','92712','70466','79994', + '22437','25280','38935','71791','73134','56571','14060', + '19505','72425','56575','74351','68786','51650','20004', + '18383','76614','11634','18906','15765','41368','73241', + '76698','78567','97189','28545','76231','75691','22246', + '51061','90578','56691','68014','51103','94167','57047', + '14867','73520','15734','63435','25733','35474','24676', + '94627','53535','17879','15559','53268','59166','11928', + '59402','33282','45721','43933','68101','33515','36634', + '71286','19736','58058','55253','67473','41918','19515', + '36495','19430','22351','77191','91393','49156','50298', + '87501','18652','53179','18767','63193','23968','65164', + '68880','21286','72823','58470','67301','13394','31016', + '70372','67030','40604','24317','45748','39127','26065', + '77721','31029','31880','60576','24671','45549','13376', + '50016','33123','19769','22927','97789','46081','72151', + '15723','46136','51949','68100','96888','64528','14171', + '79777','28709','11489','25103','32213','78668','22245', + '15798','27156','37930','62971','21337','51622','67853', + '10567','38415','15455','58263','42029','60279','37125', + '56240','88190','50308','26859','64457','89091','82136', + '62377','36233','63837','58078','17043','30010','60099', + '28810','98025','29178','87343','73273','30469','64034', + '39516','86057','21309','90257','67875','40162','11356', + '73650','61810','72013','30431','22461','19512','13375', + '55307','30625','83849','68908','26689','96451','38193', + '46820','88885','84935','69035','83144','47537','56616', + '94983','48033','69952','25486','61547','27385','61860', + '58048','56910','16807','17871','35258','31387','35458', + '35576')) + INTERSECT + (SELECT ca_zip + FROM + (SELECT + substr(ca_zip, 1, 5) ca_zip, + count(*) cnt + FROM customer_address, customer + WHERE ca_address_sk = c_current_addr_sk AND + c_preferred_cust_flag = 'Y' + GROUP BY ca_zip + HAVING count(*) > 10) A1) + ) A2 + ) V1 +WHERE ss_store_sk = s_store_sk + AND ss_sold_date_sk = d_date_sk + AND d_qoy = 2 AND d_year = 1998 + AND (substr(s_zip, 1, 2) = substr(V1.ca_zip, 1, 2)) +GROUP BY s_store_name +ORDER BY s_store_name +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q80.sql b/plugin-spark/src/test/resources/tpcds/q80.sql new file mode 100755 index 0000000000..433db87d2a --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q80.sql @@ -0,0 +1,94 @@ +WITH ssr AS +(SELECT + s_store_id AS store_id, + sum(ss_ext_sales_price) AS sales, + sum(coalesce(sr_return_amt, 0)) AS returns, + sum(ss_net_profit - coalesce(sr_net_loss, 0)) AS profit + FROM store_sales + LEFT OUTER JOIN store_returns ON + (ss_item_sk = sr_item_sk AND + ss_ticket_number = sr_ticket_number) + , + date_dim, store, item, promotion + WHERE ss_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) + AND ss_store_sk = s_store_sk + AND ss_item_sk = i_item_sk + AND i_current_price > 50 + AND ss_promo_sk = p_promo_sk + AND p_channel_tv = 'N' + GROUP BY s_store_id), + csr AS + (SELECT + cp_catalog_page_id AS catalog_page_id, + sum(cs_ext_sales_price) AS sales, + sum(coalesce(cr_return_amount, 0)) AS returns, + sum(cs_net_profit - coalesce(cr_net_loss, 0)) AS profit + FROM catalog_sales + LEFT OUTER JOIN catalog_returns ON + (cs_item_sk = cr_item_sk AND + cs_order_number = cr_order_number) + , + date_dim, catalog_page, item, promotion + WHERE cs_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) + AND cs_catalog_page_sk = cp_catalog_page_sk + AND cs_item_sk = i_item_sk + AND i_current_price > 50 + AND cs_promo_sk = p_promo_sk + AND p_channel_tv = 'N' + GROUP BY cp_catalog_page_id), + wsr AS + (SELECT + web_site_id, + sum(ws_ext_sales_price) AS sales, + sum(coalesce(wr_return_amt, 0)) AS returns, + sum(ws_net_profit - coalesce(wr_net_loss, 0)) AS profit + FROM web_sales + LEFT OUTER JOIN web_returns ON + (ws_item_sk = wr_item_sk AND ws_order_number = wr_order_number) + , + date_dim, web_site, item, promotion + WHERE ws_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('2000-08-23' AS DATE) + AND (cast('2000-08-23' AS DATE) + INTERVAL 30 days) + AND ws_web_site_sk = web_site_sk + AND ws_item_sk = i_item_sk + AND i_current_price > 50 + AND ws_promo_sk = p_promo_sk + AND p_channel_tv = 'N' + GROUP BY web_site_id) +SELECT + channel, + id, + sum(sales) AS sales, + sum(returns) AS returns, + sum(profit) AS profit +FROM (SELECT + 'store channel' AS channel, + concat('store', store_id) AS id, + sales, + returns, + profit + FROM ssr + UNION ALL + SELECT + 'catalog channel' AS channel, + concat('catalog_page', catalog_page_id) AS id, + sales, + returns, + profit + FROM csr + UNION ALL + SELECT + 'web channel' AS channel, + concat('web_site', web_site_id) AS id, + sales, + returns, + profit + FROM wsr) x +GROUP BY ROLLUP (channel, id) +ORDER BY channel, id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q81.sql b/plugin-spark/src/test/resources/tpcds/q81.sql new file mode 100755 index 0000000000..18f0ffa7e8 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q81.sql @@ -0,0 +1,38 @@ +WITH customer_total_return AS +(SELECT + cr_returning_customer_sk AS ctr_customer_sk, + ca_state AS ctr_state, + sum(cr_return_amt_inc_tax) AS ctr_total_return + FROM catalog_returns, date_dim, customer_address + WHERE cr_returned_date_sk = d_date_sk + AND d_year = 2000 + AND cr_returning_addr_sk = ca_address_sk + GROUP BY cr_returning_customer_sk, ca_state ) +SELECT + c_customer_id, + c_salutation, + c_first_name, + c_last_name, + ca_street_number, + ca_street_name, + ca_street_type, + ca_suite_number, + ca_city, + ca_county, + ca_state, + ca_zip, + ca_country, + ca_gmt_offset, + ca_location_type, + ctr_total_return +FROM customer_total_return ctr1, customer_address, customer +WHERE ctr1.ctr_total_return > (SELECT avg(ctr_total_return) * 1.2 +FROM customer_total_return ctr2 +WHERE ctr1.ctr_state = ctr2.ctr_state) + AND ca_address_sk = c_current_addr_sk + AND ca_state = 'GA' + AND ctr1.ctr_customer_sk = c_customer_sk +ORDER BY c_customer_id, c_salutation, c_first_name, c_last_name, ca_street_number, ca_street_name + , ca_street_type, ca_suite_number, ca_city, ca_county, ca_state, ca_zip, ca_country, ca_gmt_offset + , ca_location_type, ctr_total_return +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q82.sql b/plugin-spark/src/test/resources/tpcds/q82.sql new file mode 100755 index 0000000000..20942cfeb0 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q82.sql @@ -0,0 +1,15 @@ +SELECT + i_item_id, + i_item_desc, + i_current_price +FROM item, inventory, date_dim, store_sales +WHERE i_current_price BETWEEN 62 AND 62 + 30 + AND inv_item_sk = i_item_sk + AND d_date_sk = inv_date_sk + AND d_date BETWEEN cast('2000-05-25' AS DATE) AND (cast('2000-05-25' AS DATE) + INTERVAL 60 days) + AND i_manufact_id IN (129, 270, 821, 423) + AND inv_quantity_on_hand BETWEEN 100 AND 500 + AND ss_item_sk = i_item_sk +GROUP BY i_item_id, i_item_desc, i_current_price +ORDER BY i_item_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q83.sql b/plugin-spark/src/test/resources/tpcds/q83.sql new file mode 100755 index 0000000000..53c10c7ded --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q83.sql @@ -0,0 +1,56 @@ +WITH sr_items AS +(SELECT + i_item_id item_id, + sum(sr_return_quantity) sr_item_qty + FROM store_returns, item, date_dim + WHERE sr_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq IN + (SELECT d_week_seq + FROM date_dim + WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) + AND sr_returned_date_sk = d_date_sk + GROUP BY i_item_id), + cr_items AS + (SELECT + i_item_id item_id, + sum(cr_return_quantity) cr_item_qty + FROM catalog_returns, item, date_dim + WHERE cr_item_sk = i_item_sk + AND d_date IN (SELECT d_date + FROM date_dim + WHERE d_week_seq IN + (SELECT d_week_seq + FROM date_dim + WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) + AND cr_returned_date_sk = d_date_sk + GROUP BY i_item_id), + wr_items AS + (SELECT + i_item_id item_id, + sum(wr_return_quantity) wr_item_qty + FROM web_returns, item, date_dim + WHERE wr_item_sk = i_item_sk AND d_date IN + (SELECT d_date + FROM date_dim + WHERE d_week_seq IN + (SELECT d_week_seq + FROM date_dim + WHERE d_date IN ('2000-06-30', '2000-09-27', '2000-11-17'))) + AND wr_returned_date_sk = d_date_sk + GROUP BY i_item_id) +SELECT + sr_items.item_id, + sr_item_qty, + sr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 sr_dev, + cr_item_qty, + cr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 cr_dev, + wr_item_qty, + wr_item_qty / (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 * 100 wr_dev, + (sr_item_qty + cr_item_qty + wr_item_qty) / 3.0 average +FROM sr_items, cr_items, wr_items +WHERE sr_items.item_id = cr_items.item_id + AND sr_items.item_id = wr_items.item_id +ORDER BY sr_items.item_id, sr_item_qty +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q84.sql b/plugin-spark/src/test/resources/tpcds/q84.sql new file mode 100755 index 0000000000..a1076b57ce --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q84.sql @@ -0,0 +1,19 @@ +SELECT + c_customer_id AS customer_id, + concat(c_last_name, ', ', c_first_name) AS customername +FROM customer + , customer_address + , customer_demographics + , household_demographics + , income_band + , store_returns +WHERE ca_city = 'Edgewood' + AND c_current_addr_sk = ca_address_sk + AND ib_lower_bound >= 38128 + AND ib_upper_bound <= 38128 + 50000 + AND ib_income_band_sk = hd_income_band_sk + AND cd_demo_sk = c_current_cdemo_sk + AND hd_demo_sk = c_current_hdemo_sk + AND sr_cdemo_sk = cd_demo_sk +ORDER BY c_customer_id +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q85.sql b/plugin-spark/src/test/resources/tpcds/q85.sql new file mode 100755 index 0000000000..cf718b0f8a --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q85.sql @@ -0,0 +1,82 @@ +SELECT + substr(r_reason_desc, 1, 20), + avg(ws_quantity), + avg(wr_refunded_cash), + avg(wr_fee) +FROM web_sales, web_returns, web_page, customer_demographics cd1, + customer_demographics cd2, customer_address, date_dim, reason +WHERE ws_web_page_sk = wp_web_page_sk + AND ws_item_sk = wr_item_sk + AND ws_order_number = wr_order_number + AND ws_sold_date_sk = d_date_sk AND d_year = 2000 + AND cd1.cd_demo_sk = wr_refunded_cdemo_sk + AND cd2.cd_demo_sk = wr_returning_cdemo_sk + AND ca_address_sk = wr_refunded_addr_sk + AND r_reason_sk = wr_reason_sk + AND + ( + ( + cd1.cd_marital_status = 'M' + AND + cd1.cd_marital_status = cd2.cd_marital_status + AND + cd1.cd_education_status = 'Advanced Degree' + AND + cd1.cd_education_status = cd2.cd_education_status + AND + ws_sales_price BETWEEN 100.00 AND 150.00 + ) + OR + ( + cd1.cd_marital_status = 'S' + AND + cd1.cd_marital_status = cd2.cd_marital_status + AND + cd1.cd_education_status = 'College' + AND + cd1.cd_education_status = cd2.cd_education_status + AND + ws_sales_price BETWEEN 50.00 AND 100.00 + ) + OR + ( + cd1.cd_marital_status = 'W' + AND + cd1.cd_marital_status = cd2.cd_marital_status + AND + cd1.cd_education_status = '2 yr Degree' + AND + cd1.cd_education_status = cd2.cd_education_status + AND + ws_sales_price BETWEEN 150.00 AND 200.00 + ) + ) + AND + ( + ( + ca_country = 'United States' + AND + ca_state IN ('IN', 'OH', 'NJ') + AND ws_net_profit BETWEEN 100 AND 200 + ) + OR + ( + ca_country = 'United States' + AND + ca_state IN ('WI', 'CT', 'KY') + AND ws_net_profit BETWEEN 150 AND 300 + ) + OR + ( + ca_country = 'United States' + AND + ca_state IN ('LA', 'IA', 'AR') + AND ws_net_profit BETWEEN 50 AND 250 + ) + ) +GROUP BY r_reason_desc +ORDER BY substr(r_reason_desc, 1, 20) + , avg(ws_quantity) + , avg(wr_refunded_cash) + , avg(wr_fee) +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q86.sql b/plugin-spark/src/test/resources/tpcds/q86.sql new file mode 100755 index 0000000000..789a4abf7b --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q86.sql @@ -0,0 +1,24 @@ +SELECT + sum(ws_net_paid) AS total_sum, + i_category, + i_class, + grouping(i_category) + grouping(i_class) AS lochierarchy, + rank() + OVER ( + PARTITION BY grouping(i_category) + grouping(i_class), + CASE WHEN grouping(i_class) = 0 + THEN i_category END + ORDER BY sum(ws_net_paid) DESC) AS rank_within_parent +FROM + web_sales, date_dim d1, item +WHERE + d1.d_month_seq BETWEEN 1200 AND 1200 + 11 + AND d1.d_date_sk = ws_sold_date_sk + AND i_item_sk = ws_item_sk +GROUP BY ROLLUP (i_category, i_class) +ORDER BY + lochierarchy DESC, + CASE WHEN lochierarchy = 0 + THEN i_category END, + rank_within_parent +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q87.sql b/plugin-spark/src/test/resources/tpcds/q87.sql new file mode 100755 index 0000000000..4aaa9f39dc --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q87.sql @@ -0,0 +1,28 @@ +SELECT count(*) +FROM ((SELECT DISTINCT + c_last_name, + c_first_name, + d_date +FROM store_sales, date_dim, customer +WHERE store_sales.ss_sold_date_sk = date_dim.d_date_sk + AND store_sales.ss_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11) + EXCEPT + (SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM catalog_sales, date_dim, customer + WHERE catalog_sales.cs_sold_date_sk = date_dim.d_date_sk + AND catalog_sales.cs_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11) + EXCEPT + (SELECT DISTINCT + c_last_name, + c_first_name, + d_date + FROM web_sales, date_dim, customer + WHERE web_sales.ws_sold_date_sk = date_dim.d_date_sk + AND web_sales.ws_bill_customer_sk = customer.c_customer_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11) + ) cool_cust diff --git a/plugin-spark/src/test/resources/tpcds/q88.sql b/plugin-spark/src/test/resources/tpcds/q88.sql new file mode 100755 index 0000000000..25bcd90f41 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q88.sql @@ -0,0 +1,122 @@ +SELECT * +FROM + (SELECT count(*) h8_30_to_9 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 8 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s1, + (SELECT count(*) h9_to_9_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 9 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s2, + (SELECT count(*) h9_30_to_10 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 9 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s3, + (SELECT count(*) h10_to_10_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 10 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s4, + (SELECT count(*) h10_30_to_11 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 10 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s5, + (SELECT count(*) h11_to_11_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 11 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s6, + (SELECT count(*) h11_30_to_12 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 11 + AND time_dim.t_minute >= 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s7, + (SELECT count(*) h12_to_12_30 + FROM store_sales, household_demographics, time_dim, store + WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 12 + AND time_dim.t_minute < 30 + AND ( + (household_demographics.hd_dep_count = 4 AND household_demographics.hd_vehicle_count <= 4 + 2) + OR + (household_demographics.hd_dep_count = 2 AND household_demographics.hd_vehicle_count <= 2 + 2) + OR + (household_demographics.hd_dep_count = 0 AND + household_demographics.hd_vehicle_count <= 0 + 2)) + AND store.s_store_name = 'ese') s8 diff --git a/plugin-spark/src/test/resources/tpcds/q89.sql b/plugin-spark/src/test/resources/tpcds/q89.sql new file mode 100755 index 0000000000..75408cb032 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q89.sql @@ -0,0 +1,30 @@ +SELECT * +FROM ( + SELECT + i_category, + i_class, + i_brand, + s_store_name, + s_company_name, + d_moy, + sum(ss_sales_price) sum_sales, + avg(sum(ss_sales_price)) + OVER + (PARTITION BY i_category, i_brand, s_store_name, s_company_name) + avg_monthly_sales + FROM item, store_sales, date_dim, store + WHERE ss_item_sk = i_item_sk AND + ss_sold_date_sk = d_date_sk AND + ss_store_sk = s_store_sk AND + d_year IN (1999) AND + ((i_category IN ('Books', 'Electronics', 'Sports') AND + i_class IN ('computers', 'stereo', 'football')) + OR (i_category IN ('Men', 'Jewelry', 'Women') AND + i_class IN ('shirts', 'birdal', 'dresses'))) + GROUP BY i_category, i_class, i_brand, + s_store_name, s_company_name, d_moy) tmp1 +WHERE CASE WHEN (avg_monthly_sales <> 0) + THEN (abs(sum_sales - avg_monthly_sales) / avg_monthly_sales) + ELSE NULL END > 0.1 +ORDER BY sum_sales - avg_monthly_sales, s_store_name +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q9.sql b/plugin-spark/src/test/resources/tpcds/q9.sql new file mode 100755 index 0000000000..de3db9d988 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q9.sql @@ -0,0 +1,48 @@ +SELECT + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 1 AND 20) > 62316685 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 1 AND 20) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 1 AND 20) END bucket1, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 40) > 19045798 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 40) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 21 AND 40) END bucket2, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 41 AND 60) > 365541424 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 41 AND 60) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 41 AND 60) END bucket3, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 61 AND 80) > 216357808 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 61 AND 80) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 61 AND 80) END bucket4, + CASE WHEN (SELECT count(*) + FROM store_sales + WHERE ss_quantity BETWEEN 81 AND 100) > 184483884 + THEN (SELECT avg(ss_ext_discount_amt) + FROM store_sales + WHERE ss_quantity BETWEEN 81 AND 100) + ELSE (SELECT avg(ss_net_paid) + FROM store_sales + WHERE ss_quantity BETWEEN 81 AND 100) END bucket5 +FROM reason +WHERE r_reason_sk = 1 diff --git a/plugin-spark/src/test/resources/tpcds/q90.sql b/plugin-spark/src/test/resources/tpcds/q90.sql new file mode 100755 index 0000000000..85e35bf8bf --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q90.sql @@ -0,0 +1,19 @@ +SELECT cast(amc AS DECIMAL(15, 4)) / cast(pmc AS DECIMAL(15, 4)) am_pm_ratio +FROM (SELECT count(*) amc +FROM web_sales, household_demographics, time_dim, web_page +WHERE ws_sold_time_sk = time_dim.t_time_sk + AND ws_ship_hdemo_sk = household_demographics.hd_demo_sk + AND ws_web_page_sk = web_page.wp_web_page_sk + AND time_dim.t_hour BETWEEN 8 AND 8 + 1 + AND household_demographics.hd_dep_count = 6 + AND web_page.wp_char_count BETWEEN 5000 AND 5200) at, + (SELECT count(*) pmc + FROM web_sales, household_demographics, time_dim, web_page + WHERE ws_sold_time_sk = time_dim.t_time_sk + AND ws_ship_hdemo_sk = household_demographics.hd_demo_sk + AND ws_web_page_sk = web_page.wp_web_page_sk + AND time_dim.t_hour BETWEEN 19 AND 19 + 1 + AND household_demographics.hd_dep_count = 6 + AND web_page.wp_char_count BETWEEN 5000 AND 5200) pt +ORDER BY am_pm_ratio +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q91.sql b/plugin-spark/src/test/resources/tpcds/q91.sql new file mode 100755 index 0000000000..9ca7ce00ac --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q91.sql @@ -0,0 +1,23 @@ +SELECT + cc_call_center_id Call_Center, + cc_name Call_Center_Name, + cc_manager Manager, + sum(cr_net_loss) Returns_Loss +FROM + call_center, catalog_returns, date_dim, customer, customer_address, + customer_demographics, household_demographics +WHERE + cr_call_center_sk = cc_call_center_sk + AND cr_returned_date_sk = d_date_sk + AND cr_returning_customer_sk = c_customer_sk + AND cd_demo_sk = c_current_cdemo_sk + AND hd_demo_sk = c_current_hdemo_sk + AND ca_address_sk = c_current_addr_sk + AND d_year = 1998 + AND d_moy = 11 + AND ((cd_marital_status = 'M' AND cd_education_status = 'Unknown') + OR (cd_marital_status = 'W' AND cd_education_status = 'Advanced Degree')) + AND hd_buy_potential LIKE 'Unknown%' + AND ca_gmt_offset = -7 +GROUP BY cc_call_center_id, cc_name, cc_manager, cd_marital_status, cd_education_status +ORDER BY sum(cr_net_loss) DESC diff --git a/plugin-spark/src/test/resources/tpcds/q92.sql b/plugin-spark/src/test/resources/tpcds/q92.sql new file mode 100755 index 0000000000..99129c3bd9 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q92.sql @@ -0,0 +1,16 @@ +SELECT sum(ws_ext_discount_amt) AS `Excess Discount Amount ` +FROM web_sales, item, date_dim +WHERE i_manufact_id = 350 + AND i_item_sk = ws_item_sk + AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + INTERVAL 90 days) + AND d_date_sk = ws_sold_date_sk + AND ws_ext_discount_amt > + ( + SELECT 1.3 * avg(ws_ext_discount_amt) + FROM web_sales, date_dim + WHERE ws_item_sk = i_item_sk + AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + INTERVAL 90 days) + AND d_date_sk = ws_sold_date_sk + ) +ORDER BY sum(ws_ext_discount_amt) +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q93.sql b/plugin-spark/src/test/resources/tpcds/q93.sql new file mode 100755 index 0000000000..222dc31c1f --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q93.sql @@ -0,0 +1,19 @@ +SELECT + ss_customer_sk, + sum(act_sales) sumsales +FROM (SELECT + ss_item_sk, + ss_ticket_number, + ss_customer_sk, + CASE WHEN sr_return_quantity IS NOT NULL + THEN (ss_quantity - sr_return_quantity) * ss_sales_price + ELSE (ss_quantity * ss_sales_price) END act_sales +FROM store_sales + LEFT OUTER JOIN store_returns + ON (sr_item_sk = ss_item_sk AND sr_ticket_number = ss_ticket_number) + , + reason +WHERE sr_reason_sk = r_reason_sk AND r_reason_desc = 'reason 28') t +GROUP BY ss_customer_sk +ORDER BY sumsales, ss_customer_sk +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q94.sql b/plugin-spark/src/test/resources/tpcds/q94.sql new file mode 100755 index 0000000000..d6de3d75b8 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q94.sql @@ -0,0 +1,23 @@ +SELECT + count(DISTINCT ws_order_number) AS `order count `, + sum(ws_ext_ship_cost) AS `total shipping cost `, + sum(ws_net_profit) AS `total net profit ` +FROM + web_sales ws1, date_dim, customer_address, web_site +WHERE + d_date BETWEEN '1999-02-01' AND + (CAST('1999-02-01' AS DATE) + INTERVAL 60 days) + AND ws1.ws_ship_date_sk = d_date_sk + AND ws1.ws_ship_addr_sk = ca_address_sk + AND ca_state = 'IL' + AND ws1.ws_web_site_sk = web_site_sk + AND web_company_name = 'pri' + AND EXISTS(SELECT * + FROM web_sales ws2 + WHERE ws1.ws_order_number = ws2.ws_order_number + AND ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) + AND NOT EXISTS(SELECT * + FROM web_returns wr1 + WHERE ws1.ws_order_number = wr1.wr_order_number) +ORDER BY count(DISTINCT ws_order_number) +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q95.sql b/plugin-spark/src/test/resources/tpcds/q95.sql new file mode 100755 index 0000000000..df71f00bd6 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q95.sql @@ -0,0 +1,29 @@ +WITH ws_wh AS +(SELECT + ws1.ws_order_number, + ws1.ws_warehouse_sk wh1, + ws2.ws_warehouse_sk wh2 + FROM web_sales ws1, web_sales ws2 + WHERE ws1.ws_order_number = ws2.ws_order_number + AND ws1.ws_warehouse_sk <> ws2.ws_warehouse_sk) +SELECT + count(DISTINCT ws_order_number) AS `order count `, + sum(ws_ext_ship_cost) AS `total shipping cost `, + sum(ws_net_profit) AS `total net profit ` +FROM + web_sales ws1, date_dim, customer_address, web_site +WHERE + d_date BETWEEN '1999-02-01' AND + (CAST('1999-02-01' AS DATE) + INTERVAL 60 DAY) + AND ws1.ws_ship_date_sk = d_date_sk + AND ws1.ws_ship_addr_sk = ca_address_sk + AND ca_state = 'IL' + AND ws1.ws_web_site_sk = web_site_sk + AND web_company_name = 'pri' + AND ws1.ws_order_number IN (SELECT ws_order_number + FROM ws_wh) + AND ws1.ws_order_number IN (SELECT wr_order_number + FROM web_returns, ws_wh + WHERE wr_order_number = ws_wh.ws_order_number) +ORDER BY count(DISTINCT ws_order_number) +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q96.sql b/plugin-spark/src/test/resources/tpcds/q96.sql new file mode 100755 index 0000000000..7ab17e7bc4 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q96.sql @@ -0,0 +1,11 @@ +SELECT count(*) +FROM store_sales, household_demographics, time_dim, store +WHERE ss_sold_time_sk = time_dim.t_time_sk + AND ss_hdemo_sk = household_demographics.hd_demo_sk + AND ss_store_sk = s_store_sk + AND time_dim.t_hour = 20 + AND time_dim.t_minute >= 30 + AND household_demographics.hd_dep_count = 7 + AND store.s_store_name = 'ese' +ORDER BY count(*) +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q97.sql b/plugin-spark/src/test/resources/tpcds/q97.sql new file mode 100755 index 0000000000..e7e0b1a052 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q97.sql @@ -0,0 +1,30 @@ +WITH ssci AS ( + SELECT + ss_customer_sk customer_sk, + ss_item_sk item_sk + FROM store_sales, date_dim + WHERE ss_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + GROUP BY ss_customer_sk, ss_item_sk), + csci AS ( + SELECT + cs_bill_customer_sk customer_sk, + cs_item_sk item_sk + FROM catalog_sales, date_dim + WHERE cs_sold_date_sk = d_date_sk + AND d_month_seq BETWEEN 1200 AND 1200 + 11 + GROUP BY cs_bill_customer_sk, cs_item_sk) +SELECT + sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NULL + THEN 1 + ELSE 0 END) store_only, + sum(CASE WHEN ssci.customer_sk IS NULL AND csci.customer_sk IS NOT NULL + THEN 1 + ELSE 0 END) catalog_only, + sum(CASE WHEN ssci.customer_sk IS NOT NULL AND csci.customer_sk IS NOT NULL + THEN 1 + ELSE 0 END) store_and_catalog +FROM ssci + FULL OUTER JOIN csci ON (ssci.customer_sk = csci.customer_sk + AND ssci.item_sk = csci.item_sk) +LIMIT 100 diff --git a/plugin-spark/src/test/resources/tpcds/q98.sql b/plugin-spark/src/test/resources/tpcds/q98.sql new file mode 100755 index 0000000000..bb10d4bf8d --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q98.sql @@ -0,0 +1,21 @@ +SELECT + i_item_desc, + i_category, + i_class, + i_current_price, + sum(ss_ext_sales_price) AS itemrevenue, + sum(ss_ext_sales_price) * 100 / sum(sum(ss_ext_sales_price)) + OVER + (PARTITION BY i_class) AS revenueratio +FROM + store_sales, item, date_dim +WHERE + ss_item_sk = i_item_sk + AND i_category IN ('Sports', 'Books', 'Home') + AND ss_sold_date_sk = d_date_sk + AND d_date BETWEEN cast('1999-02-22' AS DATE) + AND (cast('1999-02-22' AS DATE) + INTERVAL 30 days) +GROUP BY + i_item_id, i_item_desc, i_category, i_class, i_current_price +ORDER BY + i_category, i_class, i_item_id, i_item_desc, revenueratio diff --git a/plugin-spark/src/test/resources/tpcds/q99.sql b/plugin-spark/src/test/resources/tpcds/q99.sql new file mode 100755 index 0000000000..f1a3d4d2b7 --- /dev/null +++ b/plugin-spark/src/test/resources/tpcds/q99.sql @@ -0,0 +1,34 @@ +SELECT + substr(w_warehouse_name, 1, 20), + sm_type, + cc_name, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk <= 30) + THEN 1 + ELSE 0 END) AS `30 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 30) AND + (cs_ship_date_sk - cs_sold_date_sk <= 60) + THEN 1 + ELSE 0 END) AS `31 - 60 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 60) AND + (cs_ship_date_sk - cs_sold_date_sk <= 90) + THEN 1 + ELSE 0 END) AS `61 - 90 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 90) AND + (cs_ship_date_sk - cs_sold_date_sk <= 120) + THEN 1 + ELSE 0 END) AS `91 - 120 days `, + sum(CASE WHEN (cs_ship_date_sk - cs_sold_date_sk > 120) + THEN 1 + ELSE 0 END) AS `>120 days ` +FROM + catalog_sales, warehouse, ship_mode, call_center, date_dim +WHERE + d_month_seq BETWEEN 1200 AND 1200 + 11 + AND cs_ship_date_sk = d_date_sk + AND cs_warehouse_sk = w_warehouse_sk + AND cs_ship_mode_sk = sm_ship_mode_sk + AND cs_call_center_sk = cc_call_center_sk +GROUP BY + substr(w_warehouse_name, 1, 20), sm_type, cc_name +ORDER BY substr(w_warehouse_name, 1, 20), sm_type, cc_name +LIMIT 100 diff --git a/plugin-spark/src/test/scala/org/apache/ranger/services/spark/RangerAdminClientImpl.scala b/plugin-spark/src/test/scala/org/apache/ranger/services/spark/RangerAdminClientImpl.scala new file mode 100644 index 0000000000..d4de34ee43 --- /dev/null +++ b/plugin-spark/src/test/scala/org/apache/ranger/services/spark/RangerAdminClientImpl.scala @@ -0,0 +1,56 @@ +/* + * 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.ranger.services.spark + +import java.io.File +import java.nio.file.{Files, FileSystems} +import java.util + +import com.google.gson.GsonBuilder +import org.apache.commons.logging.{Log, LogFactory} +import org.apache.ranger.admin.client.RangerAdminClient +import org.apache.ranger.plugin.util.{GrantRevokeRequest, ServicePolicies, ServiceTags} + +class RangerAdminClientImpl extends RangerAdminClient { + private val LOG: Log = LogFactory.getLog(classOf[RangerAdminClientImpl]) + private val cacheFilename = "sparkSql_hive_jenkins.json" + private val gson = + new GsonBuilder().setDateFormat("yyyyMMdd-HH:mm:ss.SSS-Z").setPrettyPrinting().create + + override def init(serviceName: String, appId: String, configPropertyPrefix: String): Unit = {} + + override def getServicePoliciesIfUpdated( + lastKnownVersion: Long, + lastActivationTimeInMillis: Long): ServicePolicies = { + val basedir = Option(System.getProperty("baseDir")).getOrElse(new File(".").getCanonicalPath) + val cachePath = FileSystems.getDefault.getPath(basedir, "/src/test/resources/" + cacheFilename) + LOG.info("Reading policies from " + cachePath) + val bytes = Files.readAllBytes(cachePath) + gson.fromJson(new String(bytes), classOf[ServicePolicies]) + } + + override def grantAccess(request: GrantRevokeRequest): Unit = {} + + override def revokeAccess(request: GrantRevokeRequest): Unit = {} + + override def getServiceTagsIfUpdated( + lastKnownVersion: Long, + lastActivationTimeInMillis: Long): ServiceTags = null + + override def getTagTypes(tagTypePattern: String): util.List[String] = null +} diff --git a/plugin-spark/src/test/scala/org/apache/ranger/services/spark/SparkRangerAuthorizerTest.scala b/plugin-spark/src/test/scala/org/apache/ranger/services/spark/SparkRangerAuthorizerTest.scala new file mode 100644 index 0000000000..f0a7bd6dae --- /dev/null +++ b/plugin-spark/src/test/scala/org/apache/ranger/services/spark/SparkRangerAuthorizerTest.scala @@ -0,0 +1,477 @@ +/* + * 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.ranger.services.spark + +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.RangerSparkTestUtils +import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodeGenerator} +import org.apache.spark.sql.catalyst.plans.logical.{Project, RangerSparkMasking, RangerSparkRowFilter} +import org.apache.spark.sql.catalyst.util.resourceToString +import org.apache.spark.sql.execution.{SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.internal.SQLConf +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +class SparkRangerAuthorizerTest extends FunSuite with BeforeAndAfterAll { + + import RangerSparkTestUtils._ + private val spark = TestHive.sparkSession + private lazy val sql = spark.sql _ + + override def beforeAll(): Unit = { + super.beforeAll() + injectRules(spark) + spark.conf.set(SQLConf.CROSS_JOINS_ENABLED.key, "true") + + sql( + """ + |CREATE TABLE `catalog_page` ( + |`cp_catalog_page_sk` INT, `cp_catalog_page_id` STRING, `cp_start_date_sk` INT, + |`cp_end_date_sk` INT, `cp_department` STRING, `cp_catalog_number` INT, + |`cp_catalog_page_number` INT, `cp_description` STRING, `cp_type` STRING) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `catalog_returns` ( + |`cr_returned_date_sk` INT, `cr_returned_time_sk` INT, `cr_item_sk` INT, + |`cr_refunded_customer_sk` INT, `cr_refunded_cdemo_sk` INT, `cr_refunded_hdemo_sk` INT, + |`cr_refunded_addr_sk` INT, `cr_returning_customer_sk` INT, `cr_returning_cdemo_sk` INT, + |`cr_returning_hdemo_sk` INT, `cr_returning_addr_sk` INT, `cr_call_center_sk` INT, + |`cr_catalog_page_sk` INT, `cr_ship_mode_sk` INT, `cr_warehouse_sk` INT, `cr_reason_sk` INT, + |`cr_order_number` INT, `cr_return_quantity` INT, `cr_return_amount` DECIMAL(7,2), + |`cr_return_tax` DECIMAL(7,2), `cr_return_amt_inc_tax` DECIMAL(7,2), `cr_fee` DECIMAL(7,2), + |`cr_return_ship_cost` DECIMAL(7,2), `cr_refunded_cash` DECIMAL(7,2), + |`cr_reversed_charge` DECIMAL(7,2), `cr_store_credit` DECIMAL(7,2), + |`cr_net_loss` DECIMAL(7,2)) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `customer` ( + |`c_customer_sk` INT, `c_customer_id` STRING, `c_current_cdemo_sk` INT, + |`c_current_hdemo_sk` INT, `c_current_addr_sk` INT, `c_first_shipto_date_sk` INT, + |`c_first_sales_date_sk` INT, `c_salutation` STRING, `c_first_name` STRING, + |`c_last_name` STRING, `c_preferred_cust_flag` STRING, `c_birth_day` INT, + |`c_birth_month` INT, `c_birth_year` INT, `c_birth_country` STRING, `c_login` STRING, + |`c_email_address` STRING, `c_last_review_date` STRING) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `customer_address` ( + |`ca_address_sk` INT, `ca_address_id` STRING, `ca_street_number` STRING, + |`ca_street_name` STRING, `ca_street_type` STRING, `ca_suite_number` STRING, + |`ca_city` STRING, `ca_county` STRING, `ca_state` STRING, `ca_zip` STRING, + |`ca_country` STRING, `ca_gmt_offset` DECIMAL(5,2), `ca_location_type` STRING) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `customer_demographics` ( + |`cd_demo_sk` INT, `cd_gender` STRING, `cd_marital_status` STRING, + |`cd_education_status` STRING, `cd_purchase_estimate` INT, `cd_credit_rating` STRING, + |`cd_dep_count` INT, `cd_dep_employed_count` INT, `cd_dep_college_count` INT) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `date_dim` ( + |`d_date_sk` INT, `d_date_id` STRING, `d_date` STRING, + |`d_month_seq` INT, `d_week_seq` INT, `d_quarter_seq` INT, `d_year` INT, `d_dow` INT, + |`d_moy` INT, `d_dom` INT, `d_qoy` INT, `d_fy_year` INT, `d_fy_quarter_seq` INT, + |`d_fy_week_seq` INT, `d_day_name` STRING, `d_quarter_name` STRING, `d_holiday` STRING, + |`d_weekend` STRING, `d_following_holiday` STRING, `d_first_dom` INT, `d_last_dom` INT, + |`d_same_day_ly` INT, `d_same_day_lq` INT, `d_current_day` STRING, `d_current_week` STRING, + |`d_current_month` STRING, `d_current_quarter` STRING, `d_current_year` STRING) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `household_demographics` ( + |`hd_demo_sk` INT, `hd_income_band_sk` INT, `hd_buy_potential` STRING, `hd_dep_count` INT, + |`hd_vehicle_count` INT) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `inventory` (`inv_date_sk` INT, `inv_item_sk` INT, `inv_warehouse_sk` INT, + |`inv_quantity_on_hand` INT) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `item` (`i_item_sk` INT, `i_item_id` STRING, `i_rec_start_date` STRING, + |`i_rec_end_date` STRING, `i_item_desc` STRING, `i_current_price` DECIMAL(7,2), + |`i_wholesale_cost` DECIMAL(7,2), `i_brand_id` INT, `i_brand` STRING, `i_class_id` INT, + |`i_class` STRING, `i_category_id` INT, `i_category` STRING, `i_manufact_id` INT, + |`i_manufact` STRING, `i_size` STRING, `i_formulation` STRING, `i_color` STRING, + |`i_units` STRING, `i_container` STRING, `i_manager_id` INT, `i_product_name` STRING) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `promotion` ( + |`p_promo_sk` INT, `p_promo_id` STRING, `p_start_date_sk` INT, `p_end_date_sk` INT, + |`p_item_sk` INT, `p_cost` DECIMAL(15,2), `p_response_target` INT, `p_promo_name` STRING, + |`p_channel_dmail` STRING, `p_channel_email` STRING, `p_channel_catalog` STRING, + |`p_channel_tv` STRING, `p_channel_radio` STRING, `p_channel_press` STRING, + |`p_channel_event` STRING, `p_channel_demo` STRING, `p_channel_details` STRING, + |`p_purpose` STRING, `p_discount_active` STRING) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `store` ( + |`s_store_sk` INT, `s_store_id` STRING, `s_rec_start_date` STRING, + |`s_rec_end_date` STRING, `s_closed_date_sk` INT, `s_store_name` STRING, + |`s_number_employees` INT, `s_floor_space` INT, `s_hours` STRING, `s_manager` STRING, + |`s_market_id` INT, `s_geography_class` STRING, `s_market_desc` STRING, + |`s_market_manager` STRING, `s_division_id` INT, `s_division_name` STRING, + |`s_company_id` INT, `s_company_name` STRING, `s_street_number` STRING, + |`s_street_name` STRING, `s_street_type` STRING, `s_suite_number` STRING, `s_city` STRING, + |`s_county` STRING, `s_state` STRING, `s_zip` STRING, `s_country` STRING, + |`s_gmt_offset` DECIMAL(5,2), `s_tax_precentage` DECIMAL(5,2)) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `store_returns` ( + |`sr_returned_date_sk` BIGINT, `sr_return_time_sk` BIGINT, `sr_item_sk` BIGINT, + |`sr_customer_sk` BIGINT, `sr_cdemo_sk` BIGINT, `sr_hdemo_sk` BIGINT, `sr_addr_sk` BIGINT, + |`sr_store_sk` BIGINT, `sr_reason_sk` BIGINT, `sr_ticket_number` BIGINT, + |`sr_return_quantity` BIGINT, `sr_return_amt` DECIMAL(7,2), `sr_return_tax` DECIMAL(7,2), + |`sr_return_amt_inc_tax` DECIMAL(7,2), `sr_fee` DECIMAL(7,2), + |`sr_return_ship_cost` DECIMAL(7,2), `sr_refunded_cash` DECIMAL(7,2), + |`sr_reversed_charge` DECIMAL(7,2), `sr_store_credit` DECIMAL(7,2), + |`sr_net_loss` DECIMAL(7,2)) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `catalog_sales` ( + |`cs_sold_date_sk` INT, `cs_sold_time_sk` INT, `cs_ship_date_sk` INT, + |`cs_bill_customer_sk` INT, `cs_bill_cdemo_sk` INT, `cs_bill_hdemo_sk` INT, + |`cs_bill_addr_sk` INT, `cs_ship_customer_sk` INT, `cs_ship_cdemo_sk` INT, + |`cs_ship_hdemo_sk` INT, `cs_ship_addr_sk` INT, `cs_call_center_sk` INT, + |`cs_catalog_page_sk` INT, `cs_ship_mode_sk` INT, `cs_warehouse_sk` INT, + |`cs_item_sk` INT, `cs_promo_sk` INT, `cs_order_number` INT, `cs_quantity` INT, + |`cs_wholesale_cost` DECIMAL(7,2), `cs_list_price` DECIMAL(7,2), + |`cs_sales_price` DECIMAL(7,2), `cs_ext_discount_amt` DECIMAL(7,2), + |`cs_ext_sales_price` DECIMAL(7,2), `cs_ext_wholesale_cost` DECIMAL(7,2), + |`cs_ext_list_price` DECIMAL(7,2), `cs_ext_tax` DECIMAL(7,2), `cs_coupon_amt` DECIMAL(7,2), + |`cs_ext_ship_cost` DECIMAL(7,2), `cs_net_paid` DECIMAL(7,2), + |`cs_net_paid_inc_tax` DECIMAL(7,2), `cs_net_paid_inc_ship` DECIMAL(7,2), + |`cs_net_paid_inc_ship_tax` DECIMAL(7,2), `cs_net_profit` DECIMAL(7,2)) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `web_sales` ( + |`ws_sold_date_sk` INT, `ws_sold_time_sk` INT, `ws_ship_date_sk` INT, `ws_item_sk` INT, + |`ws_bill_customer_sk` INT, `ws_bill_cdemo_sk` INT, `ws_bill_hdemo_sk` INT, + |`ws_bill_addr_sk` INT, `ws_ship_customer_sk` INT, `ws_ship_cdemo_sk` INT, + |`ws_ship_hdemo_sk` INT, `ws_ship_addr_sk` INT, `ws_web_page_sk` INT, `ws_web_site_sk` INT, + |`ws_ship_mode_sk` INT, `ws_warehouse_sk` INT, `ws_promo_sk` INT, `ws_order_number` INT, + |`ws_quantity` INT, `ws_wholesale_cost` DECIMAL(7,2), `ws_list_price` DECIMAL(7,2), + |`ws_sales_price` DECIMAL(7,2), `ws_ext_discount_amt` DECIMAL(7,2), + |`ws_ext_sales_price` DECIMAL(7,2), `ws_ext_wholesale_cost` DECIMAL(7,2), + |`ws_ext_list_price` DECIMAL(7,2), `ws_ext_tax` DECIMAL(7,2), + |`ws_coupon_amt` DECIMAL(7,2), `ws_ext_ship_cost` DECIMAL(7,2), `ws_net_paid` DECIMAL(7,2), + |`ws_net_paid_inc_tax` DECIMAL(7,2), `ws_net_paid_inc_ship` DECIMAL(7,2), + |`ws_net_paid_inc_ship_tax` DECIMAL(7,2), `ws_net_profit` DECIMAL(7,2)) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `store_sales` ( + |`ss_sold_date_sk` INT, `ss_sold_time_sk` INT, `ss_item_sk` INT, `ss_customer_sk` INT, + |`ss_cdemo_sk` INT, `ss_hdemo_sk` INT, `ss_addr_sk` INT, `ss_store_sk` INT, + |`ss_promo_sk` INT, `ss_ticket_number` INT, `ss_quantity` INT, + |`ss_wholesale_cost` DECIMAL(7,2), `ss_list_price` DECIMAL(7,2), + |`ss_sales_price` DECIMAL(7,2), `ss_ext_discount_amt` DECIMAL(7,2), + |`ss_ext_sales_price` DECIMAL(7,2), `ss_ext_wholesale_cost` DECIMAL(7,2), + |`ss_ext_list_price` DECIMAL(7,2), `ss_ext_tax` DECIMAL(7,2), + |`ss_coupon_amt` DECIMAL(7,2), `ss_net_paid` DECIMAL(7,2), + |`ss_net_paid_inc_tax` DECIMAL(7,2), `ss_net_profit` DECIMAL(7,2)) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `web_returns` ( + |`wr_returned_date_sk` BIGINT, `wr_returned_time_sk` BIGINT, `wr_item_sk` BIGINT, + |`wr_refunded_customer_sk` BIGINT, `wr_refunded_cdemo_sk` BIGINT, + |`wr_refunded_hdemo_sk` BIGINT, `wr_refunded_addr_sk` BIGINT, + |`wr_returning_customer_sk` BIGINT, `wr_returning_cdemo_sk` BIGINT, + |`wr_returning_hdemo_sk` BIGINT, `wr_returning_addr_sk` BIGINT, `wr_web_page_sk` BIGINT, + |`wr_reason_sk` BIGINT, `wr_order_number` BIGINT, `wr_return_quantity` BIGINT, + |`wr_return_amt` DECIMAL(7,2), `wr_return_tax` DECIMAL(7,2), + |`wr_return_amt_inc_tax` DECIMAL(7,2), `wr_fee` DECIMAL(7,2), + |`wr_return_ship_cost` DECIMAL(7,2), `wr_refunded_cash` DECIMAL(7,2), + |`wr_reversed_charge` DECIMAL(7,2), `wr_account_credit` DECIMAL(7,2), + |`wr_net_loss` DECIMAL(7,2)) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `web_site` ( + |`web_site_sk` INT, `web_site_id` STRING, `web_rec_start_date` DATE, + |`web_rec_end_date` DATE, `web_name` STRING, `web_open_date_sk` INT, + |`web_close_date_sk` INT, `web_class` STRING, `web_manager` STRING, `web_mkt_id` INT, + |`web_mkt_class` STRING, `web_mkt_desc` STRING, `web_market_manager` STRING, + |`web_company_id` INT, `web_company_name` STRING, `web_street_number` STRING, + |`web_street_name` STRING, `web_street_type` STRING, `web_suite_number` STRING, + |`web_city` STRING, `web_county` STRING, `web_state` STRING, `web_zip` STRING, + |`web_country` STRING, `web_gmt_offset` STRING, `web_tax_percentage` DECIMAL(5,2)) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `reason` ( + |`r_reason_sk` INT, `r_reason_id` STRING, `r_reason_desc` STRING) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `call_center` ( + |`cc_call_center_sk` INT, `cc_call_center_id` STRING, `cc_rec_start_date` DATE, + |`cc_rec_end_date` DATE, `cc_closed_date_sk` INT, `cc_open_date_sk` INT, `cc_name` STRING, + |`cc_class` STRING, `cc_employees` INT, `cc_sq_ft` INT, `cc_hours` STRING, + |`cc_manager` STRING, `cc_mkt_id` INT, `cc_mkt_class` STRING, `cc_mkt_desc` STRING, + |`cc_market_manager` STRING, `cc_division` INT, `cc_division_name` STRING, `cc_company` INT, + |`cc_company_name` STRING, `cc_street_number` STRING, `cc_street_name` STRING, + |`cc_street_type` STRING, `cc_suite_number` STRING, `cc_city` STRING, `cc_county` STRING, + |`cc_state` STRING, `cc_zip` STRING, `cc_country` STRING, `cc_gmt_offset` DECIMAL(5,2), + |`cc_tax_percentage` DECIMAL(5,2)) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `warehouse` ( + |`w_warehouse_sk` INT, `w_warehouse_id` STRING, `w_warehouse_name` STRING, + |`w_warehouse_sq_ft` INT, `w_street_number` STRING, `w_street_name` STRING, + |`w_street_type` STRING, `w_suite_number` STRING, `w_city` STRING, `w_county` STRING, + |`w_state` STRING, `w_zip` STRING, `w_country` STRING, `w_gmt_offset` DECIMAL(5,2)) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `ship_mode` ( + |`sm_ship_mode_sk` INT, `sm_ship_mode_id` STRING, `sm_type` STRING, `sm_code` STRING, + |`sm_carrier` STRING, `sm_contract` STRING) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `income_band` ( + |`ib_income_band_sk` INT, `ib_lower_bound` INT, `ib_upper_bound` INT) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `time_dim` ( + |`t_time_sk` INT, `t_time_id` STRING, `t_time` INT, `t_hour` INT, `t_minute` INT, + |`t_second` INT, `t_am_pm` STRING, `t_shift` STRING, `t_sub_shift` STRING, + |`t_meal_time` STRING) + |USING parquet + """.stripMargin) + + sql( + """ + |CREATE TABLE `web_page` (`wp_web_page_sk` INT, `wp_web_page_id` STRING, + |`wp_rec_start_date` DATE, `wp_rec_end_date` DATE, `wp_creation_date_sk` INT, + |`wp_access_date_sk` INT, `wp_autogen_flag` STRING, `wp_customer_sk` INT, + |`wp_url` STRING, `wp_type` STRING, `wp_char_count` INT, `wp_link_count` INT, + |`wp_image_count` INT, `wp_max_ad_count` INT) + |USING parquet + """.stripMargin) + } + + test("simple query") { + val statement = "select * from default.src" + withUser("bob") { + val df = sql(statement) + assert(df.queryExecution.optimizedPlan.find(_.isInstanceOf[RangerSparkMasking]).nonEmpty) + assert(df.queryExecution.optimizedPlan.find(_.isInstanceOf[RangerSparkRowFilter]).nonEmpty) + assert(df.queryExecution.optimizedPlan.isInstanceOf[Project]) + val project = df.queryExecution.optimizedPlan.asInstanceOf[Project] + val masker = project.projectList(1) + assert(masker.name === "value") + assert(masker.children.exists(_.sql.contains("mask_show_last_n"))) + val row = df.take(1)(0) + assert(row.getInt(0) < 20, "keys above 20 should be filtered automatically") + assert(row.getString(1).startsWith("x"), "values should be masked") + assert(df.count() === 20, "keys above 20 should be filtered automatically") + } + withUser("alice") { + val df = spark.sql(statement) + assert(df.count() === 500) + } + } + + test("projection with ranger filter key") { + val statement = "select key from default.src" + withUser("bob") { + val df = spark.sql(statement) + val row = df.take(1)(0) + assert(row.getInt(0) < 20) + } + withUser("alice") { + val df = spark.sql(statement) + assert(df.count() === 500) + } + } + + test("projection without ranger filter key") { + val statement = "select value from default.src" + withUser("bob") { + val df = spark.sql(statement) + val row = df.take(1)(0) + assert(row.getString(0).split("_")(1).toInt < 20) + } + withUser("alice") { + val df = spark.sql(statement) + assert(df.count() === 500) + } + } + + test("filter with with ranger filter key") { + val statement = "select key from default.src where key = 0" + val statement2 = "select key from default.src where key >= 20" + withUser("bob") { + val df = spark.sql(statement) + val row = df.take(1)(0) + assert(row.getInt(0) === 0) + val df2 = spark.sql(statement2) + assert(df2.count() === 0, "all keys should be filtered") + } + withUser("alice") { + val df = spark.sql(statement) + assert(df.count() === 3) + val df2 = spark.sql(statement2) + assert(df2.count() === 480) + } + } + + test("alias") { + val statement = "select key as k1, value v1 from default.src" + withUser("bob") { + val df = spark.sql(statement) + val row = df.take(1)(0) + assert(row.getInt(0) < 20, "keys above 20 should be filtered automatically") + assert(row.getString(1).startsWith("x"), "values should be masked") + assert(df.count() === 20, "keys above 20 should be filtered automatically") + } + withUser("alice") { + val df = spark.sql(statement) + assert(df.count() === 500) + } + } + + test("agg") { + val statement = "select sum(key) as k1, value v1 from default.src group by v1" + withUser("bob") { + val df = spark.sql(statement) + println(df.queryExecution.optimizedPlan) + val row = df.take(1)(0) + assert(row.getString(1).startsWith("x"), "values should be masked") + assert(row.getString(1).split("_")(1).toInt < 20) + } + withUser("alice") { + val df = spark.sql(statement) + val row = df.take(1)(0) + assert(row.getString(1).startsWith("val"), "values should not be masked") + } + } + + private val tpcdsQueries = Seq( + "q1", "q2", "q3", "q4", "q5", "q6", "q7", "q8", "q9", "q10", "q11", + "q12", "q13", "q14a", "q14b", "q15", "q16", "q17", "q18", "q19", "q20", + "q21", "q22", "q23a", "q23b", "q24a", "q24b", "q25", "q26", "q27", "q28", "q29", "q30", + "q31", "q32", "q33", "q34", "q35", "q36", "q37", "q38", "q39a", "q39b", "q40", + "q41", "q42", "q43", "q44", "q45", "q46", "q47", "q48", "q49", "q50", + "q51", "q52", "q53", "q54", "q55", "q56", "q57", "q58", "q59", "q60", + "q61", "q62", "q63", "q64", "q65", "q66", "q67", "q68", "q69", "q70", + "q71", "q72", "q73", "q74", "q75", "q76", "q77", "q78", "q79", "q80", + "q81", "q82", "q83", "q84", "q85", "q86", "q87", "q88", "q89", "q90", + "q91", "q92", "q93", "q94", "q95", "q96", "q97", "q98", "q99") + + tpcdsQueries.foreach { name => + val queryString = resourceToString(s"tpcds/$name.sql", + classLoader = Thread.currentThread().getContextClassLoader) + test(name) { + withUser("bob") { + val queryExecution = sql(queryString).queryExecution + val optimized = queryExecution.optimizedPlan + assert(optimized.find(_.isInstanceOf[RangerSparkRowFilter]).nonEmpty) + assert(optimized.find(_.isInstanceOf[RangerSparkMasking]).nonEmpty) + val plan = queryExecution.executedPlan + checkGeneratedCode(plan) + } + } + } + + /** + * Check whether the Modified queries can be properly compiled + * @param plan + */ + def checkGeneratedCode(plan: SparkPlan): Unit = { + val codegenSubtrees = new collection.mutable.HashSet[WholeStageCodegenExec]() + plan foreach { + case s: WholeStageCodegenExec => codegenSubtrees += s + } + codegenSubtrees.toSeq.foreach { subtree => + val code = subtree.doCodeGen()._2 + try { + CodeGenerator.compile(code) + } catch { + case e: Exception => + val msg = + s""" + |failed to compile: + |Subtree: + |$subtree + |Generated code: + |${CodeFormatter.format(code)} + """.stripMargin + throw new Exception(msg, e) + } + } + } +} diff --git a/plugin-spark/src/test/scala/org/apache/spark/sql/RangerSparkTestUtils.scala b/plugin-spark/src/test/scala/org/apache/spark/sql/RangerSparkTestUtils.scala new file mode 100644 index 0000000000..a723b4dce0 --- /dev/null +++ b/plugin-spark/src/test/scala/org/apache/spark/sql/RangerSparkTestUtils.scala @@ -0,0 +1,40 @@ +/* + * 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 + +import java.security.PrivilegedExceptionAction + +import org.apache.hadoop.security.UserGroupInformation +import org.apache.spark.sql.catalyst.optimizer.{RangerSparkMaskingExtension, RangerSparkRowFilterExtension} +import org.apache.spark.sql.execution.RangerSparkPlanOmitStrategy + +object RangerSparkTestUtils { + + def injectRules(spark: SparkSession): Unit = { + spark.extensions.injectOptimizerRule(RangerSparkRowFilterExtension) + spark.extensions.injectOptimizerRule(RangerSparkMaskingExtension) + spark.extensions.injectPlannerStrategy(RangerSparkPlanOmitStrategy) + } + + def withUser[T](user: String)(f: => T): T = { + val ugi = UserGroupInformation.createRemoteUser(user) + ugi.doAs(new PrivilegedExceptionAction[T] { + override def run(): T = f + }) + } +} diff --git a/plugin-spark/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtensionTest.scala b/plugin-spark/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtensionTest.scala new file mode 100644 index 0000000000..176948f5a5 --- /dev/null +++ b/plugin-spark/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtensionTest.scala @@ -0,0 +1,49 @@ +/* + * 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.catalyst.optimizer + +import org.apache.spark.sql.hive.test.TestHive +import org.scalatest.FunSuite +import org.apache.spark.sql.RangerSparkTestUtils._ +import org.apache.spark.sql.catalyst.plans.logical.{Filter, RangerSparkRowFilter} + +class RangerSparkRowFilterExtensionTest extends FunSuite { + + private val spark = TestHive.sparkSession + + test("ranger spark row filter extension") { + val extension = RangerSparkRowFilterExtension(spark) + val plan = spark.sql("select * from src").queryExecution.optimizedPlan + println(plan) + withUser("bob") { + val newPlan = extension.apply(plan) + assert(newPlan.isInstanceOf[RangerSparkRowFilter]) + val filters = newPlan.collect { case f: Filter => f } + assert(filters.nonEmpty, "ranger row level filters should be applied automatically") + println(newPlan) + } + withUser("alice") { + val newPlan = extension.apply(plan) + assert(newPlan.isInstanceOf[RangerSparkRowFilter]) + val filters = newPlan.collect { case f: Filter => f } + assert(filters.isEmpty, "alice does not have implicit filters") + println(newPlan) + } + } + +} diff --git a/plugin-spark/src/test/scala/org/apache/spark/sql/execution/RangerSparkPlanOmitStrategyTest.scala b/plugin-spark/src/test/scala/org/apache/spark/sql/execution/RangerSparkPlanOmitStrategyTest.scala new file mode 100644 index 0000000000..dea926accd --- /dev/null +++ b/plugin-spark/src/test/scala/org/apache/spark/sql/execution/RangerSparkPlanOmitStrategyTest.scala @@ -0,0 +1,44 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.plans.logical.{RangerSparkMasking, RangerSparkRowFilter} +import org.apache.spark.sql.hive.test.TestHive +import org.scalatest.FunSuite + +class RangerSparkPlanOmitStrategyTest extends FunSuite { + + private val spark = TestHive.sparkSession + + test("ranger spark plan omit strategy") { + val strategy = RangerSparkPlanOmitStrategy(spark) + val df = spark.range(0, 5) + val plan1 = df.queryExecution.optimizedPlan + assert(strategy.apply(plan1) === Nil) + val plan2 = RangerSparkRowFilter(plan1) + assert(strategy.apply(plan2) === PlanLater(plan1) :: Nil) + val plan3 = RangerSparkMasking(plan1) + assert(strategy.apply(plan3) === PlanLater(plan1) :: Nil) + val plan4 = RangerSparkMasking(plan2) + assert(strategy.apply(plan4) === PlanLater(plan2) :: Nil) + val plan5 = RangerSparkRowFilter(plan3) + assert(strategy.apply(plan5) === PlanLater(plan3) :: Nil) + } + + +} From d84050838616a14b0e66343f64ab0d2deaa6e410 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sun, 28 Apr 2019 23:11:11 +0800 Subject: [PATCH 30/35] fix ut --- .../ranger/services/spark/SparkRangerAuthorizerTest.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/plugin-spark/src/test/scala/org/apache/ranger/services/spark/SparkRangerAuthorizerTest.scala b/plugin-spark/src/test/scala/org/apache/ranger/services/spark/SparkRangerAuthorizerTest.scala index f0a7bd6dae..c928745fce 100644 --- a/plugin-spark/src/test/scala/org/apache/ranger/services/spark/SparkRangerAuthorizerTest.scala +++ b/plugin-spark/src/test/scala/org/apache/ranger/services/spark/SparkRangerAuthorizerTest.scala @@ -439,6 +439,7 @@ class SparkRangerAuthorizerTest extends FunSuite with BeforeAndAfterAll { withUser("bob") { val queryExecution = sql(queryString).queryExecution val optimized = queryExecution.optimizedPlan + // println(optimized) assert(optimized.find(_.isInstanceOf[RangerSparkRowFilter]).nonEmpty) assert(optimized.find(_.isInstanceOf[RangerSparkMasking]).nonEmpty) val plan = queryExecution.executedPlan @@ -449,16 +450,18 @@ class SparkRangerAuthorizerTest extends FunSuite with BeforeAndAfterAll { /** * Check whether the Modified queries can be properly compiled - * @param plan */ def checkGeneratedCode(plan: SparkPlan): Unit = { val codegenSubtrees = new collection.mutable.HashSet[WholeStageCodegenExec]() plan foreach { - case s: WholeStageCodegenExec => codegenSubtrees += s + case s: WholeStageCodegenExec => + codegenSubtrees += s + case s => s } codegenSubtrees.toSeq.foreach { subtree => val code = subtree.doCodeGen()._2 try { + // Just check the generated code can be properly compiled CodeGenerator.compile(code) } catch { case e: Exception => From f60aa0879275d34011f7edff71933a592af9588f Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 29 Apr 2019 19:46:32 +0800 Subject: [PATCH 31/35] support debugging in IDE --- .gitignore | 1 + agents-audit/pom.xml | 5 +++++ agents-common/pom.xml | 3 +++ agents-cred/pom.xml | 5 +++++ agents-installer/pom.xml | 5 +++++ credentialbuilder/pom.xml | 5 +++++ embeddedwebserver/pom.xml | 5 +++++ hbase-agent/pom.xml | 2 ++ hive-agent/pom.xml | 2 ++ jisql/pom.xml | 5 +++++ kms/pom.xml | 2 ++ knox-agent/pom.xml | 2 ++ plugin-atlas/pom.xml | 5 +++++ plugin-elasticsearch/pom.xml | 5 +++++ plugin-kafka/pom.xml | 2 ++ plugin-kms/pom.xml | 2 ++ plugin-kylin/pom.xml | 4 ++++ plugin-nifi-registry/pom.xml | 4 ++++ plugin-nifi/pom.xml | 5 +++++ plugin-solr/pom.xml | 5 +++++ .../apache/ranger/services/spark/RangerAdminClientImpl.scala | 4 ++-- plugin-sqoop/pom.xml | 5 +++++ plugin-yarn/pom.xml | 5 +++++ ranger-atlas-plugin-shim/pom.xml | 5 +++++ ranger-elasticsearch-plugin-shim/pom.xml | 5 +++++ ranger-examples/pom.xml | 2 ++ ranger-examples/sampleapp/pom.xml | 5 +++++ ranger-hbase-plugin-shim/pom.xml | 5 +++++ ranger-hdfs-plugin-shim/pom.xml | 5 +++++ ranger-hive-plugin-shim/pom.xml | 5 +++++ ranger-kafka-plugin-shim/pom.xml | 5 +++++ ranger-kms-plugin-shim/pom.xml | 5 +++++ ranger-knox-plugin-shim/pom.xml | 5 +++++ ranger-kylin-plugin-shim/pom.xml | 5 +++++ ranger-plugin-classloader/pom.xml | 5 +++++ ranger-solr-plugin-shim/pom.xml | 5 +++++ ranger-sqoop-plugin-shim/pom.xml | 5 +++++ ugsync/pom.xml | 2 ++ unixauthclient/pom.xml | 5 +++++ unixauthpam/pom.xml | 2 ++ 40 files changed, 162 insertions(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index c2def9d8a1..bc3924242b 100644 --- a/.gitignore +++ b/.gitignore @@ -9,3 +9,4 @@ winpkg/target .DS_Store .idea +/derby.log diff --git a/agents-audit/pom.xml b/agents-audit/pom.xml index 4aaec0996a..86f1056d0a 100644 --- a/agents-audit/pom.xml +++ b/agents-audit/pom.xml @@ -104,4 +104,9 @@ + + + target/classes + target/test-classes + diff --git a/agents-common/pom.xml b/agents-common/pom.xml index 4486af3598..9964c5fc10 100644 --- a/agents-common/pom.xml +++ b/agents-common/pom.xml @@ -21,6 +21,8 @@ Common library for Plugins Plugins Common + target/classes + target/test-classes org.apache.maven.plugins @@ -86,4 +88,5 @@ mysql-connector-java + diff --git a/agents-cred/pom.xml b/agents-cred/pom.xml index c62268329a..9542d7d93e 100644 --- a/agents-cred/pom.xml +++ b/agents-cred/pom.xml @@ -51,4 +51,9 @@ junit + + + target/classes + target/test-classes + diff --git a/agents-installer/pom.xml b/agents-installer/pom.xml index 20902cb62d..776962547c 100644 --- a/agents-installer/pom.xml +++ b/agents-installer/pom.xml @@ -34,4 +34,9 @@ ${commons.cli.version} + + + target/classes + target/test-classes + diff --git a/credentialbuilder/pom.xml b/credentialbuilder/pom.xml index 461dcd0b67..71ce3f62cb 100644 --- a/credentialbuilder/pom.xml +++ b/credentialbuilder/pom.xml @@ -78,4 +78,9 @@ ${hadoop.version} + + + target/classes + target/test-classes + diff --git a/embeddedwebserver/pom.xml b/embeddedwebserver/pom.xml index a8c7cba751..6ef58766fc 100644 --- a/embeddedwebserver/pom.xml +++ b/embeddedwebserver/pom.xml @@ -84,4 +84,9 @@ ${project.version} + + + target/classes + target/test-classes + diff --git a/hbase-agent/pom.xml b/hbase-agent/pom.xml index c044ef0c30..d4af2a1343 100644 --- a/hbase-agent/pom.xml +++ b/hbase-agent/pom.xml @@ -95,6 +95,8 @@ + target/classes + target/test-classes src/test/resources diff --git a/hive-agent/pom.xml b/hive-agent/pom.xml index 0a091b202c..23a55d7195 100644 --- a/hive-agent/pom.xml +++ b/hive-agent/pom.xml @@ -84,6 +84,8 @@ + target/classes + target/test-classes src/test/resources diff --git a/jisql/pom.xml b/jisql/pom.xml index 07fb08b360..81695af3d2 100644 --- a/jisql/pom.xml +++ b/jisql/pom.xml @@ -34,4 +34,9 @@ 3.2 + + + target/classes + target/test-classes + diff --git a/kms/pom.xml b/kms/pom.xml index 043c67fde4..49e4e6cc85 100644 --- a/kms/pom.xml +++ b/kms/pom.xml @@ -263,6 +263,8 @@ + target/classes + target/test-classes diff --git a/knox-agent/pom.xml b/knox-agent/pom.xml index 0c45c84baa..193b385673 100644 --- a/knox-agent/pom.xml +++ b/knox-agent/pom.xml @@ -155,6 +155,8 @@ + target/classes + target/test-classes ${basedir}/src/main/java ${basedir}/src/test/java diff --git a/plugin-atlas/pom.xml b/plugin-atlas/pom.xml index 87ff40d149..868844ba46 100644 --- a/plugin-atlas/pom.xml +++ b/plugin-atlas/pom.xml @@ -85,6 +85,11 @@ ${google.guava.version} + + + target/classes + target/test-classes + diff --git a/plugin-elasticsearch/pom.xml b/plugin-elasticsearch/pom.xml index b9973105c9..acefa8462d 100644 --- a/plugin-elasticsearch/pom.xml +++ b/plugin-elasticsearch/pom.xml @@ -66,4 +66,9 @@ ${httpcomponents.httpcore.version} + + + target/classes + target/test-classes + diff --git a/plugin-kafka/pom.xml b/plugin-kafka/pom.xml index 132baf7679..001d336390 100644 --- a/plugin-kafka/pom.xml +++ b/plugin-kafka/pom.xml @@ -92,6 +92,8 @@ + target/classes + target/test-classes src/test/resources diff --git a/plugin-kms/pom.xml b/plugin-kms/pom.xml index 1940f41691..546db6d212 100644 --- a/plugin-kms/pom.xml +++ b/plugin-kms/pom.xml @@ -64,6 +64,8 @@ + target/classes + target/test-classes src/test/resources diff --git a/plugin-kylin/pom.xml b/plugin-kylin/pom.xml index 69badfc7ac..298966c310 100644 --- a/plugin-kylin/pom.xml +++ b/plugin-kylin/pom.xml @@ -88,4 +88,8 @@ ${httpcomponents.httpcore.version} + + target/classes + target/test-classes + \ No newline at end of file diff --git a/plugin-nifi-registry/pom.xml b/plugin-nifi-registry/pom.xml index 3e99f04a4c..095a744268 100644 --- a/plugin-nifi-registry/pom.xml +++ b/plugin-nifi-registry/pom.xml @@ -62,4 +62,8 @@ + + target/classes + target/test-classes + diff --git a/plugin-nifi/pom.xml b/plugin-nifi/pom.xml index c510d87fe9..3380b1ac60 100644 --- a/plugin-nifi/pom.xml +++ b/plugin-nifi/pom.xml @@ -59,4 +59,9 @@ test + + + target/classes + target/test-classes + diff --git a/plugin-solr/pom.xml b/plugin-solr/pom.xml index 619cc9ebdb..a9e76d51a4 100644 --- a/plugin-solr/pom.xml +++ b/plugin-solr/pom.xml @@ -52,4 +52,9 @@ ${solr.version} + + + target/classes + target/test-classes + diff --git a/plugin-spark/src/test/scala/org/apache/ranger/services/spark/RangerAdminClientImpl.scala b/plugin-spark/src/test/scala/org/apache/ranger/services/spark/RangerAdminClientImpl.scala index d4de34ee43..ced7072f7c 100644 --- a/plugin-spark/src/test/scala/org/apache/ranger/services/spark/RangerAdminClientImpl.scala +++ b/plugin-spark/src/test/scala/org/apache/ranger/services/spark/RangerAdminClientImpl.scala @@ -37,8 +37,8 @@ class RangerAdminClientImpl extends RangerAdminClient { override def getServicePoliciesIfUpdated( lastKnownVersion: Long, lastActivationTimeInMillis: Long): ServicePolicies = { - val basedir = Option(System.getProperty("baseDir")).getOrElse(new File(".").getCanonicalPath) - val cachePath = FileSystems.getDefault.getPath(basedir, "/src/test/resources/" + cacheFilename) + val basedir = this.getClass.getProtectionDomain.getCodeSource.getLocation.getPath + val cachePath = FileSystems.getDefault.getPath(basedir, cacheFilename) LOG.info("Reading policies from " + cachePath) val bytes = Files.readAllBytes(cachePath) gson.fromJson(new String(bytes), classOf[ServicePolicies]) diff --git a/plugin-sqoop/pom.xml b/plugin-sqoop/pom.xml index af00e97782..06ff981652 100644 --- a/plugin-sqoop/pom.xml +++ b/plugin-sqoop/pom.xml @@ -73,4 +73,9 @@ ${httpcomponents.httpcore.version} + + + target/classes + target/test-classes + diff --git a/plugin-yarn/pom.xml b/plugin-yarn/pom.xml index f2735699af..445762b0e8 100644 --- a/plugin-yarn/pom.xml +++ b/plugin-yarn/pom.xml @@ -62,4 +62,9 @@ ${httpcomponents.httpcore.version} + + + target/classes + target/test-classes + diff --git a/ranger-atlas-plugin-shim/pom.xml b/ranger-atlas-plugin-shim/pom.xml index 86d6a264b1..380cc3f595 100644 --- a/ranger-atlas-plugin-shim/pom.xml +++ b/ranger-atlas-plugin-shim/pom.xml @@ -90,4 +90,9 @@ ${slf4j-api.version} + + + target/classes + target/test-classes + diff --git a/ranger-elasticsearch-plugin-shim/pom.xml b/ranger-elasticsearch-plugin-shim/pom.xml index 59a653b1cc..2bcc57047e 100644 --- a/ranger-elasticsearch-plugin-shim/pom.xml +++ b/ranger-elasticsearch-plugin-shim/pom.xml @@ -63,4 +63,9 @@ runtime + + + target/classes + target/test-classes + \ No newline at end of file diff --git a/ranger-examples/pom.xml b/ranger-examples/pom.xml index fa4cb2024c..fc0156e3e8 100644 --- a/ranger-examples/pom.xml +++ b/ranger-examples/pom.xml @@ -30,6 +30,8 @@ plugin-sampleapp + target/classes + target/test-classes diff --git a/ranger-examples/sampleapp/pom.xml b/ranger-examples/sampleapp/pom.xml index add9546115..ce4e404dfe 100644 --- a/ranger-examples/sampleapp/pom.xml +++ b/ranger-examples/sampleapp/pom.xml @@ -38,4 +38,9 @@ ${log4j.version} + + + target/classes + target/test-classes + diff --git a/ranger-hbase-plugin-shim/pom.xml b/ranger-hbase-plugin-shim/pom.xml index 9ed0aebe8e..150a870d82 100644 --- a/ranger-hbase-plugin-shim/pom.xml +++ b/ranger-hbase-plugin-shim/pom.xml @@ -56,4 +56,9 @@ gson + + + target/classes + target/test-classes + diff --git a/ranger-hdfs-plugin-shim/pom.xml b/ranger-hdfs-plugin-shim/pom.xml index f45621c5b8..75bd83f279 100644 --- a/ranger-hdfs-plugin-shim/pom.xml +++ b/ranger-hdfs-plugin-shim/pom.xml @@ -62,4 +62,9 @@ ${project.version} + + + target/classes + target/test-classes + diff --git a/ranger-hive-plugin-shim/pom.xml b/ranger-hive-plugin-shim/pom.xml index 26258d7bf2..9368ba7664 100644 --- a/ranger-hive-plugin-shim/pom.xml +++ b/ranger-hive-plugin-shim/pom.xml @@ -83,4 +83,9 @@ ${project.version} + + + target/classes + target/test-classes + diff --git a/ranger-kafka-plugin-shim/pom.xml b/ranger-kafka-plugin-shim/pom.xml index 5a7353f02d..1e2f3ba101 100644 --- a/ranger-kafka-plugin-shim/pom.xml +++ b/ranger-kafka-plugin-shim/pom.xml @@ -58,4 +58,9 @@ ${kafka.version} + + + target/classes + target/test-classes + diff --git a/ranger-kms-plugin-shim/pom.xml b/ranger-kms-plugin-shim/pom.xml index 0b505f6afa..d5e64ccbf6 100644 --- a/ranger-kms-plugin-shim/pom.xml +++ b/ranger-kms-plugin-shim/pom.xml @@ -57,4 +57,9 @@ ${project.version} + + + target/classes + target/test-classes + diff --git a/ranger-knox-plugin-shim/pom.xml b/ranger-knox-plugin-shim/pom.xml index db342b71df..03289916f7 100644 --- a/ranger-knox-plugin-shim/pom.xml +++ b/ranger-knox-plugin-shim/pom.xml @@ -79,4 +79,9 @@ ${project.version} + + + target/classes + target/test-classes + diff --git a/ranger-kylin-plugin-shim/pom.xml b/ranger-kylin-plugin-shim/pom.xml index 4bb9f34edb..d8a420a646 100644 --- a/ranger-kylin-plugin-shim/pom.xml +++ b/ranger-kylin-plugin-shim/pom.xml @@ -69,4 +69,9 @@ ${project.version} + + + target/classes + target/test-classes + \ No newline at end of file diff --git a/ranger-plugin-classloader/pom.xml b/ranger-plugin-classloader/pom.xml index 0aa71c36e9..517f89596a 100644 --- a/ranger-plugin-classloader/pom.xml +++ b/ranger-plugin-classloader/pom.xml @@ -48,4 +48,9 @@ ${slf4j-api.version} + + + target/classes + target/test-classes + diff --git a/ranger-solr-plugin-shim/pom.xml b/ranger-solr-plugin-shim/pom.xml index 46b78218a8..34ad042298 100644 --- a/ranger-solr-plugin-shim/pom.xml +++ b/ranger-solr-plugin-shim/pom.xml @@ -57,4 +57,9 @@ ${project.version} + + + target/classes + target/test-classes + diff --git a/ranger-sqoop-plugin-shim/pom.xml b/ranger-sqoop-plugin-shim/pom.xml index 86ab432332..ab12bc7461 100644 --- a/ranger-sqoop-plugin-shim/pom.xml +++ b/ranger-sqoop-plugin-shim/pom.xml @@ -59,4 +59,9 @@ ${project.version} + + + target/classes + target/test-classes + \ No newline at end of file diff --git a/ugsync/pom.xml b/ugsync/pom.xml index c30d8ca7c6..ca4459d5fe 100644 --- a/ugsync/pom.xml +++ b/ugsync/pom.xml @@ -137,6 +137,8 @@ + target/classes + target/test-classes src/test/resources diff --git a/unixauthclient/pom.xml b/unixauthclient/pom.xml index 4625925a2a..27211e2b5b 100644 --- a/unixauthclient/pom.xml +++ b/unixauthclient/pom.xml @@ -89,4 +89,9 @@ + + + target/classes + target/test-classes + diff --git a/unixauthpam/pom.xml b/unixauthpam/pom.xml index 0f2a4596f2..251bfe08b2 100644 --- a/unixauthpam/pom.xml +++ b/unixauthpam/pom.xml @@ -28,6 +28,8 @@ PAM Authenticator PAM authentication service + target/classes + target/test-classes org.codehaus.mojo From 688af61787b850bf9caaf0eae8b0ee1a8d8a890d Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 29 Apr 2019 23:27:02 +0800 Subject: [PATCH 32/35] add more uts --- .../test/resources/sparkSql_hive_jenkins.json | 615 +++++++++++++++++- .../spark/RangerAdminClientImpl.scala | 1 - .../spark/SparkRangerAuthorizerTest.scala | 120 +++- 3 files changed, 714 insertions(+), 22 deletions(-) diff --git a/plugin-spark/src/test/resources/sparkSql_hive_jenkins.json b/plugin-spark/src/test/resources/sparkSql_hive_jenkins.json index 42a9a03364..6d5e6005c4 100644 --- a/plugin-spark/src/test/resources/sparkSql_hive_jenkins.json +++ b/plugin-spark/src/test/resources/sparkSql_hive_jenkins.json @@ -1,8 +1,8 @@ { "serviceName": "hive_jenkins", "serviceId": 1, - "policyVersion": 73, - "policyUpdateTime": "20190424-16:19:30.000-+0800", + "policyVersion": 85, + "policyUpdateTime": "20190429-21:36:09.000-+0800", "policies": [ { "service": "hive_jenkins", @@ -270,7 +270,7 @@ }, { "service": "hive_jenkins", - "name": "test1", + "name": "src_key _less_than_20", "policyType": 2, "policyPriority": 0, "description": "", @@ -324,7 +324,7 @@ "id": 4, "guid": "f588a9ed-f7b1-48f7-9d0d-c12cf2b9b7ed", "isEnabled": true, - "version": 25 + "version": 26 }, { "service": "hive_jenkins", @@ -427,7 +427,7 @@ }, { "service": "hive_jenkins", - "name": "src", + "name": "src_val_show_last_4", "policyType": 1, "policyPriority": 0, "description": "", @@ -462,7 +462,8 @@ "dataMaskPolicyItems": [ { "dataMaskInfo": { - "dataMaskType": "MASK_SHOW_LAST_4" + "dataMaskType": "MASK_SHOW_LAST_4", + "valueExpr": "" }, "accesses": [ { @@ -487,7 +488,7 @@ "id": 6, "guid": "b1261fcc-b2cd-49f2-85e8-93f254f987ec", "isEnabled": true, - "version": 9 + "version": 10 }, { "service": "hive_jenkins", @@ -1434,10 +1435,608 @@ "guid": "b7847238-3a14-4d56-8257-b8625a7f25a1", "isEnabled": true, "version": 1 + }, + { + "service": "hive_jenkins", + "name": "rangertbl1_key_equals_0", + "policyType": 2, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "rangertbl1" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "key\u003d0" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 23, + "guid": "d52bc8de-2a6b-4f7c-ab26-fbaf22c05eb7", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "rangertbl2_key_in_set", + "policyType": 2, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "rangertbl2" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "key in (0, 1, 2)" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 24, + "guid": "06008a40-9b33-4699-8782-cc7e85101b85", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "rangertbl3_key_in_subquery", + "policyType": 2, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "rangertbl3" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "key in (select key from rangertbl2 where key \u003c 100)" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 25, + "guid": "d0ca382a-1d62-4faa-8b9b-aeb36d4e443e", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "rangertbl4_key_in_self", + "policyType": 2, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "rangertbl4" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "key in (select key from rangertbl4)" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 26, + "guid": "b2b730af-d106-41f2-a21e-c29626adf6f3", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "rangertbl5_key_udf", + "policyType": 2, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "rangertbl5" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "current_date()\u003d\"2019-04-28\"" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 27, + "guid": "0540df7e-fa14-4a41-b7d2-479fb42ddf5f", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "rangertbl6_key_and_or", + "policyType": 2, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "rangertbl6" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [ + { + "rowFilterInfo": { + "filterExpr": "key\u003e1 and key\u003c10 or key \u003d500" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 28, + "guid": "5805bb62-291e-44b1-81e2-9f5c5b2b3cca", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "rangertbl1_value_redact", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "value" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "rangertbl1" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 29, + "guid": "9e7a290a-3d24-4f19-a4c6-2cf0637204ab", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "rangertbl2_value_sf4", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "value" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "rangertbl2" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_SHOW_FIRST_4" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 30, + "guid": "9d50a525-b24c-4cf5-a885-d10d426368d1", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "rangertbl3_value_hash", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "value" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "rangertbl3" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_HASH" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 31, + "guid": "ed1868a1-bf79-4721-a3d5-6815cc7d4986", + "isEnabled": true, + "version": 1 + }, + { + "service": "hive_jenkins", + "name": "rangertbl4_value_nullify", + "policyType": 1, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "value" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "rangertbl4" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [ + { + "dataMaskInfo": { + "dataMaskType": "MASK_NULL" + }, + "accesses": [ + { + "type": "select", + "isAllowed": true + } + ], + "users": [ + "bob" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 32, + "guid": "98a04cd7-8d14-4466-adc9-126d87a3af69", + "isEnabled": true, + "version": 1 } ], "serviceDef": { - "name": "bob", + "name": "hive", "implClass": "org.apache.ranger.services.hive.RangerServiceHive", "label": "Hive Server2", "description": "Hive Server2", diff --git a/plugin-spark/src/test/scala/org/apache/ranger/services/spark/RangerAdminClientImpl.scala b/plugin-spark/src/test/scala/org/apache/ranger/services/spark/RangerAdminClientImpl.scala index ced7072f7c..2c96876479 100644 --- a/plugin-spark/src/test/scala/org/apache/ranger/services/spark/RangerAdminClientImpl.scala +++ b/plugin-spark/src/test/scala/org/apache/ranger/services/spark/RangerAdminClientImpl.scala @@ -17,7 +17,6 @@ package org.apache.ranger.services.spark -import java.io.File import java.nio.file.{Files, FileSystems} import java.util diff --git a/plugin-spark/src/test/scala/org/apache/ranger/services/spark/SparkRangerAuthorizerTest.scala b/plugin-spark/src/test/scala/org/apache/ranger/services/spark/SparkRangerAuthorizerTest.scala index c928745fce..879df9fdc9 100644 --- a/plugin-spark/src/test/scala/org/apache/ranger/services/spark/SparkRangerAuthorizerTest.scala +++ b/plugin-spark/src/test/scala/org/apache/ranger/services/spark/SparkRangerAuthorizerTest.scala @@ -321,6 +321,36 @@ class SparkRangerAuthorizerTest extends FunSuite with BeforeAndAfterAll { |`wp_image_count` INT, `wp_max_ad_count` INT) |USING parquet """.stripMargin) + + sql( + """ + |CREATE TABLE default.rangertbl1 AS SELECT * FROM default.src + """.stripMargin) + + sql( + """ + |CREATE TABLE default.rangertbl2 AS SELECT * FROM default.src + """.stripMargin) + + sql( + """ + |CREATE TABLE default.rangertbl3 AS SELECT * FROM default.src + """.stripMargin) + + sql( + """ + |CREATE TABLE default.rangertbl4 AS SELECT * FROM default.src + """.stripMargin) + + sql( + """ + |CREATE TABLE default.rangertbl5 AS SELECT * FROM default.src + """.stripMargin) + + sql( + """ + |CREATE TABLE default.rangertbl6 AS SELECT * FROM default.src + """.stripMargin) } test("simple query") { @@ -340,7 +370,7 @@ class SparkRangerAuthorizerTest extends FunSuite with BeforeAndAfterAll { assert(df.count() === 20, "keys above 20 should be filtered automatically") } withUser("alice") { - val df = spark.sql(statement) + val df = sql(statement) assert(df.count() === 500) } } @@ -348,12 +378,12 @@ class SparkRangerAuthorizerTest extends FunSuite with BeforeAndAfterAll { test("projection with ranger filter key") { val statement = "select key from default.src" withUser("bob") { - val df = spark.sql(statement) + val df = sql(statement) val row = df.take(1)(0) assert(row.getInt(0) < 20) } withUser("alice") { - val df = spark.sql(statement) + val df = sql(statement) assert(df.count() === 500) } } @@ -361,12 +391,12 @@ class SparkRangerAuthorizerTest extends FunSuite with BeforeAndAfterAll { test("projection without ranger filter key") { val statement = "select value from default.src" withUser("bob") { - val df = spark.sql(statement) + val df = sql(statement) val row = df.take(1)(0) assert(row.getString(0).split("_")(1).toInt < 20) } withUser("alice") { - val df = spark.sql(statement) + val df = sql(statement) assert(df.count() === 500) } } @@ -375,16 +405,16 @@ class SparkRangerAuthorizerTest extends FunSuite with BeforeAndAfterAll { val statement = "select key from default.src where key = 0" val statement2 = "select key from default.src where key >= 20" withUser("bob") { - val df = spark.sql(statement) + val df = sql(statement) val row = df.take(1)(0) assert(row.getInt(0) === 0) - val df2 = spark.sql(statement2) + val df2 = sql(statement2) assert(df2.count() === 0, "all keys should be filtered") } withUser("alice") { - val df = spark.sql(statement) + val df = sql(statement) assert(df.count() === 3) - val df2 = spark.sql(statement2) + val df2 = sql(statement2) assert(df2.count() === 480) } } @@ -392,14 +422,14 @@ class SparkRangerAuthorizerTest extends FunSuite with BeforeAndAfterAll { test("alias") { val statement = "select key as k1, value v1 from default.src" withUser("bob") { - val df = spark.sql(statement) + val df = sql(statement) val row = df.take(1)(0) assert(row.getInt(0) < 20, "keys above 20 should be filtered automatically") assert(row.getString(1).startsWith("x"), "values should be masked") assert(df.count() === 20, "keys above 20 should be filtered automatically") } withUser("alice") { - val df = spark.sql(statement) + val df = sql(statement) assert(df.count() === 500) } } @@ -407,19 +437,83 @@ class SparkRangerAuthorizerTest extends FunSuite with BeforeAndAfterAll { test("agg") { val statement = "select sum(key) as k1, value v1 from default.src group by v1" withUser("bob") { - val df = spark.sql(statement) + val df = sql(statement) println(df.queryExecution.optimizedPlan) val row = df.take(1)(0) assert(row.getString(1).startsWith("x"), "values should be masked") assert(row.getString(1).split("_")(1).toInt < 20) } withUser("alice") { - val df = spark.sql(statement) + val df = sql(statement) val row = df.take(1)(0) assert(row.getString(1).startsWith("val"), "values should not be masked") } } + test("with equal expression") { + val statement = "select * from default.rangertbl1" + withUser("bob") { + val df = sql(statement) + println(df.queryExecution.optimizedPlan) + val row = df.take(1)(0) + assert(row.getInt(0) === 0, "rangertbl1 has an internal expression key=0") + assert(row.getString(1).startsWith("x"), "values should be masked") + } + } + + test("with in set") { + val statement = "select * from default.rangertbl2" + withUser("bob") { + val df = sql(statement) + println(df.queryExecution.optimizedPlan) + val row = df.take(1)(0) + assert(row.getInt(0) === 0, "rangertbl2 has an internal expression key in (0, 1, 2)") + assert(row.getString(1).startsWith("val_x"), "values should show first 4 characters") + } + } + + test("with in subquery") { + val statement = "select * from default.rangertbl3" + withUser("bob") { + val df = sql(statement) + println(df.queryExecution.optimizedPlan) + val rows = df.collect() + assert(rows.forall(_.getInt(0) < 100), "rangertbl3 has an internal expression key in (query)") + assert(rows.forall(_.getString(1).length > 10), "values should be hashed") + } + } + + test("with in subquery self joined") { + val statement = "select * from default.rangertbl4" + withUser("bob") { + val df = sql(statement) + println(df.queryExecution.optimizedPlan) + val rows = df.collect() + assert(rows.length === 500) + assert(rows.forall(_.getString(1) === null), "values should be hashed") + } + } + + test("with udf") { + val statement = "select * from default.rangertbl5" + withUser("bob") { + val df = sql(statement) + println(df.queryExecution.optimizedPlan) + val rows = df.collect() + assert(rows.length === 0) + } + } + + test("with multiple expressions") { + val statement = "select * from default.rangertbl6" + withUser("bob") { + val df = sql(statement) + println(df.queryExecution.optimizedPlan) + val rows = df.collect() + assert(rows.forall { r => val x = r.getInt(0); x > 1 && x < 10 || x == 500 }) + } + } + private val tpcdsQueries = Seq( "q1", "q2", "q3", "q4", "q5", "q6", "q7", "q8", "q9", "q10", "q11", "q12", "q13", "q14a", "q14b", "q15", "q16", "q17", "q18", "q19", "q20", From 2e1429c2bc824d37dd4f774e8700ab285b3547e8 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sat, 18 May 2019 21:43:25 +0800 Subject: [PATCH 33/35] rm spark hive deps --- .../admin/client/RangerAdminRESTClient.java | 46 +- .../authorizer/RangerHiveAccessRequest.java | 2 +- .../hive/authorizer/RangerHiveAuthorizer.java | 2 +- plugin-spark/pom.xml | 7 +- .../authorizer/RangerSparkAccessRequest.scala | 46 +- .../authorizer/RangerSparkAuthorizer.scala | 423 ++++++------------ .../authorizer/RangerSparkResource.scala | 2 +- .../SparkAccessControlException.scala | 20 + .../spark/authorizer/SparkObjectType.scala | 2 +- .../spark/authorizer/SparkOperationType.scala | 36 ++ .../SparkPrivObjectActionType.scala} | 18 +- .../authorizer/SparkPrivilegeObject.scala | 136 ++++++ ...y.scala => SparkPrivilegeObjectType.scala} | 15 +- .../spark/authorizer/authorizer.scala | 6 - .../sql/catalyst/optimizer/Authorizable.scala | 152 ------- .../RangerSparkAuthorizerExtension.scala | 144 +++++- .../RangerSparkMaskingExtension.scala | 4 +- .../RangerSparkRowFilterExtension.scala | 4 +- .../optimizer/SparkPrivilegeObject.scala | 76 ---- .../RangerShowDatabasesCommand.scala | 38 ++ .../execution/RangerShowTablesCommand.scala | 38 ++ .../spark/sql/hive/PrivilegesBuilder.scala | 167 ++++--- .../hive/client/RangerSparkAuthzImpl.scala | 110 ----- .../test/resources/sparkSql_hive_jenkins.json | 101 ++++- .../RangerSparkAuthorizerExtensionTest.scala | 100 +++++ .../RangerSparkMaskingExtensionTest.scala | 52 +++ .../RangerSparkPlanOmitStrategyTest.scala | 2 - 27 files changed, 928 insertions(+), 821 deletions(-) create mode 100644 plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkAccessControlException.scala create mode 100644 plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkOperationType.scala rename plugin-spark/src/main/scala/org/apache/{spark/sql/catalyst/optimizer/Authorizer.scala => ranger/authorization/spark/authorizer/SparkPrivObjectActionType.scala} (62%) create mode 100644 plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkPrivilegeObject.scala rename plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/{RangerSparkAuthorizerFactory.scala => SparkPrivilegeObjectType.scala} (57%) delete mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala delete mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SparkPrivilegeObject.scala create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerShowDatabasesCommand.scala create mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerShowTablesCommand.scala delete mode 100644 plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/RangerSparkAuthzImpl.scala create mode 100644 plugin-spark/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkAuthorizerExtensionTest.scala create mode 100644 plugin-spark/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtensionTest.scala diff --git a/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java b/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java index d1d183423e..dddfbc7fef 100644 --- a/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java +++ b/agents-common/src/main/java/org/apache/ranger/admin/client/RangerAdminRESTClient.java @@ -104,34 +104,23 @@ public ServicePolicies getServicePoliciesIfUpdated(final long lastKnownVersion, UserGroupInformation user = MiscUtil.getUGILoginUser(); boolean isSecureMode = user != null && UserGroupInformation.isSecurityEnabled(); ClientResponse response = null; -// if (isSecureMode) { -// if (LOG.isDebugEnabled()) { -// LOG.debug("Checking Service policy if updated as user : " + user); -// } -// PrivilegedAction action = new PrivilegedAction() { -// public ClientResponse run() { -// WebResource secureWebResource = createWebResource(RangerRESTUtils.REST_URL_POLICY_GET_FOR_SECURE_SERVICE_IF_UPDATED + serviceName) -// .queryParam(RangerRESTUtils.REST_PARAM_LAST_KNOWN_POLICY_VERSION, Long.toString(lastKnownVersion)) -// .queryParam(RangerRESTUtils.REST_PARAM_LAST_ACTIVATION_TIME, Long.toString(lastActivationTimeInMillis)) -// .queryParam(RangerRESTUtils.REST_PARAM_PLUGIN_ID, pluginId) -// .queryParam(RangerRESTUtils.REST_PARAM_CLUSTER_NAME, clusterName); -// return secureWebResource.accept(RangerRESTUtils.REST_MIME_TYPE_JSON).get(ClientResponse.class); -// } -// }; -// response = user.doAs(action); -// } else { -// if (LOG.isDebugEnabled()) { -// LOG.debug("Checking Service policy if updated with old api call"); -// } -// WebResource webResource = createWebResource(RangerRESTUtils.REST_URL_POLICY_GET_FOR_SERVICE_IF_UPDATED + serviceName) -// .queryParam(RangerRESTUtils.REST_PARAM_LAST_KNOWN_POLICY_VERSION, Long.toString(lastKnownVersion)) -// .queryParam(RangerRESTUtils.REST_PARAM_LAST_ACTIVATION_TIME, Long.toString(lastActivationTimeInMillis)) -// .queryParam(RangerRESTUtils.REST_PARAM_PLUGIN_ID, pluginId) -// .queryParam(RangerRESTUtils.REST_PARAM_CLUSTER_NAME, clusterName); -// response = webResource.accept(RangerRESTUtils.REST_MIME_TYPE_JSON).get(ClientResponse.class); -// } - - if (LOG.isDebugEnabled()) { + if (isSecureMode) { + if (LOG.isDebugEnabled()) { + LOG.debug("Checking Service policy if updated as user : " + user); + } + PrivilegedAction action = new PrivilegedAction() { + public ClientResponse run() { + WebResource secureWebResource = createWebResource(RangerRESTUtils.REST_URL_POLICY_GET_FOR_SECURE_SERVICE_IF_UPDATED + serviceName) + .queryParam(RangerRESTUtils.REST_PARAM_LAST_KNOWN_POLICY_VERSION, Long.toString(lastKnownVersion)) + .queryParam(RangerRESTUtils.REST_PARAM_LAST_ACTIVATION_TIME, Long.toString(lastActivationTimeInMillis)) + .queryParam(RangerRESTUtils.REST_PARAM_PLUGIN_ID, pluginId) + .queryParam(RangerRESTUtils.REST_PARAM_CLUSTER_NAME, clusterName); + return secureWebResource.accept(RangerRESTUtils.REST_MIME_TYPE_JSON).get(ClientResponse.class); + } + }; + response = user.doAs(action); + } else { + if (LOG.isDebugEnabled()) { LOG.debug("Checking Service policy if updated with old api call"); } WebResource webResource = createWebResource(RangerRESTUtils.REST_URL_POLICY_GET_FOR_SERVICE_IF_UPDATED + serviceName) @@ -140,6 +129,7 @@ public ServicePolicies getServicePoliciesIfUpdated(final long lastKnownVersion, .queryParam(RangerRESTUtils.REST_PARAM_PLUGIN_ID, pluginId) .queryParam(RangerRESTUtils.REST_PARAM_CLUSTER_NAME, clusterName); response = webResource.accept(RangerRESTUtils.REST_MIME_TYPE_JSON).get(ClientResponse.class); + } if (response == null || response.getStatus() == HttpServletResponse.SC_NOT_MODIFIED) { if (response == null) { diff --git a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAccessRequest.java b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAccessRequest.java index 81fa83411f..a8bf5435b9 100644 --- a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAccessRequest.java +++ b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAccessRequest.java @@ -22,9 +22,9 @@ import java.util.Date; import java.util.Set; -import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext; import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext; import org.apache.ranger.plugin.policyengine.RangerAccessRequestImpl; import org.apache.ranger.plugin.policyengine.RangerPolicyEngine; import org.apache.ranger.plugin.util.RangerAccessRequestUtil; diff --git a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java index 8aebb8b1a2..0f3bee547d 100644 --- a/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java +++ b/hive-agent/src/main/java/org/apache/ranger/authorization/hive/authorizer/RangerHiveAuthorizer.java @@ -518,7 +518,7 @@ public List filterListCmdObjects(List if (result == null) { LOG.error("filterListCmdObjects: Internal error: null RangerAccessResult object received back from isAccessAllowed()!"); } else if (!result.getIsAllowed()) { - if (LOG.isDebugEnabled()) { + if (!LOG.isDebugEnabled()) { String path = resource.getAsString(); LOG.debug(String.format("filterListCmdObjects: Permission denied: user [%s] does not have [%s] privilege on [%s]. resource[%s], request[%s], result[%s]", user, request.getHiveAccessType().name(), path, resource, request, result)); diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml index d16225d2bf..87a7859f01 100644 --- a/plugin-spark/pom.xml +++ b/plugin-spark/pom.xml @@ -28,7 +28,6 @@ 2.2.6 2.3.2 provided - 1.2.1.spark2 ranger_spark_project @@ -55,6 +54,12 @@ spark-hive_${scala.binary.version} ${spark.version} ${spark.scope} + + + org.spark-project + * + + org.apache.ranger diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAccessRequest.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAccessRequest.scala index df4a93b97e..11a0746054 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAccessRequest.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAccessRequest.scala @@ -19,13 +19,12 @@ package org.apache.ranger.authorization.spark.authorizer import java.util.Date -import java.util.{Set => JSet} - -import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveAuthzContext, HiveAuthzSessionContext, HiveOperationType} import org.apache.ranger.authorization.spark.authorizer.SparkAccessType.SparkAccessType import org.apache.ranger.plugin.policyengine.{RangerAccessRequestImpl, RangerPolicyEngine} import org.apache.ranger.plugin.util.RangerAccessRequestUtil +import scala.collection.JavaConverters._ + class RangerSparkAccessRequest private extends RangerAccessRequestImpl { private var accessType = SparkAccessType.NONE @@ -33,53 +32,24 @@ class RangerSparkAccessRequest private extends RangerAccessRequestImpl { def this( resource: RangerSparkResource, user: String, - userGroups: JSet[String], - hiveOpTypeName: String, + groups: Set[String], + opType: String, accessType: SparkAccessType, - context: HiveAuthzContext, - sessionContext: HiveAuthzSessionContext, clusterName: String) { this() this.setResource(resource) this.setUser(user) - this.setUserGroups(userGroups) + this.setUserGroups(groups.asJava) this.setAccessTime(new Date) - this.setAction(hiveOpTypeName) + this.setAction(opType) this.setSparkAccessType(accessType) this.setUser(user) - if (context != null) { - this.setRequestData(context.getCommandString) - this.setRemoteIPAddress(context.getIpAddress) - } - if (sessionContext != null) { - this.setClientType(Option(sessionContext.getClientType).map(_.toString).orNull) - this.setSessionId(sessionContext.getSessionString) - } this.setClusterName(clusterName) } - def this( - resource: RangerSparkResource, - user: String, - userGroups: JSet[String], - hiveOpType: HiveOperationType, - accessType: SparkAccessType, - context: HiveAuthzContext, - sessionContext: HiveAuthzSessionContext, - clusterName: String) = { - this(resource, user, userGroups, hiveOpType.name(), accessType, context, sessionContext, - clusterName) - } - - def this( - resource: RangerSparkResource, - user: String, - userGroups: JSet[String], - context: HiveAuthzContext, - sessionContext: HiveAuthzSessionContext, + def this(resource: RangerSparkResource, user: String, groups: Set[String], clusterName: String) = { - this(resource, user, userGroups, "METADATA OPERATION", SparkAccessType.USE, context, - sessionContext, clusterName) + this(resource, user, groups, "METADATA OPERATION", SparkAccessType.USE, clusterName) } def getSparkAccessType: SparkAccessType = accessType diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala index 4c1024bea3..dce419f791 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizer.scala @@ -21,51 +21,38 @@ import java.util.{List => JList} import org.apache.commons.lang.StringUtils import org.apache.commons.logging.LogFactory -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.permission.FsAction +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.common.FileUtils -import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.security.authorization.plugin._ -import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.{HivePrivilegeObjectType, HivePrivObjectActionType} import org.apache.hadoop.security.UserGroupInformation import org.apache.ranger.authorization.spark.authorizer.SparkAccessType.SparkAccessType import org.apache.ranger.authorization.spark.authorizer.SparkObjectType.SparkObjectType +import org.apache.ranger.authorization.spark.authorizer.SparkOperationType.SparkOperationType import org.apache.ranger.authorization.utils.StringUtil import org.apache.ranger.plugin.policyengine.RangerAccessRequest import org.apache.ranger.plugin.util.RangerPerfTracer +import org.apache.spark.sql.SparkSession import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -class RangerSparkAuthorizer( - metastoreClientFactory: HiveMetastoreClientFactory, - hiveConf: HiveConf, - hiveAuthenticator: HiveAuthenticationProvider, - sessionContext: HiveAuthzSessionContext) extends HiveAuthorizer { - - import RangerSparkAuthorizer._ - private val mUgi = if (hiveAuthenticator == null) { - null - } else { - Option(hiveAuthenticator.getUserName).map(UserGroupInformation.createRemoteUser).orNull - } - +object RangerSparkAuthorizer { + private val LOG = LogFactory.getLog(this.getClass.getSimpleName.stripSuffix("$")) private val sparkPlugin = RangerSparkPlugin.build().getOrCreate() - override def getVersion: HiveAuthorizer.VERSION = HiveAuthorizer.VERSION.V1 + private def currentUser: UserGroupInformation = UserGroupInformation.getCurrentUser - override def checkPrivileges( - hiveOpType: HiveOperationType, - inputsHObjs: JList[SparkPrivilegeObject], - outputHObjs: JList[SparkPrivilegeObject], - context: HiveAuthzContext): Unit = { + def checkPrivileges( + spark: SparkSession, + opType: SparkOperationType, + inputs: Seq[SparkPrivilegeObject], + outputs: Seq[SparkPrivilegeObject]): Unit = { - if (mUgi == null) { - throw new HiveAccessControlException("Permission denied: user information not available") - } - val user = mUgi.getShortUserName - val groups = mUgi.getGroupNames.toSet.asJava + val ugi = currentUser + val user = ugi.getShortUserName + val groups = ugi.getGroupNames.toSet val auditHandler = new RangerSparkAuditHandler val perf = if (RangerPerfTracer.isPerfTraceEnabled(PERF_SPARKAUTH_REQUEST_LOG)) { RangerPerfTracer.getPerfTracer(PERF_SPARKAUTH_REQUEST_LOG, @@ -75,38 +62,39 @@ class RangerSparkAuthorizer( } try { val requests = new ArrayBuffer[RangerSparkAccessRequest]() - if (inputsHObjs.isEmpty && hiveOpType == HiveOperationType.SHOWDATABASES) { + if (inputs.isEmpty && opType == SparkOperationType.SHOWDATABASES) { val resource = new RangerSparkResource(SparkObjectType.DATABASE, None) - requests += new RangerSparkAccessRequest(resource, user, groups, hiveOpType.name, - SparkAccessType.USE, context, sessionContext, sparkPlugin.getClusterName) + requests += new RangerSparkAccessRequest(resource, user, groups, opType.toString, + SparkAccessType.USE, sparkPlugin.getClusterName) } - def addAccessRequest(objs: JList[SparkPrivilegeObject], isInput: Boolean): Unit = { - objs.asScala.foreach { obj => - val resource = getSparkResource(obj, hiveOpType) + def addAccessRequest(objs: Seq[SparkPrivilegeObject], isInput: Boolean): Unit = { + objs.foreach { obj => + val resource = getSparkResource(obj, opType) if (resource != null) { val objectName = obj.getObjectName val objectType = resource.getObjectType if (objectType == SparkObjectType.URI && isPathInFSScheme(objectName)) { - val fsAction = getURIAccessType(hiveOpType) - if (!isURIAccessAllowed(user, fsAction, objectName, hiveConf)) { + val fsAction = getURIAccessType(opType) + val hadoopConf = spark.sparkContext.hadoopConfiguration + if (!canAccessURI(user, fsAction, objectName, hadoopConf)) { throw new HiveAccessControlException(s"Permission denied: user [$user] does not" + s" have [${fsAction.name}] privilege on [$objectName]") } } else { - val accessType = getAccessType(obj, hiveOpType, objectType, isInput) + val accessType = getAccessType(obj, opType, objectType, isInput) if (accessType != SparkAccessType.NONE && !requests.exists( o => o.getSparkAccessType == accessType && o.getResource == resource)) { - requests += new RangerSparkAccessRequest(resource, user, groups, hiveOpType, - accessType, context, sessionContext, sparkPlugin.getClusterName) + requests += new RangerSparkAccessRequest(resource, user, groups, opType.toString, + accessType, sparkPlugin.getClusterName) } } } } } - addAccessRequest(inputsHObjs, isInput = true) - addAccessRequest(outputHObjs, isInput = false) + addAccessRequest(inputs, isInput = true) + addAccessRequest(outputs, isInput = false) requests.foreach { request => val resource = request.getResource.asInstanceOf[RangerSparkResource] if (resource.getObjectType == SparkObjectType.COLUMN && @@ -124,7 +112,7 @@ class RangerSparkAuthorizer( if (colResults != null) { for (c <- colResults.asScala) { if (c != null && !c.getIsAllowed) { - throw new HiveAccessControlException(s"Permission denied: user [$user] does not" + + throw new SparkAccessControlException(s"Permission denied: user [$user] does not" + s" have [${request.getSparkAccessType}] privilege on [${resource.getAsString}]") } } @@ -132,7 +120,7 @@ class RangerSparkAuthorizer( } else { val result = sparkPlugin.isAccessAllowed(request, auditHandler) if (result != null && !result.getIsAllowed) { - throw new HiveAccessControlException(s"Permission denied: user [$user] does not" + + throw new SparkAccessControlException(s"Permission denied: user [$user] does not" + s" have [${request.getSparkAccessType}] privilege on [${resource.getAsString}]") } } @@ -143,160 +131,36 @@ class RangerSparkAuthorizer( } } - override def filterListCmdObjects( - listObjs: JList[SparkPrivilegeObject], - context: HiveAuthzContext): JList[SparkPrivilegeObject] = { - if (LOG.isDebugEnabled) LOG.debug(s"==> filterListCmdObjects($listObjs, $context)") - - val perf = if (RangerPerfTracer.isPerfTraceEnabled(PERF_SPARKAUTH_REQUEST_LOG)) { - RangerPerfTracer.getPerfTracer(PERF_SPARKAUTH_REQUEST_LOG, - "RangerSparkAuthorizer.filterListCmdObjects()") - } else { - null - } - - val ret = - if (listObjs == null) { - LOG.debug("filterListCmdObjects: meta objects list was null!") - null - } else if (listObjs.isEmpty) { - listObjs - } else if (mUgi == null) { - LOG.warn("filterListCmdObjects: user information not available") - listObjs - } else { - if (LOG.isDebugEnabled) { - LOG.debug(s"filterListCmdObjects: number of input objects[${listObjs.size}]") - } - val user = mUgi.getShortUserName - val groups = mUgi.getGroupNames.toSet.asJava - if (LOG.isDebugEnabled) { - LOG.debug(s"filterListCmdObjects: user[$user], groups[$groups]") - } - - listObjs.asScala.filter { obj => + def isAllowed(obj: SparkPrivilegeObject): Boolean = { + val ugi = currentUser + val user = ugi.getShortUserName + val groups = ugi.getGroupNames.toSet + createSparkResource(obj) match { + case Some(resource) => + val request = + new RangerSparkAccessRequest(resource, user, groups, sparkPlugin.getClusterName) + val result = sparkPlugin.isAccessAllowed(request) + if (request == null) { + LOG.error("Internal error: null RangerAccessResult received back from isAccessAllowed") + false + } else if (!result.getIsAllowed) { if (LOG.isDebugEnabled) { - LOG.debug(s"filterListCmdObjects: actionType[${obj.getActionType}]," + - s" objectType[${obj.getType}], objectName[${obj.getObjectName}]," + - s" dbName[${obj.getDbname}], columns[${obj.getColumns}]," + - s" partitionKeys[${obj.getPartKeys}];" + - s" context: commandString[${Option(context).map(_.getCommandString).getOrElse("")}" + - s"], ipAddress[${Option(context).map(_.getIpAddress).getOrElse("")}]") + val path = resource.getAsString + LOG.debug(s"Permission denied: user [$user] does not have" + + s" [${request.getSparkAccessType}] privilege on [$path]. resource[$resource]," + + s" request[$request], result[$result]") } - createSparkResource(obj) match { - case Some(resource) => - val request = new RangerSparkAccessRequest( - resource, user, groups, context, sessionContext, sparkPlugin.getClusterName) - val result = sparkPlugin.isAccessAllowed(request) - if (request == null) { - LOG.error("filterListCmdObjects: Internal error: null RangerAccessResult object" + - " received back from isAccessAllowed()") - false - } else if (!result.getIsAllowed) { - if (LOG.isDebugEnabled) { - val path = resource.getAsString - LOG.debug(s"filterListCmdObjects: Permission denied: user [$user] does not have" + - s" [${request.getSparkAccessType}] privilege on [$path]. resource[$resource]," + - s" request[$request], result[$result]") - } - false - } else { - true - } - case _ => - LOG.error("filterListCmdObjects: RangerSparkResource returned by createHiveResource" + - " is null") - false - } - }.asJava - } - RangerPerfTracer.log(perf) - ret - } - - def getSparkResource( - hiveObj: SparkPrivilegeObject, - hiveOpType: HiveOperationType): RangerSparkResource = { - import SparkObjectType._ - val objectType = getObjectType(hiveObj, hiveOpType) - val resource = objectType match { - case DATABASE => RangerSparkResource(objectType, Option(hiveObj.getDbname)) - case TABLE | VIEW | PARTITION | FUNCTION => - RangerSparkResource(objectType, Option(hiveObj.getDbname), hiveObj.getObjectName) - case COLUMN => - RangerSparkResource(objectType, Option(hiveObj.getDbname), hiveObj.getObjectName, - StringUtils.join(hiveObj.getColumns, ",")) - case URI => RangerSparkResource(objectType, Option(hiveObj.getObjectName)) - case _ => null + false + } else { + true + } + case _ => + LOG.error("RangerSparkResource returned by createSparkResource is null") + false } - if (resource != null) resource.setServiceDef(sparkPlugin.getServiceDef) - resource - } - private def isPathInFSScheme(objectName: String): Boolean = { - objectName.nonEmpty && sparkPlugin.fsScheme.exists(objectName.startsWith) } - - ///////////////////////////////////////////////////////////////////////////////////////////////// - // Spark SQL supports no Hive DCLs, remain the functions with a default implementation // - ///////////////////////////////////////////////////////////////////////////////////////////////// - - override def grantPrivileges( - hivePrincipals: JList[HivePrincipal], - hivePrivileges: JList[HivePrivilege], - hivePrivObject: SparkPrivilegeObject, - grantorPrincipal: HivePrincipal, - grantOption: Boolean): Unit = {} - - override def revokePrivileges(hivePrincipals: JList[HivePrincipal], - hivePrivileges: JList[HivePrivilege], - hivePrivObject: SparkPrivilegeObject, - grantorPrincipal: HivePrincipal, - grantOption: Boolean): Unit = {} - - override def createRole(roleName: String, adminGrantor: HivePrincipal): Unit = {} - - override def dropRole(roleName: String): Unit = {} - - override def getPrincipalGrantInfoForRole(roleName: String): JList[HiveRoleGrant] = { - Seq.empty.asJava - } - - override def getRoleGrantInfoForPrincipal(principal: HivePrincipal): JList[HiveRoleGrant] = { - Seq.empty.asJava - } - - override def grantRole( - hivePrincipals: JList[HivePrincipal], - roles: JList[String], - grantOption: Boolean, - grantorPrinc: HivePrincipal): Unit = {} - - override def revokeRole( - hivePrincipals: JList[HivePrincipal], - roles: JList[String], - grantOption: Boolean, - grantorPrinc: HivePrincipal): Unit = {} - - override def getAllRoles: JList[String] = Seq.empty.asJava - - override def showPrivileges( - principal: HivePrincipal, - privObj: SparkPrivilegeObject): JList[HivePrivilegeInfo] = Seq.empty.asJava - - override def setCurrentRole(roleName: String): Unit = {} - - override def getCurrentRoleNames: JList[String] = Seq.empty.asJava - - override def applyAuthorizationConfigPolicy(hiveConf: HiveConf): Unit = {} -} - -object RangerSparkAuthorizer { - import HivePrivilegeObjectType._ - - private val LOG = LogFactory.getLog(classOf[RangerSparkAuthorizer].getSimpleName.stripSuffix("$")) - private val PERF_SPARKAUTH_REQUEST_LOG = RangerPerfTracer.getPerfLogger("sparkauth.request") def createSparkResource(privilegeObject: SparkPrivilegeObject): Option[RangerSparkResource] = { @@ -304,53 +168,90 @@ object RangerSparkAuthorizer { val dbName = privilegeObject.getDbname val objectType = privilegeObject.getType objectType match { - case DATABASE => + case SparkPrivilegeObjectType.DATABASE => Some(RangerSparkResource(SparkObjectType.DATABASE, Option(objectName))) - case TABLE_OR_VIEW => + case SparkPrivilegeObjectType.TABLE_OR_VIEW => Some(RangerSparkResource(SparkObjectType.DATABASE, Option(dbName), objectName)) case _ => - LOG.warn(s"RangerSparkAuthorizer.createHiveResource: unexpected objectType: $objectType") + LOG.warn(s"RangerSparkAuthorizer.createSparkResource: unexpected objectType: $objectType") None } } - private def getObjectType( - hiveObj: SparkPrivilegeObject, - hiveOpType: HiveOperationType): SparkObjectType = hiveObj.getType match { - case DATABASE | null => SparkObjectType.DATABASE - case PARTITION => SparkObjectType.PARTITION - case TABLE_OR_VIEW if hiveOpType.name.toLowerCase.contains("view") => SparkObjectType.VIEW - case TABLE_OR_VIEW => SparkObjectType.TABLE - case FUNCTION => SparkObjectType.FUNCTION - case DFS_URI | LOCAL_URI => SparkObjectType.URI - case _ => SparkObjectType.NONE + private def getAccessType(obj: SparkPrivilegeObject, opType: SparkOperationType, + objectType: SparkObjectType, isInput: Boolean): SparkAccessType = { + objectType match { + case SparkObjectType.URI if isInput => SparkAccessType.READ + case SparkObjectType.URI => SparkAccessType.WRITE + case _ => obj.getActionType match { + case SparkPrivObjectActionType.INSERT | SparkPrivObjectActionType.INSERT_OVERWRITE => + SparkAccessType.UPDATE + case SparkPrivObjectActionType.OTHER => + import SparkOperationType._ + opType match { + case CREATEDATABASE if obj.getType == SparkPrivilegeObjectType.DATABASE => + SparkAccessType.CREATE + case CREATEFUNCTION if obj.getType == SparkPrivilegeObjectType.FUNCTION => + SparkAccessType.CREATE + case CREATETABLE | CREATEVIEW | CREATETABLE_AS_SELECT + if obj.getType == SparkPrivilegeObjectType.TABLE_OR_VIEW => + if (isInput) SparkAccessType.SELECT else SparkAccessType.CREATE + case ALTERDATABASE | ALTERTABLE_ADDCOLS | + ALTERTABLE_ADDPARTS | ALTERTABLE_DROPPARTS | + ALTERTABLE_LOCATION | ALTERTABLE_PROPERTIES | ALTERTABLE_SERDEPROPERTIES | + ALTERVIEW_RENAME | MSCK => SparkAccessType.ALTER + case DROPFUNCTION | DROPTABLE | DROPVIEW | DROPDATABASE => + SparkAccessType.DROP + case LOAD => if (isInput) SparkAccessType.SELECT else SparkAccessType.UPDATE + case QUERY | SHOW_CREATETABLE | SHOWPARTITIONS | + SHOW_TBLPROPERTIES => SparkAccessType.SELECT + case SHOWCOLUMNS | DESCTABLE => + StringUtil.toLower(RangerSparkPlugin.showColumnsOption) match { + case "show-all" => SparkAccessType.USE + case _ => SparkAccessType.SELECT + } + case SHOWDATABASES | SWITCHDATABASE | DESCDATABASE| SHOWTABLES => SparkAccessType.USE + case TRUNCATETABLE => SparkAccessType.UPDATE + case _ => SparkAccessType.NONE + } + } + } } - private def getURIAccessType(hiveOpType: HiveOperationType): FsAction = { - import HiveOperationType._ + private def getObjectType( + obj: SparkPrivilegeObject, opType: SparkOperationType): SparkObjectType = { + obj.getType match { + case SparkPrivilegeObjectType.DATABASE | null => SparkObjectType.DATABASE + case SparkPrivilegeObjectType.TABLE_OR_VIEW if !StringUtil.isEmpty(obj.getColumns.asJava) => + SparkObjectType.COLUMN + case SparkPrivilegeObjectType.TABLE_OR_VIEW if opType.toString.toLowerCase.contains("view") => + SparkObjectType.VIEW + case SparkPrivilegeObjectType.TABLE_OR_VIEW => SparkObjectType.TABLE + case SparkPrivilegeObjectType.FUNCTION => SparkObjectType.FUNCTION + case SparkPrivilegeObjectType.DFS_URI => SparkObjectType.URI + case _ => SparkObjectType.NONE + } + } - hiveOpType match { - case LOAD | IMPORT => FsAction.READ - case EXPORT => FsAction.WRITE - case CREATEDATABASE | CREATETABLE | CREATETABLE_AS_SELECT | ALTERDATABASE | - ALTERDATABASE_OWNER | ALTERTABLE_ADDCOLS | ALTERTABLE_REPLACECOLS | - ALTERTABLE_RENAMECOL | ALTERTABLE_RENAMEPART | ALTERTABLE_RENAME | - ALTERTABLE_DROPPARTS | ALTERTABLE_ADDPARTS | ALTERTABLE_TOUCH | - ALTERTABLE_ARCHIVE | ALTERTABLE_UNARCHIVE | ALTERTABLE_PROPERTIES | - ALTERTABLE_SERIALIZER | ALTERTABLE_PARTCOLTYPE | ALTERTABLE_SERDEPROPERTIES | - ALTERTABLE_CLUSTER_SORT | ALTERTABLE_BUCKETNUM | ALTERTABLE_UPDATETABLESTATS | - ALTERTABLE_UPDATEPARTSTATS | ALTERTABLE_PROTECTMODE | ALTERTABLE_FILEFORMAT | - ALTERTABLE_LOCATION | ALTERINDEX_PROPS | ALTERTABLE_MERGEFILES | ALTERTABLE_SKEWED | - ALTERTABLE_COMPACT | ALTERPARTITION_SERIALIZER | ALTERPARTITION_SERIALIZER | - ALTERPARTITION_SERDEPROPERTIES | ALTERPARTITION_BUCKETNUM | ALTERPARTITION_PROTECTMODE | - ALTERPARTITION_FILEFORMAT | ALTERPARTITION_LOCATION | ALTERPARTITION_MERGEFILES | - ALTERTBLPART_SKEWED_LOCATION | QUERY => FsAction.ALL - case _ => FsAction.NONE + private def getSparkResource( + obj: SparkPrivilegeObject, opType: SparkOperationType): RangerSparkResource = { + import SparkObjectType._ + val objectType = getObjectType(obj, opType) + val resource = objectType match { + case DATABASE => RangerSparkResource(objectType, Option(obj.getDbname)) + case TABLE | VIEW | FUNCTION => + RangerSparkResource(objectType, Option(obj.getDbname), obj.getObjectName) + case COLUMN => + RangerSparkResource(objectType, Option(obj.getDbname), obj.getObjectName, + obj.getColumns.mkString(",")) + case _ => null } + if (resource != null) resource.setServiceDef(sparkPlugin.getServiceDef) + resource } - private def isURIAccessAllowed( - userName: String, action: FsAction, uri: String, conf: HiveConf): Boolean = action match { + private def canAccessURI( + user: String, action: FsAction, uri: String, conf: Configuration): Boolean = action match { case FsAction.NONE => true case _ => try { @@ -358,11 +259,11 @@ object RangerSparkAuthorizer { val fs = FileSystem.get(filePath.toUri, conf) val fileStat = fs.globStatus(filePath) if (fileStat != null && fileStat.nonEmpty) fileStat.forall { file => - FileUtils.isOwnerOfFileHierarchy(fs, file, userName) || - FileUtils.isActionPermittedForFileHierarchy(fs, file, userName, action) + FileUtils.isOwnerOfFileHierarchy(fs, file, user) || + FileUtils.isActionPermittedForFileHierarchy(fs, file, user, action) } else { val file = FileUtils.getPathOrParentThatExists(fs, filePath) - FileUtils.checkFileAccessWithImpersonation(fs, file, action, userName) + FileUtils.checkFileAccessWithImpersonation(fs, file, action, user) true } } catch { @@ -372,58 +273,20 @@ object RangerSparkAuthorizer { } } - private def getAccessType( - hiveObj: SparkPrivilegeObject, - hiveOpType: HiveOperationType, - sparkObjectType: SparkObjectType, - isInput: Boolean): SparkAccessType = { - sparkObjectType match { - case SparkObjectType.URI if isInput => SparkAccessType.READ - case SparkObjectType.URI => SparkAccessType.WRITE - case _ => hiveObj.getActionType match { - case HivePrivObjectActionType.INSERT | HivePrivObjectActionType.INSERT_OVERWRITE | - HivePrivObjectActionType.UPDATE | HivePrivObjectActionType.DELETE => - SparkAccessType.UPDATE - case HivePrivObjectActionType.OTHER => - import HiveOperationType._ - hiveOpType match { - case CREATEDATABASE if hiveObj.getType == HivePrivilegeObjectType.DATABASE => - SparkAccessType.CREATE - case CREATEFUNCTION if hiveObj.getType == HivePrivilegeObjectType.FUNCTION => - SparkAccessType.CREATE - case CREATETABLE | CREATEVIEW | CREATETABLE_AS_SELECT - if hiveObj.getType == HivePrivilegeObjectType.TABLE_OR_VIEW => - if (isInput) SparkAccessType.SELECT else SparkAccessType.CREATE - case ALTERDATABASE | ALTERDATABASE_OWNER | ALTERINDEX_PROPS | ALTERINDEX_REBUILD | - ALTERPARTITION_BUCKETNUM | ALTERPARTITION_FILEFORMAT | ALTERPARTITION_LOCATION | - ALTERPARTITION_MERGEFILES | ALTERPARTITION_PROTECTMODE | - ALTERPARTITION_SERDEPROPERTIES | ALTERPARTITION_SERIALIZER | ALTERTABLE_ADDCOLS | - ALTERTABLE_ADDPARTS | ALTERTABLE_ARCHIVE | ALTERTABLE_BUCKETNUM | - ALTERTABLE_CLUSTER_SORT | ALTERTABLE_COMPACT | ALTERTABLE_DROPPARTS | - ALTERTABLE_FILEFORMAT | ALTERTABLE_LOCATION | ALTERTABLE_MERGEFILES | - ALTERTABLE_PARTCOLTYPE | ALTERTABLE_PROPERTIES | ALTERTABLE_PROTECTMODE | - ALTERTABLE_RENAME | ALTERTABLE_RENAMECOL | ALTERTABLE_RENAMEPART | - ALTERTABLE_REPLACECOLS | ALTERTABLE_SERDEPROPERTIES | ALTERTABLE_SERIALIZER | - ALTERTABLE_SKEWED | ALTERTABLE_TOUCH | ALTERTABLE_UNARCHIVE | - ALTERTABLE_UPDATEPARTSTATS | ALTERTABLE_UPDATETABLESTATS | - ALTERTBLPART_SKEWED_LOCATION | ALTERVIEW_AS | ALTERVIEW_PROPERTIES | - ALTERVIEW_RENAME | DROPVIEW_PROPERTIES | MSCK => SparkAccessType.ALTER - case DROPFUNCTION | DROPINDEX | DROPTABLE | DROPVIEW | DROPDATABASE => - SparkAccessType.DROP - case IMPORT => if (isInput) SparkAccessType.SELECT else SparkAccessType.CREATE - case EXPORT | LOAD => if (isInput) SparkAccessType.SELECT else SparkAccessType.UPDATE - case QUERY | SHOW_TABLESTATUS | SHOW_CREATETABLE | SHOWINDEXES | SHOWPARTITIONS | - SHOW_TBLPROPERTIES | ANALYZE_TABLE => SparkAccessType.SELECT - case SHOWCOLUMNS | DESCTABLE => - StringUtil.toLower(RangerSparkPlugin.showColumnsOption) match { - case "show-all" => SparkAccessType.USE - case _ => SparkAccessType.SELECT - } - case SHOWDATABASES | SWITCHDATABASE | DESCDATABASE| SHOWTABLES => SparkAccessType.USE - case TRUNCATETABLE => SparkAccessType.UPDATE - case _ => SparkAccessType.NONE - } - } + private def getURIAccessType(operationType: SparkOperationType): FsAction = { + import SparkOperationType._ + + operationType match { + case LOAD => FsAction.READ + case CREATEDATABASE | CREATETABLE | CREATETABLE_AS_SELECT | ALTERDATABASE | + ALTERTABLE_ADDCOLS | ALTERTABLE_RENAMECOL | ALTERTABLE_RENAMEPART | ALTERTABLE_RENAME | + ALTERTABLE_DROPPARTS | ALTERTABLE_ADDPARTS | ALTERTABLE_PROPERTIES | + ALTERTABLE_SERDEPROPERTIES | ALTERTABLE_LOCATION | QUERY => FsAction.ALL + case _ => FsAction.NONE } } + + private def isPathInFSScheme(objectName: String): Boolean = { + objectName.nonEmpty && sparkPlugin.fsScheme.exists(objectName.startsWith) + } } diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala index b366d53f57..2daa6368fc 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkResource.scala @@ -45,7 +45,7 @@ class RangerSparkResource( setValue(KEY_DATABASE, databaseOrUrl.getOrElse("*")) setValue(KEY_TABLE, tableOrUdf) setValue(KEY_COLUMN, column) - case TABLE | VIEW | PARTITION => + case TABLE | VIEW => setValue(KEY_DATABASE, databaseOrUrl.getOrElse("*")) setValue(KEY_TABLE, tableOrUdf) case URI => setValue(KEY_URL, databaseOrUrl.getOrElse("*")) diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkAccessControlException.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkAccessControlException.scala new file mode 100644 index 0000000000..78837dfa9a --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkAccessControlException.scala @@ -0,0 +1,20 @@ +/* + * 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.ranger.authorization.spark.authorizer + +class SparkAccessControlException(msg: String) extends Exception(msg) diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkObjectType.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkObjectType.scala index 7871e836f0..826a3d368d 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkObjectType.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkObjectType.scala @@ -20,5 +20,5 @@ package org.apache.ranger.authorization.spark.authorizer object SparkObjectType extends Enumeration { type SparkObjectType = Value - val NONE, DATABASE, TABLE, VIEW, PARTITION, COLUMN, FUNCTION, URI = Value + val NONE, DATABASE, TABLE, VIEW, COLUMN, FUNCTION, URI = Value } diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkOperationType.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkOperationType.scala new file mode 100644 index 0000000000..b23885563d --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkOperationType.scala @@ -0,0 +1,36 @@ +/* + * 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.ranger.authorization.spark.authorizer + +/** + * Subset of HiveOperationTypes supported by Apache Spark. + * + */ +object SparkOperationType extends Enumeration { + type SparkOperationType = Value + + val + ALTERDATABASE, ALTERTABLE_ADDCOLS, ALTERTABLE_ADDPARTS, ALTERTABLE_RENAMECOL, + ALTERTABLE_DROPPARTS, MSCK, ALTERTABLE_RENAMEPART, ALTERTABLE_RENAME, + ALTERVIEW_RENAME, ALTERTABLE_PROPERTIES, ALTERTABLE_SERDEPROPERTIES, + ALTERTABLE_LOCATION, QUERY, CREATEDATABASE, CREATETABLE_AS_SELECT, CREATEFUNCTION, CREATETABLE, + CREATEVIEW, DESCTABLE, DESCDATABASE, DESCFUNCTION, DROPDATABASE, DROPTABLE, DROPFUNCTION, LOAD, + SHOWCONF, SWITCHDATABASE, SHOW_CREATETABLE, SHOWCOLUMNS, SHOWDATABASES, SHOWFUNCTIONS, + SHOWPARTITIONS, SHOWTABLES, SHOW_TBLPROPERTIES, TRUNCATETABLE, DROPVIEW, EXPLAIN = Value + +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizer.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkPrivObjectActionType.scala similarity index 62% rename from plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizer.scala rename to plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkPrivObjectActionType.scala index be2c263163..e320439dfa 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizer.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkPrivObjectActionType.scala @@ -15,19 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.optimizer +package org.apache.ranger.authorization.spark.authorizer -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.rules.Rule - -/** - * An Optimizer Rule to do Hive Authorization V2 for Spark SQL. - * - * For Apache Spark 2.1.x - */ -object Authorizer extends Rule[LogicalPlan] with Authorizable { - override def spark: SparkSession = { - SparkSession.getActiveSession.getOrElse(SparkSession.getDefaultSession.get) - } +object SparkPrivObjectActionType extends Enumeration { + type SparkPrivObjectActionType = Value + val OTHER, INSERT, INSERT_OVERWRITE = Value } diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkPrivilegeObject.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkPrivilegeObject.scala new file mode 100644 index 0000000000..4f67ff775a --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkPrivilegeObject.scala @@ -0,0 +1,136 @@ +/* + * 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.ranger.authorization.spark.authorizer + +import org.apache.ranger.authorization.spark.authorizer.SparkPrivilegeObjectType.SparkPrivilegeObjectType + +import scala.collection.JavaConverters._ +import org.apache.ranger.authorization.spark.authorizer.SparkPrivObjectActionType.SparkPrivObjectActionType + +class SparkPrivilegeObject( + private val typ: SparkPrivilegeObjectType, + private val dbname: String, + private val objectName: String, + private val partKeys: Seq[String], + private val columns: Seq[String], + private val actionType: SparkPrivObjectActionType) + extends Ordered[SparkPrivilegeObject] { + + override def compare(that: SparkPrivilegeObject): Int = { + typ compareTo that.typ match { + case 0 => + compare(dbname, that.dbname) match { + case 0 => + compare(objectName, that.objectName) match { + case 0 => + compare(partKeys, that.partKeys) match { + case 0 => compare(columns, that.columns) + case o => o + } + case o => o + } + case o => o + } + case o => o + } + } + + private def compare(o1: String, o2: String): Int = { + if (o1 != null) { + if (o2 != null) o1.compareTo(o2) else 1 + } else { + if (o2 != null) -1 else 0 + } + } + + private def compare(o1: Seq[String], o2: Seq[String]): Int = { + if (o1 != null) { + if (o2 != null) { + for ((x, y) <- o1.zip(o2)) { + val ret = compare(x, y) + if (ret != 0) { + return ret + } + } + if (o1.size > o2.size) { + 1 + } else if (o1.size < o2.size) { + -1 + } else { + 0 + } + } else { + 1 + } + } else { + if (o2 != null) { + -1 + } else { + 0 + } + } + } + + def this(typ: SparkPrivilegeObjectType, dbname: String, objectName: String, + partKeys: Seq[String], columns: Seq[String]) = + this(typ, dbname, objectName, partKeys, columns, SparkPrivObjectActionType.OTHER) + + def this(typ: SparkPrivilegeObjectType, dbname: String, objectName: String, + actionType: SparkPrivObjectActionType) = + this(typ, dbname, objectName, Nil, Nil, actionType) + + def this(typ: SparkPrivilegeObjectType, dbname: String, objectName: String) = + this(typ, dbname, objectName, SparkPrivObjectActionType.OTHER) + + def getType: SparkPrivilegeObjectType = typ + + def getDbname: String = dbname + + def getObjectName: String = objectName + + def getActionType: SparkPrivObjectActionType = actionType + + def getPartKeys: Seq[String] = partKeys + + def getColumns: Seq[String] = columns + + override def toString: String = { + val name = typ match { + case SparkPrivilegeObjectType.DATABASE => dbname + case SparkPrivilegeObjectType.TABLE_OR_VIEW => + getDbObjectName + (if (partKeys != null) partKeys.asJava.toString else "") + case SparkPrivilegeObjectType.FUNCTION => getDbObjectName + case _ => "" + } + + val at = if (actionType != null) { + actionType match { + case SparkPrivObjectActionType.INSERT | + SparkPrivObjectActionType.INSERT_OVERWRITE => ", action=" + actionType + case _ => "" + } + } else { + "" + } + "Object [type=" + typ + ", name=" + name + at + "]" + } + + private def getDbObjectName: String = { + (if (dbname == null) "" else dbname + ".") + objectName + } +} diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizerFactory.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkPrivilegeObjectType.scala similarity index 57% rename from plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizerFactory.scala rename to plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkPrivilegeObjectType.scala index 2150cfb5bf..e1b9e70df7 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/RangerSparkAuthorizerFactory.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/SparkPrivilegeObjectType.scala @@ -17,16 +17,7 @@ package org.apache.ranger.authorization.spark.authorizer -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveAuthorizer, HiveAuthorizerFactory, HiveAuthzSessionContext, HiveMetastoreClientFactory} -import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider - -class RangerSparkAuthorizerFactory extends HiveAuthorizerFactory { - override def createHiveAuthorizer( - metastoreClientFactory: HiveMetastoreClientFactory, - conf: HiveConf, - hiveAuthenticator: HiveAuthenticationProvider, - ctx: HiveAuthzSessionContext): HiveAuthorizer = { - new RangerSparkAuthorizer(metastoreClientFactory, conf, hiveAuthenticator, ctx) - } +object SparkPrivilegeObjectType extends Enumeration { + type SparkPrivilegeObjectType = Value + val DATABASE, TABLE_OR_VIEW, FUNCTION, DFS_URI = Value } diff --git a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/authorizer.scala b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/authorizer.scala index 8a85ce07fc..fdf0c850cc 100644 --- a/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/authorizer.scala +++ b/plugin-spark/src/main/scala/org/apache/ranger/authorization/spark/authorizer/authorizer.scala @@ -17,16 +17,10 @@ package org.apache.ranger.authorization.spark -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType import org.apache.spark.sql.SparkSessionExtensions package object authorizer { type Extensions = SparkSessionExtensions => Unit - type SparkPrivilegeObject = HivePrivilegeObject - - type SparkPrivilegeObjectType = HivePrivilegeObjectType - } diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala deleted file mode 100644 index 80c1dd4fc7..0000000000 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Authorizable.scala +++ /dev/null @@ -1,152 +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.spark.sql.catalyst.optimizer - -import org.apache.hadoop.hive.ql.plan.HiveOperation -import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveAuthzContext, HiveOperationType} -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources.{CreateTempViewUsing, InsertIntoDataSourceCommand, InsertIntoHadoopFsRelationCommand} -import org.apache.spark.sql.hive.{HiveExternalCatalog, PrivilegesBuilder} -import org.apache.spark.sql.hive.client.RangerSparkAuthzImpl -import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand - -trait Authorizable extends Rule[LogicalPlan] { - - def spark: SparkSession - - /** - * Visit the [[LogicalPlan]] recursively to get all hive privilege objects, check the privileges - * using Hive Authorizer V2 which provide sql based authorization and can implements - * ranger-hive-plugins. - * If the user is authorized, then the original plan will be returned; otherwise, interrupted by - * some particular privilege exceptions. - * @param plan a spark LogicalPlan for verifying privileges - * @return a plan itself which has gone through the privilege check. - */ - override def apply(plan: LogicalPlan): LogicalPlan = { - val operationType: HiveOperationType = getOperationType(plan) - val authzContext = new HiveAuthzContext.Builder().build() - val (in, out) = PrivilegesBuilder.build(plan) - spark.sharedState.externalCatalog match { - case _: HiveExternalCatalog => - RangerSparkAuthzImpl.checkPrivileges(spark, operationType, in, out, authzContext) - case _ => - } - plan - } - - /** - * Mapping of [[LogicalPlan]] -> [[HiveOperation]] - * @param plan a spark LogicalPlan - * @return - */ - def getHiveOperation(plan: LogicalPlan): HiveOperation = { - plan match { - case c: Command => c match { - case _: AlterDatabasePropertiesCommand => HiveOperation.ALTERDATABASE - case p if p.nodeName == "AlterTableAddColumnsCommand" => HiveOperation.ALTERTABLE_ADDCOLS - case _: AlterTableAddPartitionCommand => HiveOperation.ALTERTABLE_ADDPARTS - case p if p.nodeName == "AlterTableChangeColumnCommand" => - HiveOperation.ALTERTABLE_RENAMECOL - case _: AlterTableDropPartitionCommand => HiveOperation.ALTERTABLE_DROPPARTS - case _: AlterTableRecoverPartitionsCommand => HiveOperation.MSCK - case _: AlterTableRenamePartitionCommand => HiveOperation.ALTERTABLE_RENAMEPART - case a: AlterTableRenameCommand => - if (!a.isView) HiveOperation.ALTERTABLE_RENAME else HiveOperation.ALTERVIEW_RENAME - case _: AlterTableSetPropertiesCommand - | _: AlterTableUnsetPropertiesCommand => HiveOperation.ALTERTABLE_PROPERTIES - case _: AlterTableSerDePropertiesCommand => HiveOperation.ALTERTABLE_SERDEPROPERTIES - case _: AlterTableSetLocationCommand => HiveOperation.ALTERTABLE_LOCATION - case _: AlterViewAsCommand => HiveOperation.QUERY - // case _: AlterViewAsCommand => HiveOperation.ALTERVIEW_AS - - case _: AnalyzeColumnCommand => HiveOperation.QUERY - // case _: AnalyzeTableCommand => HiveOperation.ANALYZE_TABLE - // Hive treat AnalyzeTableCommand as QUERY, obey it. - case _: AnalyzeTableCommand => HiveOperation.QUERY - case p if p.nodeName == "AnalyzePartitionCommand" => HiveOperation.QUERY - - case _: CreateDatabaseCommand => HiveOperation.CREATEDATABASE - case _: CreateDataSourceTableAsSelectCommand - | _: CreateHiveTableAsSelectCommand => HiveOperation.CREATETABLE_AS_SELECT - case _: CreateFunctionCommand => HiveOperation.CREATEFUNCTION - case _: CreateTableCommand - | _: CreateDataSourceTableCommand => HiveOperation.CREATETABLE - case _: CreateTableLikeCommand => HiveOperation.CREATETABLE - case _: CreateViewCommand - | _: CacheTableCommand - | _: CreateTempViewUsing => HiveOperation.CREATEVIEW - - case p if p.nodeName == "DescribeColumnCommand" => HiveOperation.DESCTABLE - case _: DescribeDatabaseCommand => HiveOperation.DESCDATABASE - case _: DescribeFunctionCommand => HiveOperation.DESCFUNCTION - case _: DescribeTableCommand => HiveOperation.DESCTABLE - - case _: DropDatabaseCommand => HiveOperation.DROPDATABASE - // Hive don't check privileges for `drop function command`, what about a unverified user - // try to drop functions. - // We treat permanent functions as tables for verifying. - case d: DropFunctionCommand if !d.isTemp => HiveOperation.DROPTABLE - case d: DropFunctionCommand if d.isTemp => HiveOperation.DROPFUNCTION - case _: DropTableCommand => HiveOperation.DROPTABLE - - case e: ExplainCommand => getHiveOperation(e.logicalPlan) - - case _: InsertIntoDataSourceCommand => HiveOperation.QUERY - case p if p.nodeName == "InsertIntoDataSourceDirCommand" => HiveOperation.QUERY - case _: InsertIntoHadoopFsRelationCommand => HiveOperation.CREATETABLE_AS_SELECT - case p if p.nodeName == "InsertIntoHiveDirCommand" => HiveOperation.QUERY - case p if p.nodeName == "InsertIntoHiveTable" => HiveOperation.QUERY - - case _: LoadDataCommand => HiveOperation.LOAD - - case p if p.nodeName == "SaveIntoDataSourceCommand" => HiveOperation.QUERY - case s: SetCommand if s.kv.isEmpty || s.kv.get._2.isEmpty => HiveOperation.SHOWCONF - case _: SetDatabaseCommand => HiveOperation.SWITCHDATABASE - case _: ShowCreateTableCommand => HiveOperation.SHOW_CREATETABLE - case _: ShowColumnsCommand => HiveOperation.SHOWCOLUMNS - case _: ShowDatabasesCommand => HiveOperation.SHOWDATABASES - case _: ShowFunctionsCommand => HiveOperation.SHOWFUNCTIONS - case _: ShowPartitionsCommand => HiveOperation.SHOWPARTITIONS - case _: ShowTablesCommand => HiveOperation.SHOWTABLES - case _: ShowTablePropertiesCommand => HiveOperation.SHOW_TBLPROPERTIES - case s: StreamingExplainCommand => - getHiveOperation(s.queryExecution.optimizedPlan) - - case _: TruncateTableCommand => HiveOperation.TRUNCATETABLE - - case _: UncacheTableCommand => HiveOperation.DROPVIEW - - // Commands that do not need build privilege goes as explain type - case _ => - // AddFileCommand - // AddJarCommand - // ... - HiveOperation.EXPLAIN - } - case _ => HiveOperation.QUERY - } - } - - def getOperationType(logicalPlan: LogicalPlan): HiveOperationType = { - HiveOperationType.valueOf(getHiveOperation(logicalPlan).name()) - } -} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkAuthorizerExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkAuthorizerExtension.scala index 0563ea3579..da68923a08 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkAuthorizerExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkAuthorizerExtension.scala @@ -17,14 +17,152 @@ package org.apache.spark.sql.catalyst.optimizer +import org.apache.commons.logging.LogFactory +import org.apache.ranger.authorization.spark.authorizer.{RangerSparkAuthorizer, SparkAccessControlException, SparkOperationType} import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.datasources.{CreateTempViewUsing, InsertIntoDataSourceCommand, InsertIntoHadoopFsRelationCommand} +import org.apache.spark.sql.execution.{RangerShowDatabasesCommand, RangerShowTablesCommand} +import org.apache.spark.sql.hive.PrivilegesBuilder +import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand /** * An Optimizer Rule to do Hive Authorization V2 for Spark SQL. * * For Apache Spark 2.2.x and later */ -case class RangerSparkAuthorizerExtension(spark: SparkSession) - extends Rule[LogicalPlan] with Authorizable +case class RangerSparkAuthorizerExtension(spark: SparkSession) extends Rule[LogicalPlan] { + import SparkOperationType._ + + private val LOG = LogFactory.getLog(classOf[RangerSparkAuthorizerExtension]) + + /** + * Visit the [[LogicalPlan]] recursively to get all spark privilege objects, check the privileges + * + * If the user is authorized, then the original plan will be returned; otherwise, interrupted by + * some particular privilege exceptions. + * @param plan a spark LogicalPlan for verifying privileges + * @return a plan itself which has gone through the privilege check. + */ + override def apply(plan: LogicalPlan): LogicalPlan = { + plan match { + case s: ShowTablesCommand => RangerShowTablesCommand(s) + case s: ShowDatabasesCommand => RangerShowDatabasesCommand(s) + case r: RangerShowTablesCommand => r + case r: RangerShowDatabasesCommand => r + case _ => + val operationType: SparkOperationType = toOperationType(plan) + val (in, out) = PrivilegesBuilder.build(plan) + try { + RangerSparkAuthorizer.checkPrivileges(spark, operationType, in, out) + plan + } catch { + case ace: SparkAccessControlException => + LOG.error( + s""" + |+===============================+ + ||Spark SQL Authorization Failure| + ||-------------------------------| + ||${ace.getMessage} + ||-------------------------------| + ||Spark SQL Authorization Failure| + |+===============================+ + """.stripMargin) + throw ace + } + } + } + + /** + * Mapping of [[LogicalPlan]] -> [[SparkOperationType]] + * @param plan a spark LogicalPlan + * @return + */ + private def toOperationType(plan: LogicalPlan): SparkOperationType = { + plan match { + case c: Command => c match { + case _: AlterDatabasePropertiesCommand => ALTERDATABASE + case p if p.nodeName == "AlterTableAddColumnsCommand" => ALTERTABLE_ADDCOLS + case _: AlterTableAddPartitionCommand => ALTERTABLE_ADDPARTS + case p if p.nodeName == "AlterTableChangeColumnCommand" => ALTERTABLE_RENAMECOL + case _: AlterTableDropPartitionCommand => ALTERTABLE_DROPPARTS + case _: AlterTableRecoverPartitionsCommand => MSCK + case _: AlterTableRenamePartitionCommand => ALTERTABLE_RENAMEPART + case a: AlterTableRenameCommand => if (!a.isView) ALTERTABLE_RENAME else ALTERVIEW_RENAME + case _: AlterTableSetPropertiesCommand + | _: AlterTableUnsetPropertiesCommand => ALTERTABLE_PROPERTIES + case _: AlterTableSerDePropertiesCommand => ALTERTABLE_SERDEPROPERTIES + case _: AlterTableSetLocationCommand => ALTERTABLE_LOCATION + case _: AlterViewAsCommand => QUERY + + case _: AnalyzeColumnCommand => QUERY + // case _: AnalyzeTableCommand => HiveOperation.ANALYZE_TABLE + // Hive treat AnalyzeTableCommand as QUERY, obey it. + case _: AnalyzeTableCommand => QUERY + case p if p.nodeName == "AnalyzePartitionCommand" => QUERY + + case _: CreateDatabaseCommand => CREATEDATABASE + case _: CreateDataSourceTableAsSelectCommand + | _: CreateHiveTableAsSelectCommand => CREATETABLE_AS_SELECT + case _: CreateFunctionCommand => CREATEFUNCTION + case _: CreateTableCommand + | _: CreateDataSourceTableCommand => CREATETABLE + case _: CreateTableLikeCommand => CREATETABLE + case _: CreateViewCommand + | _: CacheTableCommand + | _: CreateTempViewUsing => CREATEVIEW + + case p if p.nodeName == "DescribeColumnCommand" => DESCTABLE + case _: DescribeDatabaseCommand => DESCDATABASE + case _: DescribeFunctionCommand => DESCFUNCTION + case _: DescribeTableCommand => DESCTABLE + + case _: DropDatabaseCommand => DROPDATABASE + // Hive don't check privileges for `drop function command`, what about a unverified user + // try to drop functions. + // We treat permanent functions as tables for verifying. + case d: DropFunctionCommand if !d.isTemp => DROPTABLE + case d: DropFunctionCommand if d.isTemp => DROPFUNCTION + case _: DropTableCommand => DROPTABLE + + case e: ExplainCommand => toOperationType(e.logicalPlan) + + case _: InsertIntoDataSourceCommand => QUERY + case p if p.nodeName == "InsertIntoDataSourceDirCommand" => QUERY + case _: InsertIntoHadoopFsRelationCommand => CREATETABLE_AS_SELECT + case p if p.nodeName == "InsertIntoHiveDirCommand" => QUERY + case p if p.nodeName == "InsertIntoHiveTable" => QUERY + + case _: LoadDataCommand => LOAD + + case p if p.nodeName == "SaveIntoDataSourceCommand" => QUERY + case s: SetCommand if s.kv.isEmpty || s.kv.get._2.isEmpty => SHOWCONF + case _: SetDatabaseCommand => SWITCHDATABASE + case _: ShowCreateTableCommand => SHOW_CREATETABLE + case _: ShowColumnsCommand => SHOWCOLUMNS + case _: ShowDatabasesCommand => SHOWDATABASES + case _: ShowFunctionsCommand => SHOWFUNCTIONS + case _: ShowPartitionsCommand => SHOWPARTITIONS + case _: ShowTablesCommand => SHOWTABLES + case _: ShowTablePropertiesCommand => SHOW_TBLPROPERTIES + case s: StreamingExplainCommand => + toOperationType(s.queryExecution.optimizedPlan) + + case _: TruncateTableCommand => TRUNCATETABLE + + case _: UncacheTableCommand => DROPVIEW + + // Commands that do not need build privilege goes as explain type + case _ => + // AddFileCommand + // AddJarCommand + // ... + EXPLAIN + } + case _ => QUERY + } + } + +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala index b026af84d0..b758e3bb63 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtension.scala @@ -71,7 +71,7 @@ case class RangerSparkMaskingExtension(spark: SparkSession) extends Rule[Logical val auditHandler = new RangerSparkAuditHandler() val ugi = UserGroupInformation.getCurrentUser val userName = ugi.getShortUserName - val groups = ugi.getGroupNames.toSet.asJava + val groups = ugi.getGroupNames.toSet try { val identifier = table.identifier import SparkObjectType._ @@ -79,7 +79,7 @@ case class RangerSparkMaskingExtension(spark: SparkSession) extends Rule[Logical val maskEnableResults = plan.output.map { expr => val resource = RangerSparkResource(COLUMN, identifier.database, identifier.table, expr.name) val req = new RangerSparkAccessRequest(resource, userName, groups, COLUMN.toString, - SparkAccessType.SELECT, null, null, sparkPlugin.getClusterName) + SparkAccessType.SELECT, sparkPlugin.getClusterName) (expr, sparkPlugin.evalDataMaskPolicies(req, auditHandler)) }.filter(x => isMaskEnabled(x._2)) diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala index 0d4b4dc835..7edc9455d0 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkRowFilterExtension.scala @@ -54,8 +54,8 @@ case class RangerSparkRowFilterExtension(spark: SparkSession) extends Rule[Logic RangerSparkResource(SparkObjectType.TABLE, identifier.database, identifier.table) val ugi = UserGroupInformation.getCurrentUser val request = new RangerSparkAccessRequest(resource, ugi.getShortUserName, - ugi.getGroupNames.toSet.asJava, SparkObjectType.TABLE.toString, SparkAccessType.SELECT, - null, null, sparkPlugin.getClusterName) + ugi.getGroupNames.toSet, SparkObjectType.TABLE.toString, SparkAccessType.SELECT, + sparkPlugin.getClusterName) val result = sparkPlugin.evalRowFilterPolicies(request, auditHandler) if (isRowFilterEnabled(result)) { val sql = s"select ${plan.output.map(_.name).mkString(",")} from ${table.qualifiedName}" + diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SparkPrivilegeObject.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SparkPrivilegeObject.scala deleted file mode 100644 index f1c7565b4f..0000000000 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SparkPrivilegeObject.scala +++ /dev/null @@ -1,76 +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.spark.sql.catalyst.optimizer - -import java.util.{List => JList} - -import org.apache.hadoop.hive.ql.security.authorization.plugin.{HivePrivilegeObject => HBO} -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.{HivePrivilegeObjectType, HivePrivObjectActionType} - -/** - * Helper class for initializing [[HBO]] with more Constructors. - */ -private[sql] object SparkPrivilegeObject { - def apply( - `type`: HivePrivilegeObjectType, - dbname: String, - objectName: String, - partKeys: JList[String], - columns: JList[String], - actionType: HivePrivObjectActionType, - commandParams: JList[String]): HBO = { - new HBO( - `type`, dbname, objectName, partKeys, columns, actionType, commandParams) - } - - def apply( - `type`: HivePrivilegeObjectType, - dbname: String, - objectName: String, - partKeys: JList[String], - columns: JList[String], - commandParams: JList[String]): HBO = { - apply( - `type`, dbname, objectName, partKeys, columns, HivePrivObjectActionType.OTHER, commandParams) - } - - def apply( - `type`: HivePrivilegeObjectType, - dbname: String, - objectName: String, - partKeys: JList[String], - columns: JList[String]): HBO = { - apply( - `type`, dbname, objectName, partKeys, columns, HivePrivObjectActionType.OTHER, null) - } - - def apply( - `type`: HivePrivilegeObjectType, - dbname: String, - objectName: String, - actionType: HivePrivObjectActionType): HBO = { - apply(`type`, dbname, objectName, null, null, actionType, null) - } - - def apply( - `type`: HivePrivilegeObjectType, - dbname: String, - objectName: String): HBO = { - apply(`type`, dbname, objectName, HivePrivObjectActionType.OTHER) - } -} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerShowDatabasesCommand.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerShowDatabasesCommand.scala new file mode 100644 index 0000000000..1cde50b3e7 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerShowDatabasesCommand.scala @@ -0,0 +1,38 @@ +/* + * 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 + +import org.apache.ranger.authorization.spark.authorizer.{RangerSparkAuthorizer, SparkPrivilegeObject, SparkPrivilegeObjectType} +import org.apache.spark.sql.execution.command.{RunnableCommand, ShowDatabasesCommand} +import org.apache.spark.sql.{Row, SparkSession} + +case class RangerShowDatabasesCommand(child: ShowDatabasesCommand) extends RunnableCommand { + override val output = child.output + + override def run(sparkSession: SparkSession): Seq[Row] = { + val rows = child.run(sparkSession) + rows.filter(r => RangerSparkAuthorizer.isAllowed(toSparkPrivilegeObject(r))) + } + + private def toSparkPrivilegeObject(row: Row): SparkPrivilegeObject = { + val database = row.getString(0) + new SparkPrivilegeObject(SparkPrivilegeObjectType.DATABASE, database, database) + } + + +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerShowTablesCommand.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerShowTablesCommand.scala new file mode 100644 index 0000000000..1c31fe32e4 --- /dev/null +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/execution/RangerShowTablesCommand.scala @@ -0,0 +1,38 @@ +/* + * 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 + +import org.apache.ranger.authorization.spark.authorizer.{RangerSparkAuthorizer, SparkPrivilegeObject, SparkPrivilegeObjectType} +import org.apache.spark.sql.execution.command.{RunnableCommand, ShowTablesCommand} +import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.sql.catalyst.expressions.Attribute + +case class RangerShowTablesCommand(child: ShowTablesCommand) extends RunnableCommand { + + override val output: Seq[Attribute] = child.output + override def run(sparkSession: SparkSession): Seq[Row] = { + val rows = child.run(sparkSession) + rows.filter(r => RangerSparkAuthorizer.isAllowed(toSparkPrivilegeObject(r))) + } + + private def toSparkPrivilegeObject(row: Row): SparkPrivilegeObject = { + val database = row.getString(0) + val table = row.getString(1) + new SparkPrivilegeObject(SparkPrivilegeObjectType.TABLE_OR_VIEW, database, table) + } +} diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala index 5c82e5addc..fc20a24fc5 100644 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala +++ b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/PrivilegesBuilder.scala @@ -17,19 +17,14 @@ package org.apache.spark.sql.hive -import java.util.{ArrayList => JAList, List => JList} - -import scala.collection.JavaConverters._ - -import org.apache.hadoop.hive.ql.security.authorization.plugin.{HivePrivilegeObject => HPO} -import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.{HivePrivilegeObjectType, HivePrivObjectActionType} +import org.apache.ranger.authorization.spark.authorizer.{SparkPrivilegeObject, SparkPrivilegeObjectType, SparkPrivObjectActionType} +import org.apache.ranger.authorization.spark.authorizer.SparkPrivObjectActionType.SparkPrivObjectActionType import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.NamedExpression -import org.apache.spark.sql.catalyst.optimizer.SparkPrivilegeObject import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources._ @@ -37,6 +32,8 @@ import org.apache.spark.sql.AuthzUtils._ import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand import org.apache.spark.sql.types.StructField +import scala.collection.mutable.ArrayBuffer + /** * [[LogicalPlan]] -> list of [[SparkPrivilegeObject]]s */ @@ -53,11 +50,11 @@ private[sql] object PrivilegesBuilder { * * @param plan A Spark [[LogicalPlan]] */ - def build(plan: LogicalPlan): (JList[HPO], JList[HPO]) = { + def build(plan: LogicalPlan): (Seq[SparkPrivilegeObject], Seq[SparkPrivilegeObject]) = { - def doBuild(plan: LogicalPlan): (JList[HPO], JList[HPO]) = { - val inputObjs = new JAList[HPO] - val outputObjs = new JAList[HPO] + def doBuild(plan: LogicalPlan): (Seq[SparkPrivilegeObject], Seq[SparkPrivilegeObject]) = { + val inputObjs = new ArrayBuffer[SparkPrivilegeObject] + val outputObjs = new ArrayBuffer[SparkPrivilegeObject] plan match { // RunnableCommand case cmd: Command => buildCommand(cmd, inputObjs, outputObjs) @@ -74,14 +71,14 @@ private[sql] object PrivilegesBuilder { } /** - * Build HivePrivilegeObjects from Spark LogicalPlan - * @param plan a Spark LogicalPlan used to generate HivePrivilegeObjects - * @param hivePrivilegeObjects input or output hive privilege object list + * Build SparkPrivilegeObjects from Spark LogicalPlan + * @param plan a Spark LogicalPlan used to generate SparkPrivilegeObjects + * @param privilegeObjects input or output spark privilege object list * @param projectionList Projection list after pruning */ private def buildQuery( plan: LogicalPlan, - hivePrivilegeObjects: JList[HPO], + privilegeObjects: ArrayBuffer[SparkPrivilegeObject], projectionList: Seq[NamedExpression] = Nil): Unit = { /** @@ -92,20 +89,20 @@ private[sql] object PrivilegesBuilder { if (projectionList.isEmpty) { addTableOrViewLevelObjs( table.identifier, - hivePrivilegeObjects, + privilegeObjects, table.partitionColumnNames, table.schema.fieldNames) } else { addTableOrViewLevelObjs( table.identifier, - hivePrivilegeObjects, + privilegeObjects, table.partitionColumnNames.filter(projectionList.map(_.name).contains(_)), projectionList.map(_.name)) } } plan match { - case p: Project => buildQuery(p.child, hivePrivilegeObjects, p.projectList) + case p: Project => buildQuery(p.child, privilegeObjects, p.projectList) case h if h.nodeName == "HiveTableRelation" => mergeProjection(getFieldVal(h, "tableMeta").asInstanceOf[CatalogTable]) @@ -120,25 +117,25 @@ private[sql] object PrivilegesBuilder { // Unfortunately, the real world is always a place where miracles happen. // We check the privileges directly without resolving the plan and leave everything // to spark to do. - addTableOrViewLevelObjs(u.tableIdentifier, hivePrivilegeObjects) + addTableOrViewLevelObjs(u.tableIdentifier, privilegeObjects) case p => for (child <- p.children) { - buildQuery(child, hivePrivilegeObjects, projectionList) + buildQuery(child, privilegeObjects, projectionList) } } } /** - * Build HivePrivilegeObjects from Spark LogicalPlan - * @param plan a Spark LogicalPlan used to generate HivePrivilegeObjects - * @param inputObjs input hive privilege object list - * @param outputObjs output hive privilege object list + * Build SparkPrivilegeObjects from Spark LogicalPlan + * @param plan a Spark LogicalPlan used to generate SparkPrivilegeObjects + * @param inputObjs input spark privilege object list + * @param outputObjs output spark privilege object list */ private def buildCommand( plan: LogicalPlan, - inputObjs: JList[HPO], - outputObjs: JList[HPO]): Unit = { + inputObjs: ArrayBuffer[SparkPrivilegeObject], + outputObjs: ArrayBuffer[SparkPrivilegeObject]): Unit = { plan match { case a: AlterDatabasePropertiesCommand => addDbLevelObjs(a.databaseName, outputObjs) @@ -320,7 +317,11 @@ private[sql] object PrivilegesBuilder { getFieldVal(i, "table").asInstanceOf[CatalogTable].identifier, outputObjs) buildQuery(getFieldVal(i, "query").asInstanceOf[LogicalPlan], inputObjs) - case l: LoadDataCommand => addTableOrViewLevelObjs(l.table, outputObjs) + case l: LoadDataCommand => + addTableOrViewLevelObjs(l.table, outputObjs) + if (!l.isLocal) { + inputObjs += new SparkPrivilegeObject(SparkPrivilegeObjectType.DFS_URI, l.path, l.path) + } case s if s.nodeName == "SaveIntoDataSourceCommand" => buildQuery(getFieldVal(s, "query").asInstanceOf[LogicalPlan], outputObjs) @@ -360,109 +361,99 @@ private[sql] object PrivilegesBuilder { } /** - * Add database level hive privilege objects to input or output list - * @param dbName database name as hive privilege object - * @param hivePrivilegeObjects input or output list + * Add database level spark privilege objects to input or output list + * @param dbName database name as spark privilege object + * @param privilegeObjects input or output list */ private def addDbLevelObjs( dbName: String, - hivePrivilegeObjects: JList[HPO]): Unit = { - hivePrivilegeObjects.add( - SparkPrivilegeObject(HivePrivilegeObjectType.DATABASE, dbName, dbName)) + privilegeObjects: ArrayBuffer[SparkPrivilegeObject]): Unit = { + privilegeObjects += new SparkPrivilegeObject(SparkPrivilegeObjectType.DATABASE, dbName, dbName) } /** - * Add database level hive privilege objects to input or output list - * @param dbOption an option of database name as hive privilege object - * @param hivePrivilegeObjects input or output hive privilege object list + * Add database level spark privilege objects to input or output list + * @param dbOption an option of database name as spark privilege object + * @param privilegeObjects input or output spark privilege object list */ private def addDbLevelObjs( dbOption: Option[String], - hivePrivilegeObjects: JList[HPO]): Unit = { + privilegeObjects: ArrayBuffer[SparkPrivilegeObject]): Unit = { dbOption match { case Some(db) => - hivePrivilegeObjects.add( - SparkPrivilegeObject(HivePrivilegeObjectType.DATABASE, db, db)) + privilegeObjects += new SparkPrivilegeObject(SparkPrivilegeObjectType.DATABASE, db, db) case _ => } } /** - * Add database level hive privilege objects to input or output list - * @param tableIdentifier table identifier contains database name as hive privilege object - * @param hivePrivilegeObjects input or output hive privilege object list + * Add database level spark privilege objects to input or output list + * @param identifier table identifier contains database name as hive privilege object + * @param privilegeObjects input or output spark privilege object list */ private def addDbLevelObjs( - tableIdentifier: TableIdentifier, - hivePrivilegeObjects: JList[HPO]): Unit = { - tableIdentifier.database match { + identifier: TableIdentifier, + privilegeObjects: ArrayBuffer[SparkPrivilegeObject]): Unit = { + identifier.database match { case Some(db) => - hivePrivilegeObjects.add( - SparkPrivilegeObject(HivePrivilegeObjectType.DATABASE, db, db)) + privilegeObjects += new SparkPrivilegeObject(SparkPrivilegeObjectType.DATABASE, db, db) case _ => } } /** - * Add table level hive privilege objects to input or output list - * @param tableIdentifier table identifier contains database name, and table name as hive - * privilege object - * @param hivePrivilegeObjects input or output list - * @param mode Append or overwrite + * Add function level spark privilege objects to input or output list + * @param databaseName database name + * @param functionName function name as spark privilege object + * @param privilegeObjects input or output list */ - private def addTableOrViewLevelObjs( - tableIdentifier: TableIdentifier, - hivePrivilegeObjects: JList[HPO], - partKeys: Seq[String] = Nil, - columns: Seq[String] = Nil, - mode: SaveMode = SaveMode.ErrorIfExists, - cmdParams: Seq[String] = Nil): Unit = { - tableIdentifier.database match { + private def addFunctionLevelObjs( + databaseName: Option[String], + functionName: String, + privilegeObjects: ArrayBuffer[SparkPrivilegeObject]): Unit = { + databaseName match { case Some(db) => - val tbName = tableIdentifier.table - val hivePrivObjectActionType = getHivePrivObjActionType(mode) - hivePrivilegeObjects.add( - SparkPrivilegeObject( - HivePrivilegeObjectType.TABLE_OR_VIEW, - db, - tbName, - partKeys.asJava, - columns.asJava, - hivePrivObjectActionType, - cmdParams.asJava)) + privilegeObjects += new SparkPrivilegeObject( + SparkPrivilegeObjectType.FUNCTION, db, functionName) case _ => } } /** - * Add function level hive privilege objects to input or output list - * @param databaseName database name - * @param functionName function name as hive privilege object - * @param hivePrivilegeObjects input or output list + * Add table level spark privilege objects to input or output list + * @param identifier table identifier contains database name, and table name as hive + * privilege object + * @param privilegeObjects input or output list + * @param mode Append or overwrite */ - private def addFunctionLevelObjs( - databaseName: Option[String], - functionName: String, - hivePrivilegeObjects: JList[HPO]): Unit = { - databaseName match { + private def addTableOrViewLevelObjs(identifier: TableIdentifier, + privilegeObjects: ArrayBuffer[SparkPrivilegeObject], partKeys: Seq[String] = Nil, + columns: Seq[String] = Nil, mode: SaveMode = SaveMode.ErrorIfExists): Unit = { + identifier.database match { case Some(db) => - hivePrivilegeObjects.add( - SparkPrivilegeObject(HivePrivilegeObjectType.FUNCTION, db, functionName)) + val tbName = identifier.table + val actionType = toActionType(mode) + privilegeObjects += new SparkPrivilegeObject( + SparkPrivilegeObjectType.TABLE_OR_VIEW, + db, + tbName, + partKeys, + columns, + actionType) case _ => } } /** - * HivePrivObjectActionType INSERT or INSERT_OVERWRITE + * SparkPrivObjectActionType INSERT or INSERT_OVERWRITE * * @param mode Append or Overwrite - * @return */ - private def getHivePrivObjActionType(mode: SaveMode): HivePrivObjectActionType = { + private def toActionType(mode: SaveMode): SparkPrivObjectActionType = { mode match { - case SaveMode.Append => HivePrivObjectActionType.INSERT - case SaveMode.Overwrite => HivePrivObjectActionType.INSERT_OVERWRITE - case _ => HivePrivObjectActionType.OTHER + case SaveMode.Append => SparkPrivObjectActionType.INSERT + case SaveMode.Overwrite => SparkPrivObjectActionType.INSERT_OVERWRITE + case _ => SparkPrivObjectActionType.OTHER } } } diff --git a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/RangerSparkAuthzImpl.scala b/plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/RangerSparkAuthzImpl.scala deleted file mode 100644 index 6eeaf725f6..0000000000 --- a/plugin-spark/src/main/scala/org/apache/spark/sql/hive/client/RangerSparkAuthzImpl.scala +++ /dev/null @@ -1,110 +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.spark.sql.hive.client - -import java.util.{List => JList} -import org.apache.commons.logging.LogFactory -import org.apache.hadoop.hive.ql.security.authorization.plugin._ -import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.security.UserGroupInformation -import org.apache.spark.sql.{AuthzUtils, SparkSession} -import org.apache.spark.sql.hive.HiveExternalCatalog -import org.apache.spark.sql.internal.NonClosableMutableURLClassLoader - -/** - * A Tool for Authorizer implementation. - * - * The [[SessionState]] generates the authorizer and authenticator, we use these to check - * the privileges of a Spark LogicalPlan, which is mapped to hive privilege objects and operation - * type. - * - * [[SparkSession]] with hive catalog implemented has its own instance of [[SessionState]]. I am - * strongly willing to reuse it, but for the reason that it belongs to an isolated classloader - * which makes it unreachable for us to visit it in Spark's context classloader. So, when - * [[ClassCastException]] occurs, we turn off [[IsolatedClientLoader]] to use Spark's builtin - * Hive client jars to generate a new metastore client to replace the original one, once it is - * generated, will be reused then. - * - */ -private[sql] object RangerSparkAuthzImpl { - - private val logger = LogFactory.getLog(getClass.getSimpleName.stripSuffix("$")) - - def checkPrivileges( - spark: SparkSession, - hiveOpType: HiveOperationType, - inputObjs: JList[HivePrivilegeObject], - outputObjs: JList[HivePrivilegeObject], - context: HiveAuthzContext): Unit = { - val client = spark.sharedState - .externalCatalog.asInstanceOf[HiveExternalCatalog] - .client - val clientImpl = try { - client.asInstanceOf[HiveClientImpl] - } catch { - case _: ClassCastException => - val clientLoader = - AuthzUtils.getFieldVal(client, "clientLoader").asInstanceOf[IsolatedClientLoader] - AuthzUtils.setFieldVal(clientLoader, "isolationOn", false) - AuthzUtils.setFieldVal(clientLoader, - "classLoader", new NonClosableMutableURLClassLoader(clientLoader.baseClassLoader)) - clientLoader.cachedHive = null - val newClient = clientLoader.createClient() - AuthzUtils.setFieldVal( - spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog], - "client", - newClient) - newClient.asInstanceOf[HiveClientImpl] - } - - val state = clientImpl.state - SessionState.setCurrentSessionState(state) - val user = UserGroupInformation.getCurrentUser.getShortUserName - if (state.getAuthenticator.getUserName != user) { - val hiveConf = state.getConf - val newState = new SessionState(hiveConf, user) - SessionState.start(newState) - AuthzUtils.setFieldVal(clientImpl, "state", newState) - } - - val authz = clientImpl.state.getAuthorizerV2 - clientImpl.withHiveState { - if (authz != null) { - try { - authz.checkPrivileges(hiveOpType, inputObjs, outputObjs, context) - } catch { - case hae: HiveAccessControlException => - logger.error( - s""" - |+===============================+ - ||Spark SQL Authorization Failure| - ||-------------------------------| - ||${hae.getMessage} - ||-------------------------------| - ||Spark SQL Authorization Failure| - |+===============================+ - """.stripMargin) - throw hae - case e: Exception => throw e - } - } else { - logger.warn("Authorizer V2 not configured. Skipping privilege checking") - } - } - } -} diff --git a/plugin-spark/src/test/resources/sparkSql_hive_jenkins.json b/plugin-spark/src/test/resources/sparkSql_hive_jenkins.json index 6d5e6005c4..6691216d7c 100644 --- a/plugin-spark/src/test/resources/sparkSql_hive_jenkins.json +++ b/plugin-spark/src/test/resources/sparkSql_hive_jenkins.json @@ -403,9 +403,7 @@ "users": [ "bob" ], - "groups": [ - "public" - ], + "groups": [], "conditions": [], "delegateAdmin": false } @@ -425,6 +423,103 @@ "isEnabled": true, "version": 5 }, + { + "service": "hive_jenkins", + "name": "default_kent", + "policyType": 0, + "policyPriority": 0, + "description": "", + "isAuditEnabled": true, + "resources": { + "database": { + "values": [ + "default" + ], + "isExcludes": false, + "isRecursive": false + }, + "column": { + "values": [ + "key" + ], + "isExcludes": false, + "isRecursive": false + }, + "table": { + "values": [ + "src" + ], + "isExcludes": false, + "isRecursive": false + } + }, + "policyItems": [ + { + "accesses": [ + { + "type": "select", + "isAllowed": true + }, + { + "type": "update", + "isAllowed": true + }, + { + "type": "create", + "isAllowed": true + }, + { + "type": "drop", + "isAllowed": true + }, + { + "type": "alter", + "isAllowed": true + }, + { + "type": "index", + "isAllowed": true + }, + { + "type": "lock", + "isAllowed": true + }, + { + "type": "all", + "isAllowed": true + }, + { + "type": "read", + "isAllowed": true + }, + { + "type": "write", + "isAllowed": true + } + ], + "users": [ + "kent" + ], + "groups": [], + "conditions": [], + "delegateAdmin": false + } + ], + "denyPolicyItems": [], + "allowExceptions": [], + "denyExceptions": [], + "dataMaskPolicyItems": [], + "rowFilterPolicyItems": [], + "options": {}, + "validitySchedules": [], + "policyLabels": [ + "" + ], + "id": 5, + "guid": "fd24db19-f7cc-4e13-a8ba-bbd5a07a2d8d", + "isEnabled": true, + "version": 5 + }, { "service": "hive_jenkins", "name": "src_val_show_last_4", diff --git a/plugin-spark/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkAuthorizerExtensionTest.scala b/plugin-spark/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkAuthorizerExtensionTest.scala new file mode 100644 index 0000000000..a650a10835 --- /dev/null +++ b/plugin-spark/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkAuthorizerExtensionTest.scala @@ -0,0 +1,100 @@ +/* + * 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.catalyst.optimizer + +import org.apache.ranger.authorization.spark.authorizer.SparkAccessControlException +import org.apache.spark.sql.hive.test.TestHive +import org.scalatest.FunSuite +import org.apache.spark.sql.RangerSparkTestUtils._ +import org.apache.spark.sql.execution.{RangerShowDatabasesCommand, RangerShowTablesCommand} + +class RangerSparkAuthorizerExtensionTest extends FunSuite { + private val spark = TestHive.sparkSession + val extension = RangerSparkAuthorizerExtension(spark) + + test("convert show tables command") { + val df = spark.sql("show tables") + val plan = df.queryExecution.optimizedPlan + val newPlan = extension.apply(plan) + assert(newPlan.isInstanceOf[RangerShowTablesCommand]) + assert(extension.apply(newPlan) === newPlan) + } + + test("convert show databases command") { + val df = spark.sql("show databases") + val plan = df.queryExecution.optimizedPlan + val newPlan = extension.apply(plan) + assert(newPlan.isInstanceOf[RangerShowDatabasesCommand]) + assert(extension.apply(newPlan) === newPlan) + } + + test("simple select") { + val df = spark.sql("select * from src") + val plan = df.queryExecution.optimizedPlan + withUser("bob") { + assert(extension.apply(plan) === plan, "bob has all privileges of table src") + } + withUser("alice") { + val e = intercept[SparkAccessControlException](extension.apply(plan)) + assert(e.getMessage === "Permission denied: user [alice] does not have [SELECT] privilege" + + " on [default/src/key,value]", "alice is not allow to access table src") + } + withUser("kent") { + val e = intercept[SparkAccessControlException](extension.apply(plan)) + assert(e.getMessage === "Permission denied: user [kent] does not have [SELECT] privilege" + + " on [default/src/key,value]", "kent can only access table src.key") + } + } + + test("projection select") { + val df1 = spark.sql("select key from src") + val df2 = spark.sql("select value from src") + + val plan1 = df1.queryExecution.optimizedPlan + val plan2 = df2.queryExecution.optimizedPlan + + withUser("bob") { + assert(extension.apply(plan1) === plan1, "bob has all privileges of table src") + assert(extension.apply(plan2) === plan2, "bob has all privileges of table src") + } + withUser("alice") { + val e = intercept[SparkAccessControlException](extension.apply(plan1)) + assert(e.getMessage === "Permission denied: user [alice] does not have [SELECT] privilege" + + " on [default/src/key]", "alice is not allow to access table src") + } + withUser("kent") { + assert(extension.apply(plan1) === plan1, "kent can only access table src.key") + val e = intercept[SparkAccessControlException](extension.apply(plan2)) + assert(e.getMessage === "Permission denied: user [kent] does not have [SELECT] privilege" + + " on [default/src/value]", "kent can only access table src.key") + } + } + + test("alter database set properties") { + val df = spark.sql("ALTER DATABASE default SET DBPROPERTIES (hero='i am iron man')") + val plan = df.queryExecution.optimizedPlan + withUser("bob") { + assert(extension.apply(plan) === plan) + } + withUser("alice") { + val e = intercept[SparkAccessControlException](extension.apply(plan)) + assert(e.getMessage === "Permission denied: user [alice] does not have [ALTER] privilege" + + " on [default]", "alice is not allow to set properties to default") + } + } +} diff --git a/plugin-spark/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtensionTest.scala b/plugin-spark/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtensionTest.scala new file mode 100644 index 0000000000..ce4c574183 --- /dev/null +++ b/plugin-spark/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RangerSparkMaskingExtensionTest.scala @@ -0,0 +1,52 @@ +/* + * 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.catalyst.optimizer + +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.RangerSparkTestUtils._ +import org.apache.spark.sql.catalyst.expressions.Alias +import org.apache.spark.sql.catalyst.plans.logical.{Project, RangerSparkMasking} +import org.scalatest.FunSuite + +class RangerSparkMaskingExtensionTest extends FunSuite { + + private val spark = TestHive.sparkSession + + test("data masking for bob show last 4") { + val extension = RangerSparkMaskingExtension(spark) + val plan = spark.sql("select * from src").queryExecution.optimizedPlan + println(plan) + withUser("bob") { + val newPlan = extension.apply(plan) + assert(newPlan.isInstanceOf[Project]) + val project = newPlan.asInstanceOf[Project] + val key = project.projectList.head + assert(key.name === "key", "no affect on un masking attribute") + val value = project.projectList.tail + assert(value.head.name === "value", "attibute name should be unchanged") + assert(value.head.asInstanceOf[Alias].child.sql === + "mask_show_last_n(`value`, 4, 'x', 'x', 'x', -1, '1')") + } + + withUser("alice") { + val newPlan = extension.apply(plan) + assert(newPlan === RangerSparkMasking(plan)) + } + } + +} diff --git a/plugin-spark/src/test/scala/org/apache/spark/sql/execution/RangerSparkPlanOmitStrategyTest.scala b/plugin-spark/src/test/scala/org/apache/spark/sql/execution/RangerSparkPlanOmitStrategyTest.scala index dea926accd..f5c11bd6f6 100644 --- a/plugin-spark/src/test/scala/org/apache/spark/sql/execution/RangerSparkPlanOmitStrategyTest.scala +++ b/plugin-spark/src/test/scala/org/apache/spark/sql/execution/RangerSparkPlanOmitStrategyTest.scala @@ -39,6 +39,4 @@ class RangerSparkPlanOmitStrategyTest extends FunSuite { val plan5 = RangerSparkRowFilter(plan3) assert(strategy.apply(plan5) === PlanLater(plan3) :: Nil) } - - } From 5e6c3f56140b5dbce97bc377e1b27f5a19c3771a Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sat, 18 May 2019 23:06:15 +0800 Subject: [PATCH 34/35] add profiles --- plugin-spark/pom.xml | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/plugin-spark/pom.xml b/plugin-spark/pom.xml index 87a7859f01..7b9130323d 100644 --- a/plugin-spark/pom.xml +++ b/plugin-spark/pom.xml @@ -382,4 +382,24 @@ + + + + + spark-2.3 + + 2.3.2 + 3.0.3 + + + + + spark-2.4 + + 2.4.0 + 3.0.3 + + + + From 5107aeca14b1c079b248c2d93b4c12f3d79c0d84 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Sat, 18 May 2019 23:11:04 +0800 Subject: [PATCH 35/35] typo --- pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/pom.xml b/pom.xml index 14b53082cb..3dcee01484 100644 --- a/pom.xml +++ b/pom.xml @@ -170,7 +170,6 @@ 7.0.91 1.7 3.4.6 - 0.6.3