You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@kyuubi.apache.org by "zml1206 (via GitHub)" <gi...@apache.org> on 2023/10/26 06:45:02 UTC

[PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

zml1206 opened a new pull request, #5530:
URL: https://github.com/apache/kyuubi/pull/5530

   
   
   ### _Why are the changes needed?_
   To close #5529.
   Support create table command for Delta Lake in Authz. 
   https://docs.delta.io/latest/delta-batch.html#create-a-table
   
   
   ### _How was this patch tested?_
   - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible
   
   - [ ] Add screenshots for manual tests if appropriate
   
   - [x] [Run test](https://kyuubi.readthedocs.io/en/master/contributing/code/testing.html#running-tests) locally before make a pull request
   
   
   ### _Was this patch authored or co-authored using generative AI tooling?_
   No.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "cfmcgrady (via GitHub)" <gi...@apache.org>.
cfmcgrady commented on PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#issuecomment-1780617807

   > Path-based tables are supported in Authz now. No path based policies (eg. HDFS service def) of Ranger are supported.
   
   Does Delta Lake have an option for the globally disabled path-based table, similar to how `spark.sql.runSQLOnFiles` works? If not, I think we can introduce a configuration in Kyuubi AuthZ to disable this behavior temporarily, prior to the completion of the path-based table feature. WDYT? cc @yaooqinn 
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#issuecomment-1780511658

   Nice! cc @bowenliang123 and @AngersZhuuuu 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "cfmcgrady (via GitHub)" <gi...@apache.org>.
cfmcgrady commented on PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#issuecomment-1780583391

   delta lake also supports creating a table at a path
   
   ```sql
   -- Create or replace table with path
   CREATE OR REPLACE TABLE delta.`/tmp/delta/people10m` (
     id INT,
     firstName STRING,
     middleName STRING,
     lastName STRING,
     gender STRING,
     birthDate TIMESTAMP,
     ssn STRING,
     salary INT
   ) USING DELTA
   ```
   
   Shall we also add UTs for this case?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#discussion_r1378415442


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/DeltaCatalogRangerSparkExtensionSuite.scala:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.kyuubi.plugin.spark.authz.ranger
+
+import org.scalatest.Outcome
+
+import org.apache.kyuubi.Utils
+import org.apache.kyuubi.plugin.spark.authz.AccessControlException
+import org.apache.kyuubi.plugin.spark.authz.RangerTestNamespace._
+import org.apache.kyuubi.plugin.spark.authz.RangerTestUsers._
+import org.apache.kyuubi.tags.DeltaTest
+import org.apache.kyuubi.util.AssertionUtils._
+
+/**
+ * Tests for RangerSparkExtensionSuite on Delta Lake
+ */
+@DeltaTest
+class DeltaCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite {
+  override protected val catalogImpl: String = "hive"
+
+  val namespace1 = deltaNamespace
+  val table1 = "table1_delta"
+  val table2 = "table2_delta"
+
+  override def withFixture(test: NoArgTest): Outcome = {
+    test()
+  }
+
+  override def beforeAll(): Unit = {
+    spark.conf.set(
+      s"spark.sql.catalog.$sparkCatalog",
+      "org.apache.spark.sql.delta.catalog.DeltaCatalog")
+    spark.conf.set(
+      s"spark.sql.catalog.$sparkCatalog.warehouse",
+      Utils.createTempDir("delta-hadoop").toString)
+    super.beforeAll()
+  }
+
+  override def afterAll(): Unit = {
+    super.afterAll()
+    spark.sessionState.catalog.reset()
+    spark.sessionState.conf.clear()
+  }
+
+  test("create table") {
+    withCleanTmpResources(Seq(
+      (s"$namespace1.$table1", "table"),
+      (s"$namespace1.$table2", "table"),
+      (s"$namespace1", "database"))) {
+      doAs(admin, sql(s"CREATE DATABASE IF NOT EXISTS $namespace1"))
+      val createNonPartitionTableSql =
+        s"""
+           |CREATE TABLE IF NOT EXISTS $namespace1.$table1 (
+           |  id INT,
+           |  firstName STRING,
+           |  middleName STRING,
+           |  lastName STRING,
+           |  gender STRING,
+           |  birthDate TIMESTAMP,
+           |  ssn STRING,
+           |  salary INT
+           |) USING DELTA
+           |""".stripMargin
+      interceptContains[AccessControlException] {
+        doAs(someone, sql(createNonPartitionTableSql))
+      }(s"does not have [create] privilege on [$namespace1/$table1]")
+      doAs(admin, createNonPartitionTableSql)
+
+      val createPartitionTableSql =
+        s"""
+           |CREATE TABLE IF NOT EXISTS $namespace1.$table2 (
+           |  id INT,
+           |  firstName STRING,
+           |  middleName STRING,
+           |  lastName STRING,
+           |  gender STRING,
+           |  birthDate TIMESTAMP,
+           |  ssn STRING,
+           |  salary INT
+           |)
+           |USING DELTA
+           |PARTITIONED BY (gender)
+           |""".stripMargin
+      interceptContains[AccessControlException] {
+        doAs(someone, sql(createPartitionTableSql))
+      }(s"does not have [create] privilege on [$namespace1/$table2]")
+      doAs(admin, createPartitionTableSql)
+    }
+  }
+
+  test("create or replace table") {
+    withCleanTmpResources(
+      Seq((s"$namespace1.$table1", "table"), (s"$namespace1", "database"))) {
+      val createOrReplaceTableSql =
+        s"""
+           |CREATE OR REPLACE TABLE $namespace1.$table1 (
+           |  id INT,
+           |  firstName STRING,
+           |  middleName STRING,
+           |  lastName STRING,
+           |  gender STRING,
+           |  birthDate TIMESTAMP,
+           |  ssn STRING,
+           |  salary INT
+           |) USING DELTA
+           |""".stripMargin
+      interceptContains[AccessControlException] {
+        doAs(someone, sql(createOrReplaceTableSql))
+      }(s"does not have [create] privilege on [$namespace1/$table1]")
+      doAs(admin, createOrReplaceTableSql)

Review Comment:
   ditto



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "zml1206 (via GitHub)" <gi...@apache.org>.
zml1206 commented on PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#issuecomment-1780592061

   > LGTM. Would be more clare if we reorder the test cases in the order for the non-partitioned table, the partitioned table, and then the create/replace table. Or just separated into separated ut.
   
   Thanks, Updated.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "zml1206 (via GitHub)" <gi...@apache.org>.
zml1206 commented on PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#issuecomment-1780903165

   > @AngersZhuuuu is currently working one the path/uri authorization
   > 
   > What's the plan of CREATE OR REPLACE TABLE delta.`/tmp/delta/people10m`
   ```
   == Analyzed Logical Plan ==
   ReplaceTable [StructField(id,IntegerType,true)], TableSpec(Map(),Some(DELTA),Map(),None,None,None,false), true
   +- ResolvedDBObjectName org.apache.spark.sql.delta.catalog.DeltaCatalog@2f9addd4, [delta, /tmp/delta/people10m]
   
   == Optimized Logical Plan ==
   CommandResult AtomicReplaceTable org.apache.spark.sql.delta.catalog.DeltaCatalog@2f9addd4, delta.`/tmp/delta/people10m`, [StructField(id,IntegerType,true)], TableSpec(Map(),Some(DELTA),Map(),None,None,None,false), true, org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$$Lambda$1627/2047805101@71b3ba5a
      +- ReplaceTable [StructField(id,IntegerType,true)], TableSpec(Map(),Some(DELTA),Map(),None,None,None,false), true
         +- ResolvedDBObjectName org.apache.spark.sql.delta.catalog.DeltaCatalog@2f9addd4, [delta, /tmp/delta/people10m]
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "cfmcgrady (via GitHub)" <gi...@apache.org>.
cfmcgrady commented on PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#issuecomment-1782524811

   thanks, merging to master(v1.9.0).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#issuecomment-1780561263

   Cool, let's bring Ranger authorisation to Delta in Kyuubi Authz !


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "bowenliang123 (via GitHub)" <gi...@apache.org>.
bowenliang123 commented on PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#issuecomment-1780587500

   Path-based tables are supported in Authz now. No path based policies (eg. HDFS service def) of Ranger are supported.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "cfmcgrady (via GitHub)" <gi...@apache.org>.
cfmcgrady closed pull request #5530: [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake
URL: https://github.com/apache/kyuubi/pull/5530


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "zml1206 (via GitHub)" <gi...@apache.org>.
zml1206 commented on code in PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#discussion_r1378422300


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/DeltaCatalogRangerSparkExtensionSuite.scala:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.kyuubi.plugin.spark.authz.ranger
+
+import org.scalatest.Outcome
+
+import org.apache.kyuubi.Utils
+import org.apache.kyuubi.plugin.spark.authz.AccessControlException
+import org.apache.kyuubi.plugin.spark.authz.RangerTestNamespace._
+import org.apache.kyuubi.plugin.spark.authz.RangerTestUsers._
+import org.apache.kyuubi.tags.DeltaTest
+import org.apache.kyuubi.util.AssertionUtils._
+
+/**
+ * Tests for RangerSparkExtensionSuite on Delta Lake
+ */
+@DeltaTest
+class DeltaCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite {
+  override protected val catalogImpl: String = "hive"
+
+  val namespace1 = deltaNamespace
+  val table1 = "table1_delta"
+  val table2 = "table2_delta"
+
+  override def withFixture(test: NoArgTest): Outcome = {
+    test()
+  }
+
+  override def beforeAll(): Unit = {
+    spark.conf.set(
+      s"spark.sql.catalog.$sparkCatalog",
+      "org.apache.spark.sql.delta.catalog.DeltaCatalog")
+    spark.conf.set(
+      s"spark.sql.catalog.$sparkCatalog.warehouse",
+      Utils.createTempDir("delta-hadoop").toString)
+    super.beforeAll()
+  }
+
+  override def afterAll(): Unit = {
+    super.afterAll()
+    spark.sessionState.catalog.reset()
+    spark.sessionState.conf.clear()
+  }
+
+  test("create table") {
+    withCleanTmpResources(Seq(
+      (s"$namespace1.$table1", "table"),
+      (s"$namespace1.$table2", "table"),
+      (s"$namespace1", "database"))) {
+      doAs(admin, sql(s"CREATE DATABASE IF NOT EXISTS $namespace1"))
+      val createNonPartitionTableSql =
+        s"""
+           |CREATE TABLE IF NOT EXISTS $namespace1.$table1 (
+           |  id INT,
+           |  firstName STRING,
+           |  middleName STRING,
+           |  lastName STRING,
+           |  gender STRING,
+           |  birthDate TIMESTAMP,
+           |  ssn STRING,
+           |  salary INT
+           |) USING DELTA
+           |""".stripMargin
+      interceptContains[AccessControlException] {
+        doAs(someone, sql(createNonPartitionTableSql))
+      }(s"does not have [create] privilege on [$namespace1/$table1]")
+      doAs(admin, createNonPartitionTableSql)

Review Comment:
   Yes, I'll fix it right away, thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on code in PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#discussion_r1378415332


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/DeltaCatalogRangerSparkExtensionSuite.scala:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.kyuubi.plugin.spark.authz.ranger
+
+import org.scalatest.Outcome
+
+import org.apache.kyuubi.Utils
+import org.apache.kyuubi.plugin.spark.authz.AccessControlException
+import org.apache.kyuubi.plugin.spark.authz.RangerTestNamespace._
+import org.apache.kyuubi.plugin.spark.authz.RangerTestUsers._
+import org.apache.kyuubi.tags.DeltaTest
+import org.apache.kyuubi.util.AssertionUtils._
+
+/**
+ * Tests for RangerSparkExtensionSuite on Delta Lake
+ */
+@DeltaTest
+class DeltaCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite {
+  override protected val catalogImpl: String = "hive"
+
+  val namespace1 = deltaNamespace
+  val table1 = "table1_delta"
+  val table2 = "table2_delta"
+
+  override def withFixture(test: NoArgTest): Outcome = {
+    test()
+  }
+
+  override def beforeAll(): Unit = {
+    spark.conf.set(
+      s"spark.sql.catalog.$sparkCatalog",
+      "org.apache.spark.sql.delta.catalog.DeltaCatalog")
+    spark.conf.set(
+      s"spark.sql.catalog.$sparkCatalog.warehouse",
+      Utils.createTempDir("delta-hadoop").toString)
+    super.beforeAll()
+  }
+
+  override def afterAll(): Unit = {
+    super.afterAll()
+    spark.sessionState.catalog.reset()
+    spark.sessionState.conf.clear()
+  }
+
+  test("create table") {
+    withCleanTmpResources(Seq(
+      (s"$namespace1.$table1", "table"),
+      (s"$namespace1.$table2", "table"),
+      (s"$namespace1", "database"))) {
+      doAs(admin, sql(s"CREATE DATABASE IF NOT EXISTS $namespace1"))
+      val createNonPartitionTableSql =
+        s"""
+           |CREATE TABLE IF NOT EXISTS $namespace1.$table1 (
+           |  id INT,
+           |  firstName STRING,
+           |  middleName STRING,
+           |  lastName STRING,
+           |  gender STRING,
+           |  birthDate TIMESTAMP,
+           |  ssn STRING,
+           |  salary INT
+           |) USING DELTA
+           |""".stripMargin
+      interceptContains[AccessControlException] {
+        doAs(someone, sql(createNonPartitionTableSql))
+      }(s"does not have [create] privilege on [$namespace1/$table1]")
+      doAs(admin, createNonPartitionTableSql)

Review Comment:
   Hi @zml1206, this line has an issue that it does not trigger a sql operation.



##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/DeltaCatalogRangerSparkExtensionSuite.scala:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.kyuubi.plugin.spark.authz.ranger
+
+import org.scalatest.Outcome
+
+import org.apache.kyuubi.Utils
+import org.apache.kyuubi.plugin.spark.authz.AccessControlException
+import org.apache.kyuubi.plugin.spark.authz.RangerTestNamespace._
+import org.apache.kyuubi.plugin.spark.authz.RangerTestUsers._
+import org.apache.kyuubi.tags.DeltaTest
+import org.apache.kyuubi.util.AssertionUtils._
+
+/**
+ * Tests for RangerSparkExtensionSuite on Delta Lake
+ */
+@DeltaTest
+class DeltaCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite {
+  override protected val catalogImpl: String = "hive"
+
+  val namespace1 = deltaNamespace
+  val table1 = "table1_delta"
+  val table2 = "table2_delta"
+
+  override def withFixture(test: NoArgTest): Outcome = {
+    test()
+  }
+
+  override def beforeAll(): Unit = {
+    spark.conf.set(
+      s"spark.sql.catalog.$sparkCatalog",
+      "org.apache.spark.sql.delta.catalog.DeltaCatalog")
+    spark.conf.set(
+      s"spark.sql.catalog.$sparkCatalog.warehouse",
+      Utils.createTempDir("delta-hadoop").toString)
+    super.beforeAll()
+  }
+
+  override def afterAll(): Unit = {
+    super.afterAll()
+    spark.sessionState.catalog.reset()
+    spark.sessionState.conf.clear()
+  }
+
+  test("create table") {
+    withCleanTmpResources(Seq(
+      (s"$namespace1.$table1", "table"),
+      (s"$namespace1.$table2", "table"),
+      (s"$namespace1", "database"))) {
+      doAs(admin, sql(s"CREATE DATABASE IF NOT EXISTS $namespace1"))
+      val createNonPartitionTableSql =
+        s"""
+           |CREATE TABLE IF NOT EXISTS $namespace1.$table1 (
+           |  id INT,
+           |  firstName STRING,
+           |  middleName STRING,
+           |  lastName STRING,
+           |  gender STRING,
+           |  birthDate TIMESTAMP,
+           |  ssn STRING,
+           |  salary INT
+           |) USING DELTA
+           |""".stripMargin
+      interceptContains[AccessControlException] {
+        doAs(someone, sql(createNonPartitionTableSql))
+      }(s"does not have [create] privilege on [$namespace1/$table1]")
+      doAs(admin, createNonPartitionTableSql)
+
+      val createPartitionTableSql =
+        s"""
+           |CREATE TABLE IF NOT EXISTS $namespace1.$table2 (
+           |  id INT,
+           |  firstName STRING,
+           |  middleName STRING,
+           |  lastName STRING,
+           |  gender STRING,
+           |  birthDate TIMESTAMP,
+           |  ssn STRING,
+           |  salary INT
+           |)
+           |USING DELTA
+           |PARTITIONED BY (gender)
+           |""".stripMargin
+      interceptContains[AccessControlException] {
+        doAs(someone, sql(createPartitionTableSql))
+      }(s"does not have [create] privilege on [$namespace1/$table2]")
+      doAs(admin, createPartitionTableSql)

Review Comment:
   ditto



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#issuecomment-1780787014

   @AngersZhuuuu is currently working one the path/uri authorization
   
   What's the plan of CREATE OR REPLACE TABLE delta.`/tmp/delta/people10m`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "cfmcgrady (via GitHub)" <gi...@apache.org>.
cfmcgrady commented on PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#issuecomment-1782173545

   > Delta Lake is also controlled by `spark.sql.runSQLOnFiles`.
   
   thank you for your input. @zml1206 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#issuecomment-1782190350

   ## [Codecov](https://app.codecov.io/gh/apache/kyuubi/pull/5530?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#5530](https://app.codecov.io/gh/apache/kyuubi/pull/5530?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (facd8f7) into [master](https://app.codecov.io/gh/apache/kyuubi/commit/5b9290a9141781f5c723722a76284cfb2028be0b?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (5b9290a) will **decrease** coverage by `0.10%`.
   > Report is 2 commits behind head on master.
   > The diff coverage is `n/a`.
   
   > :exclamation: Current head facd8f7 differs from pull request most recent head 10138d8. Consider uploading reports for the commit 10138d8 to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #5530      +/-   ##
   ============================================
   - Coverage     61.42%   61.32%   -0.10%     
     Complexity       23       23              
   ============================================
     Files           598      598              
     Lines         34254    34254              
     Branches       4488     4488              
   ============================================
   - Hits          21039    21007      -32     
   - Misses        11095    11118      +23     
   - Partials       2120     2129       +9     
   ```
   
   
   [see 10 files with indirect coverage changes](https://app.codecov.io/gh/apache/kyuubi/pull/5530/indirect-changes?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "zml1206 (via GitHub)" <gi...@apache.org>.
zml1206 commented on PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#issuecomment-1780858385

   > > Path-based tables are supported in Authz now. No path based policies (eg. HDFS service def) of Ranger are supported.
   > 
   > Does Delta Lake have an option for the globally disabled path-based table, similar to how `spark.sql.runSQLOnFiles` works? If not, I think we can introduce a configuration in Kyuubi AuthZ to disable this behavior temporarily, prior to the completion of the path-based table feature. WDYT? cc @yaooqinn
   
   https://github.com/delta-io/delta/blob/8639c411890a5c77386f04e2282fcf4caa401eff/spark/src/test/scala/org/apache/spark/sql/delta/DeltaDDLUsingPathSuite.scala#L125
    Delta Lake is also controlled by `spark.sql.runSQLOnFiles`.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org


Re: [PR] [KYUUBI #5529][AUTHZ] Support create table command for Delta Lake [kyuubi]

Posted by "zml1206 (via GitHub)" <gi...@apache.org>.
zml1206 commented on code in PR #5530:
URL: https://github.com/apache/kyuubi/pull/5530#discussion_r1378428267


##########
extensions/spark/kyuubi-spark-authz/src/test/scala/org/apache/kyuubi/plugin/spark/authz/ranger/DeltaCatalogRangerSparkExtensionSuite.scala:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.kyuubi.plugin.spark.authz.ranger
+
+import org.scalatest.Outcome
+
+import org.apache.kyuubi.Utils
+import org.apache.kyuubi.plugin.spark.authz.AccessControlException
+import org.apache.kyuubi.plugin.spark.authz.RangerTestNamespace._
+import org.apache.kyuubi.plugin.spark.authz.RangerTestUsers._
+import org.apache.kyuubi.tags.DeltaTest
+import org.apache.kyuubi.util.AssertionUtils._
+
+/**
+ * Tests for RangerSparkExtensionSuite on Delta Lake
+ */
+@DeltaTest
+class DeltaCatalogRangerSparkExtensionSuite extends RangerSparkExtensionSuite {
+  override protected val catalogImpl: String = "hive"
+
+  val namespace1 = deltaNamespace
+  val table1 = "table1_delta"
+  val table2 = "table2_delta"
+
+  override def withFixture(test: NoArgTest): Outcome = {
+    test()
+  }
+
+  override def beforeAll(): Unit = {
+    spark.conf.set(
+      s"spark.sql.catalog.$sparkCatalog",
+      "org.apache.spark.sql.delta.catalog.DeltaCatalog")
+    spark.conf.set(
+      s"spark.sql.catalog.$sparkCatalog.warehouse",
+      Utils.createTempDir("delta-hadoop").toString)
+    super.beforeAll()
+  }
+
+  override def afterAll(): Unit = {
+    super.afterAll()
+    spark.sessionState.catalog.reset()
+    spark.sessionState.conf.clear()
+  }
+
+  test("create table") {
+    withCleanTmpResources(Seq(
+      (s"$namespace1.$table1", "table"),
+      (s"$namespace1.$table2", "table"),
+      (s"$namespace1", "database"))) {
+      doAs(admin, sql(s"CREATE DATABASE IF NOT EXISTS $namespace1"))
+      val createNonPartitionTableSql =
+        s"""
+           |CREATE TABLE IF NOT EXISTS $namespace1.$table1 (
+           |  id INT,
+           |  firstName STRING,
+           |  middleName STRING,
+           |  lastName STRING,
+           |  gender STRING,
+           |  birthDate TIMESTAMP,
+           |  ssn STRING,
+           |  salary INT
+           |) USING DELTA
+           |""".stripMargin
+      interceptContains[AccessControlException] {
+        doAs(someone, sql(createNonPartitionTableSql))
+      }(s"does not have [create] privilege on [$namespace1/$table1]")
+      doAs(admin, createNonPartitionTableSql)

Review Comment:
   Fix in #5597 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org