You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/07/28 22:29:00 UTC

[GitHub] [iceberg] kbendick opened a new pull request, #5377: Spark - Add Spark FunctionCatalog with Basic iceberg_version Function for Testing

kbendick opened a new pull request, #5377:
URL: https://github.com/apache/iceberg/pull/5377

   This PR stems from https://github.com/apache/iceberg/pull/5305 and covers just the `FunctionCatalog`
   
   ## FunctionCatalog
   
   This allows users of `SparkCatalog` and `SparkSessionCatalog` to use functions (such as the `iceberg_version` function added here) without having to register it as a UDF.
    
   All Iceberg functions that we register into the function catalog are accessible when used with an Iceberg spark catalog and:
   
   1. No namespace is referenced - the storage partitioned joins implementation requires this.
   e.g. `my_catalog.iceberg_versio()`. 
   **Note** - Using `iceberg_version` typically does not work, as Spark adds the namespace to the call. Hence the `system` namespace syntax should be preferred.
   2. The `system` namespace is referenced, to match called procedure syntax. 
   **Note** this only works right now with the `SparkCatalog`, as the `SparkSessionCatalog` has logic in Spark [to verify that the namespace exists](https://github.com/apache/iceberg/pull/5305#discussion_r927178316).
    e.g. `my_catalog.system.iceberg_version()` or `system.iceberg_version()` if already having called `USE my_catalog`.
   
   ### iceberg_version function
   This also adds a simple function `iceberg_version`, which simply returns the (short) version string. This is mostly for testing but will be useful on its own.


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933474624


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunctionImpl.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * {@link org.apache.spark.sql.connector.catalog.functions.BoundFunction} that implements {@link
+ * org.apache.iceberg.spark.functions.IcebergVersionFunction} by returning the short version string,
+ * e.g. "0.14.0" or "0.15.0-SNAPSHOT".
+ */
+public class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {
+
+  private static final String icebergVersionAsString = IcebergBuild.version();
+  private static final UTF8String icebergVersionAsNative =

Review Comment:
   This variable name seems odd to me. I think you're using "as native" to mean Spark's representation, but I wouldn't think of that when hearing the term "native" in that context. How about calling this `versionAsSparkString`?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933565757


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** A function that gets registered to return the current Iceberg version in use from within SQL */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format("Cannot bind: %s does not accept arguments", name()));
+    }
+
+    return new IcebergVersionFunctionImpl();
+  }
+
+  @Override
+  public String description() {
+    return name() + " - Returns the runtime Iceberg version";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  /**
+   * Concrete implementation of {@link IcebergVersionFunction} that returns the short version string
+   * when called, e.g. {@code SELECT system.iceberg_version()} will produce results like "0.14.0" or
+   * "0.15.0-SNAPSHOT".
+   */
+  static class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {

Review Comment:
   Why is this exposed? It should be private and could be a singleton.
   
   Also, this name is very verbose for an inner class. Why not `Impl`?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934046493


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
+  private static final String[] DB_NAMESPACE = new String[] {"db"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog() {
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createDefaultNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
+  }
+
+  @After
+  public void dropDefaultNamespace() {
+    sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assert.assertArrayEquals(
+        "Listing functions in an existing namespace that's not system should not throw",
+        new Identifier[0],
+        asFunctionCatalog.listFunctions(DEFAULT_NAMESPACE));
+
+    AssertHelpers.assertThrows(
+        "Listing functions in a namespace that does not exist should throw",
+        NoSuchNamespaceException.class,
+        "Namespace 'db' not found",
+        () -> asFunctionCatalog.listFunctions(DB_NAMESPACE));
+  }
+
+  @Test
+  public void testLoadFunctions() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func)
+          .isNotNull()
+          .isInstanceOf(UnboundFunction.class)
+          .isExactlyInstanceOf(IcebergVersionFunction.class);
+    }
+
+    AssertHelpers.assertThrows(
+        "Cannot load a function if it's not used with the system namespace or the empty namespace",
+        NoSuchFunctionException.class,
+        "Undefined function: default.iceberg_version",
+        () -> asFunctionCatalog.loadFunction(Identifier.of(DEFAULT_NAMESPACE, "iceberg_version")));
+
+    AssertHelpers.assertThrows(
+        "Cannot load a function that does not exist",
+        NoSuchFunctionException.class,
+        "Undefined function: system.undefined_function",
+        () ->
+            asFunctionCatalog.loadFunction(Identifier.of(SYSTEM_NAMESPACE, "undefined_function")));
+
+    AssertHelpers.assertThrows(
+        "Using an undefined function from SQL should fail analysis",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT undefined_function(1, 2)"));
+  }
+
+  @Test
+  public void testIcebergVersionFunctionReturnsCorrectResult() {
+    String buildVersion = IcebergBuild.version();
+
+    // Check that value correctly loads from file, even in codegen etc
+    Assert.assertNotEquals(
+        "iceberg_version should not return undefined",
+        "undefined",
+        scalarSql("SELECT %s.system.iceberg_version()", catalogName));

Review Comment:
   Removed!



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933560765


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunctionImpl.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * {@link org.apache.spark.sql.connector.catalog.functions.BoundFunction} that implements {@link
+ * org.apache.iceberg.spark.functions.IcebergVersionFunction} by returning the short version string,
+ * e.g. "0.14.0" or "0.15.0-SNAPSHOT".
+ */
+public class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {

Review Comment:
   I made it public so I could test that it resolves to the right class type. But it could be at the least nested within the `IcebergVersionFunction` (which is how I had it before). I can update that.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format("Invalid input type. %s does not except arguments.", name()));

Review Comment:
   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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933594702


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func).isExactlyInstanceOf(IcebergVersionFunction.class);
+      Assertions.assertThat(func.name()).asString().isEqualTo("iceberg_version");
+      Assertions.assertThat(func.description())
+          .containsIgnoringCase("Returns the runtime Iceberg version");
+    }
+  }
+
+  @Test
+  public void testBindAndUseFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction unboundFunc = asFunctionCatalog.loadFunction(identifier);
+
+      BoundFunction boundFunc = unboundFunc.bind(new StructType());
+      Assertions.assertThat(boundFunc.getClass().toString())
+          .containsIgnoringCase("IcebergVersionFunctionImpl");
+      Assertions.assertThat(boundFunc.canonicalName()).isEqualTo("iceberg.iceberg_version");
+      Assertions.assertThat(boundFunc.isResultNullable()).isFalse();
+      Assertions.assertThat(boundFunc.isDeterministic()).isTrue();
+
+      // Ensure the bound result produces the expected UTF8String type.
+      ScalarFunction<UTF8String> scalarFunc = (ScalarFunction<UTF8String>) boundFunc;
+      Assert.assertEquals(
+          UTF8String.fromString(IcebergBuild.version()),
+          scalarFunc.produceResult(InternalRow.empty()));
+    }
+  }
+
+  @Test
+  public void testCannotLoadFunctionsFromInvalidNamespace() {

Review Comment:
   Added a test that uses the catalog directly so it can also assert on the error message.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932744986


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));
+    }
+
+    return new IcebergVersion();
+  }
+
+  @Override
+  public String description() {
+    return "iceberg_version - Returns the runtime Iceberg version for debugging purposes";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  public static class IcebergVersion implements ScalarFunction<String> {
+
+    private static final UTF8String icebergBuildVersion =
+        UTF8String.fromString(IcebergBuild.version());
+
+    public static UTF8String invoke() {
+      return icebergBuildVersion;
+    }
+
+    @Override
+    public DataType[] inputTypes() {
+      return new DataType[0];
+    }
+
+    @Override
+    public DataType resultType() {
+      return DataTypes.StringType;
+    }
+
+    @Override
+    public String canonicalName() {
+      return "org.apache.iceberg.spark.functions.iceberg_version()";

Review Comment:
   Sure.
   
   Spark uses these to differentiate the identity of two functions across catalogs and namespaces, so I erred on the longer side.
   
   But I think `iceberg.` is fine as this isn’t really meant for end users anyway and if some fork maintainer is using `iceberg.iceberg_version` for different logic I’d be very surprised.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932732149


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));
+    }
+
+    return new IcebergVersion();
+  }
+
+  @Override
+  public String description() {
+    return "iceberg_version - Returns the runtime Iceberg version for debugging purposes";

Review Comment:
   This doesn't need to suggest a purpose for calling the function. "Returns the runtime Iceberg version" is sufficient.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934047571


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * A function for use in SQL that returns the current Iceberg version, e.g. {@code SELECT
+ * system.iceberg_version()} will return a String such as "0.14.0" or "0.15.0-SNAPSHOT"
+ */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format("Cannot bind: %s does not accept arguments", name()));
+    }
+
+    return new IcebergVersionFunctionImpl();
+  }
+
+  @Override
+  public String description() {
+    return name() + " - Returns the runtime Iceberg version";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  // class must be at least package private to be resolvable

Review Comment:
   To be resolvable here means that the actual concrete implementation can be loaded by the function catalog and used within SQL and passes through the analyzer and is callable at runtime.
   
   I've updated the sentence to be more specific though.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933492008


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBaseWithCatalog.java:
##########
@@ -80,6 +80,7 @@ public SparkTestBaseWithCatalog(
     spark.conf().set("spark.sql.catalog." + catalogName, implementation);
     config.forEach(
         (key, value) -> spark.conf().set("spark.sql.catalog." + catalogName + "." + key, value));
+    spark.conf().set("spark.sql.defaultCatalog", catalogName);

Review Comment:
   I removed it. Just set it as a convenience when seeing what could be done with the session catalog or not.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933570940


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func).isExactlyInstanceOf(IcebergVersionFunction.class);
+      Assertions.assertThat(func.name()).asString().isEqualTo("iceberg_version");
+      Assertions.assertThat(func.description())
+          .containsIgnoringCase("Returns the runtime Iceberg version");
+    }
+  }
+
+  @Test
+  public void testBindAndUseFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction unboundFunc = asFunctionCatalog.loadFunction(identifier);
+
+      BoundFunction boundFunc = unboundFunc.bind(new StructType());

Review Comment:
   This is testing the function and should be removed.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933575643


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** A function that gets registered to return the current Iceberg version in use from within SQL */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format("Cannot bind: %s does not accept arguments", name()));
+    }
+
+    return new IcebergVersionFunctionImpl();
+  }
+
+  @Override
+  public String description() {
+    return name() + " - Returns the runtime Iceberg version";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  /**
+   * Concrete implementation of {@link IcebergVersionFunction} that returns the short version string
+   * when called, e.g. {@code SELECT system.iceberg_version()} will produce results like "0.14.0" or
+   * "0.15.0-SNAPSHOT".
+   */
+  static class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {

Review Comment:
   It cannot be private. 
   
   It can be package-private, but making it private makes the invoke method not accessible. I tried several combinations to make it private, but none of them worked.
   
   When invoke is public static and the subclass is private static I get the error message:
   
   interface org.apache.spark.sql.catalyst.expressions.objects.InvokeLike cannot access a member of class org.apache.iceberg.spark.functions.IcebergVersionFunction$IcebergVersionFunctionImpl with modifiers "public static"
   java.lang.IllegalAccessException: interface org.apache.spark.sql.catalyst.expressions.objects.InvokeLike cannot access a member of class org.apache.iceberg.spark.functions.IcebergVersionFunction$IcebergVersionFunctionImpl with modifiers "public static"
   	at java.base/jdk.internal.reflect.Reflection.newIllegalAccessException(Reflection.java:361)
   	at java.base/java.lang.reflect.AccessibleObject.checkAccess(AccessibleObject.java:591)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:558)
   Making invoke anything other than public static gives an error about no function invoke defined (indicating it needs to be public). So the best it seems I can do is package-private as a subcl



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933578107


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** A function that gets registered to return the current Iceberg version in use from within SQL */

Review Comment:
   I removed the "gets registered" as that's not relevant to the user.
   
   From the user's perspective, this is essentially used from SQL. I guess the user could go about the full `loadFunction` and then `.bind` and then call `produceResult`, but it's really more efficient and intended for usage with SQL / code gen.
   
   I can remove the SQL part though I think it's helpful to have the example.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933561164


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {SparkCatalogConfig.HADOOP}, {SparkCatalogConfig.HIVE}, {SparkCatalogConfig.SPARK}
+    };
+  }
+
+  private static final Namespace NS = Namespace.of("db");
+  private final boolean isSessionCatalog;
+  private final String fullNamespace;
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.isSessionCatalog = "spark_catalog".equals(catalogName);
+    this.fullNamespace = (isSessionCatalog ? "" : catalogName + ".") + NS;

Review Comment:
   Updated t



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933708157


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java:
##########
@@ -47,4 +56,38 @@ public Procedure loadProcedure(Identifier ident) throws NoSuchProcedureException
 
     throw new NoSuchProcedureException(ident);
   }
+
+  @Override
+  public Identifier[] listFunctions(String[] namespace) throws NoSuchNamespaceException {
+    if (namespace.length == 0
+        || (namespace.length == 1 && namespace[0].equalsIgnoreCase("system"))) {

Review Comment:
   It is also used in the method for loading procedures.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932732980


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));
+    }
+
+    return new IcebergVersion();
+  }
+
+  @Override
+  public String description() {
+    return "iceberg_version - Returns the runtime Iceberg version for debugging purposes";

Review Comment:
   Also, this should probably delegate to `name()`.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932748127


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {SparkCatalogConfig.HADOOP}, {SparkCatalogConfig.HIVE}, {SparkCatalogConfig.SPARK}
+    };
+  }
+
+  private static final Namespace NS = Namespace.of("db");
+  private final boolean isSessionCatalog;
+  private final String fullNamespace;
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.isSessionCatalog = "spark_catalog".equals(catalogName);
+    this.fullNamespace = (isSessionCatalog ? "" : catalogName + ".") + NS;
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", fullNamespace);
+  }
+
+  @After
+  public void cleanNamespaces() {
+    sql("DROP NAMESPACE IF EXISTS %s", fullNamespace);
+  }
+
+  @Test
+  public void testLoadListAndUseFunctionsFromSystemNamespace()
+      throws NoSuchFunctionException, NoSuchNamespaceException {
+    // Session catalog requires that the namespace actually exists
+    if (isSessionCatalog) {

Review Comment:
   Let's remove some of the code that is specific to a catalog and just use a regular SparkCatalog for now. That should simplify these tests.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933570750


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func).isExactlyInstanceOf(IcebergVersionFunction.class);
+      Assertions.assertThat(func.name()).asString().isEqualTo("iceberg_version");
+      Assertions.assertThat(func.description())
+          .containsIgnoringCase("Returns the runtime Iceberg version");

Review Comment:
   These are okay, but the assertions for name and description are testing the implementation, not the catalog.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933565757


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** A function that gets registered to return the current Iceberg version in use from within SQL */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format("Cannot bind: %s does not accept arguments", name()));
+    }
+
+    return new IcebergVersionFunctionImpl();
+  }
+
+  @Override
+  public String description() {
+    return name() + " - Returns the runtime Iceberg version";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  /**
+   * Concrete implementation of {@link IcebergVersionFunction} that returns the short version string
+   * when called, e.g. {@code SELECT system.iceberg_version()} will produce results like "0.14.0" or
+   * "0.15.0-SNAPSHOT".
+   */
+  static class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {

Review Comment:
   Why is this exposed? It should be private.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#issuecomment-1199940719

   I ran coverage.
   
   The changes I made in:
   - `BaseCatalog` - 100% for both methods and lines
   - `SparkFunctions` - 100% across the board
   - `IcebergVersionFunction` - 100% for methods and classes, but 87% for lines (not tests for calling `bind` with arguments).
   
   I figure since we didn't really want to test `bind` much anyway that that's fine.


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933721234


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java:
##########
@@ -47,4 +56,38 @@ public Procedure loadProcedure(Identifier ident) throws NoSuchProcedureException
 
     throw new NoSuchProcedureException(ident);
   }
+
+  @Override
+  public Identifier[] listFunctions(String[] namespace) throws NoSuchNamespaceException {
+    if (namespace.length == 0
+        || (namespace.length == 1 && namespace[0].equalsIgnoreCase("system"))) {

Review Comment:
   Cool. Updated all 3. This helps with the funky formatting too. Thank you!



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933469722


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {SparkCatalogConfig.HADOOP}, {SparkCatalogConfig.HIVE}, {SparkCatalogConfig.SPARK}
+    };
+  }
+
+  private static final Namespace NS = Namespace.of("db");
+  private final boolean isSessionCatalog;
+  private final String fullNamespace;
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.isSessionCatalog = "spark_catalog".equals(catalogName);
+    this.fullNamespace = (isSessionCatalog ? "" : catalogName + ".") + NS;
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", fullNamespace);
+  }
+
+  @After
+  public void cleanNamespaces() {
+    sql("DROP NAMESPACE IF EXISTS %s", fullNamespace);
+  }
+
+  @Test
+  public void testLoadListAndUseFunctionsFromSystemNamespace()
+      throws NoSuchFunctionException, NoSuchNamespaceException {
+    // Session catalog requires that the namespace actually exists
+    if (isSessionCatalog) {
+      sql("CREATE NAMESPACE IF NOT EXISTS system");
+    }
+
+    try {
+      String[] namespace = {"system"};
+      String name = "iceberg_version";
+      Identifier identifier = Identifier.of(namespace, name);
+
+      assertListingLoadingAndBindingFrom(identifier);
+    } finally {
+      if (isSessionCatalog) {
+        sql("DROP NAMESPACE IF EXISTS system");
+      }
+    }
+  }
+
+  @Test
+  public void testLoadListAndUseFunctionsFromEmptyNamespace()
+      throws NoSuchFunctionException, NoSuchNamespaceException {
+    String[] namespace = {};
+    String name = "iceberg_version";
+    Identifier identifier = Identifier.of(namespace, name);
+
+    assertListingLoadingAndBindingFrom(identifier);
+  }
+
+  @Test
+  public void testCannotLoadFunctionsFromInvalidNamespace() {
+    AssertHelpers.assertThrows(
+        "Function Catalog functions should only be accessible from the system namespace and empty namespace",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT %s.iceberg_version()", fullNamespace));
+  }
+
+  @Test
+  public void testCannotUseUndefinedFunction() {
+    AssertHelpers.assertThrows(
+        "Using an undefined function should throw",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT undefined_function(1, 2)"));
+  }
+
+  private void assertListingLoadingAndBindingFrom(Identifier identifier)

Review Comment:
   Ok so we can't exactly test `DESCRIBE FUNCTION` as mentioned. We might want to put in the work upstream to fix that for V2 functions.
   
   When testing `DESCRIBE FUNCTION` with the session catalog (from SQL), it resolves to the v1 session catalog and I get the following error



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934046534


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
+  private static final String[] DB_NAMESPACE = new String[] {"db"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog() {
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createDefaultNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
+  }
+
+  @After
+  public void dropDefaultNamespace() {
+    sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assert.assertArrayEquals(
+        "Listing functions in an existing namespace that's not system should not throw",
+        new Identifier[0],
+        asFunctionCatalog.listFunctions(DEFAULT_NAMESPACE));
+
+    AssertHelpers.assertThrows(
+        "Listing functions in a namespace that does not exist should throw",
+        NoSuchNamespaceException.class,
+        "Namespace 'db' not found",
+        () -> asFunctionCatalog.listFunctions(DB_NAMESPACE));
+  }
+
+  @Test
+  public void testLoadFunctions() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func)
+          .isNotNull()
+          .isInstanceOf(UnboundFunction.class)
+          .isExactlyInstanceOf(IcebergVersionFunction.class);
+    }
+
+    AssertHelpers.assertThrows(
+        "Cannot load a function if it's not used with the system namespace or the empty namespace",
+        NoSuchFunctionException.class,
+        "Undefined function: default.iceberg_version",
+        () -> asFunctionCatalog.loadFunction(Identifier.of(DEFAULT_NAMESPACE, "iceberg_version")));
+
+    AssertHelpers.assertThrows(
+        "Cannot load a function that does not exist",
+        NoSuchFunctionException.class,
+        "Undefined function: system.undefined_function",
+        () ->
+            asFunctionCatalog.loadFunction(Identifier.of(SYSTEM_NAMESPACE, "undefined_function")));
+
+    AssertHelpers.assertThrows(
+        "Using an undefined function from SQL should fail analysis",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT undefined_function(1, 2)"));
+  }
+
+  @Test
+  public void testIcebergVersionFunctionReturnsCorrectResult() {

Review Comment:
   Renamed!



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934048717


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
+  private static final String[] DB_NAMESPACE = new String[] {"db"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog() {
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createDefaultNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
+  }
+
+  @After
+  public void dropDefaultNamespace() {
+    sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));

Review Comment:
   I updated the tests, but built-in functions are case-insensitive by default.
   
   Tests done on Spark 3.3:
   
   ```scala
   scala> spark.sql("SELECT uuid() as _uuid").show()
   +--------------------+
   |               _uuid|
   +--------------------+
   |3d52c2c7-225c-44c...|
   +--------------------+
   
   
   scala> spark.sql("SELECT UUID() as _uuid").show()
   +--------------------+
   |               _uuid|
   +--------------------+
   |1babdfb6-1f71-498...|
   +--------------------+
   
   
   scala> spark.sql("SELECT UuID() as _uuid").show()
   +--------------------+
   |               _uuid|
   +--------------------+
   |d63892a1-e2e6-49d...|
   +--------------------+
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933575333


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestIcebergVersionFunction.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.iceberg.spark.sql;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestIcebergVersionFunction extends SparkTestBaseWithCatalog {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.SPARK}};
+  }
+
+  private final String fullyQualifiedSystemNamespace;
+
+  public TestIcebergVersionFunction(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.fullyQualifiedSystemNamespace = catalogConfig.catalogName() + ".system";
+  }
+
+  @Test
+  public void testIcebergVersionFunctionReturnsCorrectResult() {

Review Comment:
   Since the `TestFunctionCatalog` suite needs a test that exercises load and everything through SQL, I think you should just move this test case there and remove this suite. It isn't doing much anyway.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933590124


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {

Review Comment:
   Not the root namespace, I mean tests for a namespace that exists, but has no functions in it.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933612846


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestIcebergVersionFunction.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.iceberg.spark.sql;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestIcebergVersionFunction extends SparkTestBaseWithCatalog {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.SPARK}};
+  }
+
+  private final String fullyQualifiedSystemNamespace;
+
+  public TestIcebergVersionFunction(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.fullyQualifiedSystemNamespace = catalogConfig.catalogName() + ".system";
+  }
+
+  @Test
+  public void testIcebergVersionFunctionReturnsCorrectResult() {
+    // Check that value correctly loads from file, even in codegen etc
+    String buildVersion = IcebergBuild.version();
+    Assert.assertNotEquals(
+        "iceberg_version should not return undefined",
+        "undefined",
+        scalarSql("SELECT %s.iceberg_version()", fullyQualifiedSystemNamespace));
+
+    Assert.assertEquals(
+        "Should be able to use the Iceberg version function from the fully qualified system namespace",
+        buildVersion,
+        scalarSql("SELECT %s.iceberg_version()", fullyQualifiedSystemNamespace));

Review Comment:
   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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933620930


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {

Review Comment:
   I added that too.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933573197


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func).isExactlyInstanceOf(IcebergVersionFunction.class);
+      Assertions.assertThat(func.name()).asString().isEqualTo("iceberg_version");
+      Assertions.assertThat(func.description())
+          .containsIgnoringCase("Returns the runtime Iceberg version");
+    }
+  }
+
+  @Test
+  public void testBindAndUseFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction unboundFunc = asFunctionCatalog.loadFunction(identifier);
+
+      BoundFunction boundFunc = unboundFunc.bind(new StructType());
+      Assertions.assertThat(boundFunc.getClass().toString())
+          .containsIgnoringCase("IcebergVersionFunctionImpl");
+      Assertions.assertThat(boundFunc.canonicalName()).isEqualTo("iceberg.iceberg_version");
+      Assertions.assertThat(boundFunc.isResultNullable()).isFalse();
+      Assertions.assertThat(boundFunc.isDeterministic()).isTrue();
+
+      // Ensure the bound result produces the expected UTF8String type.
+      ScalarFunction<UTF8String> scalarFunc = (ScalarFunction<UTF8String>) boundFunc;
+      Assert.assertEquals(
+          UTF8String.fromString(IcebergBuild.version()),
+          scalarFunc.produceResult(InternalRow.empty()));
+    }
+  }
+
+  @Test
+  public void testCannotLoadFunctionsFromInvalidNamespace() {
+    String dbNamespace = catalogName + ".db";
+    try {
+      sql("CREATE NAMESPACE IF NOT EXISTS %s", dbNamespace);
+      AssertHelpers.assertThrows(
+          "Function Catalog functions should only be accessible from the system namespace and empty namespace",
+          AnalysisException.class,
+          "Undefined function",
+          () -> sql("SELECT %s.iceberg_version()", dbNamespace));
+    } finally {
+      sql("DROP NAMESPACE IF EXISTS %s", dbNamespace);
+    }
+  }
+
+  @Test
+  public void testCannotUseUndefinedFunction() {

Review Comment:
   This is testing Spark, not really the Iceberg catalog.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933604250


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func).isExactlyInstanceOf(IcebergVersionFunction.class);
+      Assertions.assertThat(func.name()).asString().isEqualTo("iceberg_version");
+      Assertions.assertThat(func.description())
+          .containsIgnoringCase("Returns the runtime Iceberg version");
+    }
+  }
+
+  @Test
+  public void testBindAndUseFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction unboundFunc = asFunctionCatalog.loadFunction(identifier);
+
+      BoundFunction boundFunc = unboundFunc.bind(new StructType());
+      Assertions.assertThat(boundFunc.getClass().toString())
+          .containsIgnoringCase("IcebergVersionFunctionImpl");
+      Assertions.assertThat(boundFunc.canonicalName()).isEqualTo("iceberg.iceberg_version");
+      Assertions.assertThat(boundFunc.isResultNullable()).isFalse();
+      Assertions.assertThat(boundFunc.isDeterministic()).isTrue();
+
+      // Ensure the bound result produces the expected UTF8String type.
+      ScalarFunction<UTF8String> scalarFunc = (ScalarFunction<UTF8String>) boundFunc;
+      Assert.assertEquals(
+          UTF8String.fromString(IcebergBuild.version()),
+          scalarFunc.produceResult(InternalRow.empty()));
+    }
+  }
+
+  @Test
+  public void testCannotLoadFunctionsFromInvalidNamespace() {
+    String dbNamespace = catalogName + ".db";
+    try {
+      sql("CREATE NAMESPACE IF NOT EXISTS %s", dbNamespace);
+      AssertHelpers.assertThrows(
+          "Function Catalog functions should only be accessible from the system namespace and empty namespace",
+          AnalysisException.class,
+          "Undefined function",
+          () -> sql("SELECT %s.iceberg_version()", dbNamespace));
+    } finally {
+      sql("DROP NAMESPACE IF EXISTS %s", dbNamespace);
+    }
+  }
+
+  @Test
+  public void testCannotUseUndefinedFunction() {

Review Comment:
   I moved the test that is for the catalog into `testLoadFunctionsViaCatalog()` and removed this test that's just testing Spark.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932743659


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {

Review Comment:
   Can you run this with just one catalog? 3 seems unnecessary.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932732851


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));

Review Comment:
   Referring to "fields" here seems strange. How about `name() + " does not expect arguments"`?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934045366


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
+  private static final String[] DB_NAMESPACE = new String[] {"db"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog() {
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createDefaultNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
+  }
+
+  @After
+  public void dropDefaultNamespace() {
+    sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));

Review Comment:
   I chose this as function resolution is case-insensitive in Spark, at least according to the existing code for `Procedure`'s. This way, it also matches our current `ProcedureCatalog` callables. I can update it though.
   
   https://github.com/apache/iceberg/blob/14f4bc1ebe021bce35b756685a88026d85dd6025/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java#L34-L37



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934045366


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
+  private static final String[] DB_NAMESPACE = new String[] {"db"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog() {
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createDefaultNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
+  }
+
+  @After
+  public void dropDefaultNamespace() {
+    sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));

Review Comment:
   I chose this as function resolution is case-insensitive in Spark, at least according to the existing code for `Procedure`'s as well as my own investigation. This way, it also matches our current `ProcedureCatalog` callables. I can update it though.
   
   https://github.com/apache/iceberg/blob/14f4bc1ebe021bce35b756685a88026d85dd6025/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java#L34-L37
   
   Note that I tried with and without `spark.sql.caseSensitive` as `true` or `false` and it didn't change anything. Function resolution is case-insensitive (this is Spark 3.3.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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934738480


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
+  private static final String[] DB_NAMESPACE = new String[] {"db"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog() {
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createDefaultNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
+  }
+
+  @After
+  public void dropDefaultNamespace() {
+    sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));

Review Comment:
   Yeah, I think Spark functions are case insensitive no matter what.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932765298


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestIcebergVersionFunction.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.iceberg.spark.sql;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestIcebergVersionFunction extends SparkTestBaseWithCatalog {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}, {SparkCatalogConfig.HIVE}};

Review Comment:
   Yeah I've reduced it to one. At one point both test suites were in the same file, hence the usage of two originally here.
   
   But it's just one now.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933706520


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java:
##########
@@ -47,4 +56,38 @@ public Procedure loadProcedure(Identifier ident) throws NoSuchProcedureException
 
     throw new NoSuchProcedureException(ident);
   }
+
+  @Override
+  public Identifier[] listFunctions(String[] namespace) throws NoSuchNamespaceException {
+    if (namespace.length == 0
+        || (namespace.length == 1 && namespace[0].equalsIgnoreCase("system"))) {

Review Comment:
   nit: What about defining a helper function `isSystemNamespace`? There are 3 places where we can call it.
   
   ```
   private boolean isSystemNamespace(String[] namespace) {
     return namespace.length == 1 && namespace[0].equalsIgnoreCase("system");
   }
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933707745


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * A function for use in SQL that returns the current Iceberg version, e.g. e.g. {@code SELECT
+ * system.iceberg_version()} will return a value such as "0.14.0" or "0.15.0-SNAPSHOT"
+ */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format("Cannot bind: %s does not accept arguments", name()));
+    }
+
+    return new IcebergVersionFunctionImpl();
+  }
+
+  @Override
+  public String description() {
+    return name() + " - Returns the runtime Iceberg version";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  // class must be at least package private to be resolvable
+  static class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {
+    private static final UTF8String versionAsSparkString =

Review Comment:
   nit: Static final vars must use all capital letters. Also, what about shortening the name so that it fits on one line?
   
   ```
   private static final UTF8String VERSION = UTF8String.fromString(IcebergBuild.version());
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933706520


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java:
##########
@@ -47,4 +56,38 @@ public Procedure loadProcedure(Identifier ident) throws NoSuchProcedureException
 
     throw new NoSuchProcedureException(ident);
   }
+
+  @Override
+  public Identifier[] listFunctions(String[] namespace) throws NoSuchNamespaceException {
+    if (namespace.length == 0
+        || (namespace.length == 1 && namespace[0].equalsIgnoreCase("system"))) {

Review Comment:
   nit: What about defining a helper function `isSystemNamespace`? There are 3 places where we can call it.
   
   ```
   private boolean isSystemNamespace(String[] namespace) {
     return namespace.length == 1 && namespace[0].equalsIgnoreCase("system");
   }
   ```
   
   Then the condition would fit on one line.
   
   ```
   if (namespace.length == 0 || isSystemNamespace(namespace)) {
   ...
   }
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934045366


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
+  private static final String[] DB_NAMESPACE = new String[] {"db"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog() {
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createDefaultNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
+  }
+
+  @After
+  public void dropDefaultNamespace() {
+    sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));

Review Comment:
   I chose this as function resolution is case-insensitive in Spark, at least according to the existing code for `Procedure`'s as well as my own investigation. This way, it also matches our current `ProcedureCatalog` callables. I can update it though.
   
   https://github.com/apache/iceberg/blob/14f4bc1ebe021bce35b756685a88026d85dd6025/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java#L34-L37



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933471934


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));
+    }
+
+    return new IcebergVersion();
+  }
+
+  @Override
+  public String description() {
+    return "iceberg_version - Returns the runtime Iceberg version for debugging purposes";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  public static class IcebergVersion implements ScalarFunction<String> {
+
+    private static final UTF8String icebergBuildVersion =
+        UTF8String.fromString(IcebergBuild.version());
+
+    public static UTF8String invoke() {
+      return icebergBuildVersion;
+    }
+
+    @Override
+    public DataType[] inputTypes() {
+      return new DataType[0];
+    }
+
+    @Override
+    public DataType resultType() {
+      return DataTypes.StringType;
+    }
+
+    @Override
+    public String canonicalName() {
+      return "org.apache.iceberg.spark.functions.iceberg_version()";

Review Comment:
   Yeah, we do want to make sure that the string is unique. But there is also no need to make it overly complicated.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933571738


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunctionImpl.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * {@link org.apache.spark.sql.connector.catalog.functions.BoundFunction} that implements {@link
+ * org.apache.iceberg.spark.functions.IcebergVersionFunction} by returning the short version string,
+ * e.g. "0.14.0" or "0.15.0-SNAPSHOT".
+ */
+public class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {

Review Comment:
   So it can be a nested subclass (known).
   
   It can also be package-private, but making it `private` makes the `invoke` method not accessible.
   
   When invoke is `public static` and the subclass is private static I get the error message:
   
   ```
   interface org.apache.spark.sql.catalyst.expressions.objects.InvokeLike cannot access a member of class org.apache.iceberg.spark.functions.IcebergVersionFunction$IcebergVersionFunctionImpl with modifiers "public static"
   java.lang.IllegalAccessException: interface org.apache.spark.sql.catalyst.expressions.objects.InvokeLike cannot access a member of class org.apache.iceberg.spark.functions.IcebergVersionFunction$IcebergVersionFunctionImpl with modifiers "public static"
   	at java.base/jdk.internal.reflect.Reflection.newIllegalAccessException(Reflection.java:361)
   	at java.base/java.lang.reflect.AccessibleObject.checkAccess(AccessibleObject.java:591)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:558)
   ```
   
   Making invoke anything other than `public static` gives an error about no function `invoke` defined  (indicating it needs to be public). So the best it seems I can do is package-private as a subclass.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933604680


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestIcebergVersionFunction.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.iceberg.spark.sql;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestIcebergVersionFunction extends SparkTestBaseWithCatalog {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.SPARK}};
+  }
+
+  private final String fullyQualifiedSystemNamespace;
+
+  public TestIcebergVersionFunction(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.fullyQualifiedSystemNamespace = catalogConfig.catalogName() + ".system";
+  }
+
+  @Test
+  public void testIcebergVersionFunctionReturnsCorrectResult() {

Review Comment:
   Sure thing.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934036915


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
+  private static final String[] DB_NAMESPACE = new String[] {"db"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog() {
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createDefaultNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
+  }
+
+  @After
+  public void dropDefaultNamespace() {
+    sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assert.assertArrayEquals(
+        "Listing functions in an existing namespace that's not system should not throw",
+        new Identifier[0],
+        asFunctionCatalog.listFunctions(DEFAULT_NAMESPACE));
+
+    AssertHelpers.assertThrows(
+        "Listing functions in a namespace that does not exist should throw",
+        NoSuchNamespaceException.class,
+        "Namespace 'db' not found",
+        () -> asFunctionCatalog.listFunctions(DB_NAMESPACE));
+  }
+
+  @Test
+  public void testLoadFunctions() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func)
+          .isNotNull()
+          .isInstanceOf(UnboundFunction.class)
+          .isExactlyInstanceOf(IcebergVersionFunction.class);
+    }
+
+    AssertHelpers.assertThrows(
+        "Cannot load a function if it's not used with the system namespace or the empty namespace",
+        NoSuchFunctionException.class,
+        "Undefined function: default.iceberg_version",
+        () -> asFunctionCatalog.loadFunction(Identifier.of(DEFAULT_NAMESPACE, "iceberg_version")));
+
+    AssertHelpers.assertThrows(
+        "Cannot load a function that does not exist",
+        NoSuchFunctionException.class,
+        "Undefined function: system.undefined_function",
+        () ->
+            asFunctionCatalog.loadFunction(Identifier.of(SYSTEM_NAMESPACE, "undefined_function")));
+
+    AssertHelpers.assertThrows(
+        "Using an undefined function from SQL should fail analysis",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT undefined_function(1, 2)"));
+  }
+
+  @Test
+  public void testIcebergVersionFunctionReturnsCorrectResult() {
+    String buildVersion = IcebergBuild.version();
+
+    // Check that value correctly loads from file, even in codegen etc
+    Assert.assertNotEquals(
+        "iceberg_version should not return undefined",
+        "undefined",
+        scalarSql("SELECT %s.system.iceberg_version()", catalogName));

Review Comment:
   Minor: if you have a check for a specific value, that's also a check that it is not some other value. This `assertNotEquals` is redundant.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933491665


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {SparkCatalogConfig.HADOOP}, {SparkCatalogConfig.HIVE}, {SparkCatalogConfig.SPARK}
+    };
+  }
+
+  private static final Namespace NS = Namespace.of("db");
+  private final boolean isSessionCatalog;
+  private final String fullNamespace;
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.isSessionCatalog = "spark_catalog".equals(catalogName);
+    this.fullNamespace = (isSessionCatalog ? "" : catalogName + ".") + NS;
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", fullNamespace);
+  }
+
+  @After
+  public void cleanNamespaces() {
+    sql("DROP NAMESPACE IF EXISTS %s", fullNamespace);
+  }
+
+  @Test
+  public void testLoadListAndUseFunctionsFromSystemNamespace()
+      throws NoSuchFunctionException, NoSuchNamespaceException {
+    // Session catalog requires that the namespace actually exists
+    if (isSessionCatalog) {
+      sql("CREATE NAMESPACE IF NOT EXISTS system");
+    }
+
+    try {
+      String[] namespace = {"system"};
+      String name = "iceberg_version";
+      Identifier identifier = Identifier.of(namespace, name);
+
+      assertListingLoadingAndBindingFrom(identifier);
+    } finally {
+      if (isSessionCatalog) {
+        sql("DROP NAMESPACE IF EXISTS system");
+      }
+    }
+  }
+
+  @Test
+  public void testLoadListAndUseFunctionsFromEmptyNamespace()
+      throws NoSuchFunctionException, NoSuchNamespaceException {
+    String[] namespace = {};
+    String name = "iceberg_version";
+    Identifier identifier = Identifier.of(namespace, name);
+
+    assertListingLoadingAndBindingFrom(identifier);
+  }
+
+  @Test
+  public void testCannotLoadFunctionsFromInvalidNamespace() {
+    AssertHelpers.assertThrows(
+        "Function Catalog functions should only be accessible from the system namespace and empty namespace",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT %s.iceberg_version()", fullNamespace));
+  }
+
+  @Test
+  public void testCannotUseUndefinedFunction() {
+    AssertHelpers.assertThrows(
+        "Using an undefined function should throw",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT undefined_function(1, 2)"));
+  }
+
+  private void assertListingLoadingAndBindingFrom(Identifier identifier)

Review Comment:
   ```
   scalarSQL("DESCRIBE FUNCTION iceberg_version");
   ```
   
   class org.apache.iceberg.spark.functions.IcebergVersionFunction cannot be cast to class org.apache.spark.sql.internal.connector.V1Function (org.apache.iceberg.spark.functions.IcebergVersionFunction and org.apache.spark.sql.internal.connector.V1Function are in unnamed module of loader 'app')
   java.lang.ClassCastException: class org.apache.iceberg.spark.functions.IcebergVersionFunction cannot be cast to class org.apache.spark.sql.internal.connector.V1Function (org.apache.iceberg.spark.functions.IcebergVersionFunction and org.apache.spark.sql.internal.connector.V1Function are in unnamed module of loader 'app')
   	at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog$$anonfun$apply$1.applyOrElse(ResolveSessionCatalog.scala:389)
   	at org.apache.spark.sql.catalyst.analysis.ResolveSessionCatalog$$anonfun$apply$1.applyOrElse(ResolveSessionCatalog.scala:49)
   	at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsUpWithPruning$3(AnalysisHelper.scala:138)
   
   So it seems that we can't `DESCRIBE` on a non-v1 function from within the tests. The tests in Spark also support this idea.
   
   So calling the function directly with code is the best it seems we can do for those, but the function resolution (e.e. `SELECT system.iceberg_version` could be moved over here and we could get rid of `TestIcebergVersionFunction`.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933477113


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {SparkCatalogConfig.HADOOP}, {SparkCatalogConfig.HIVE}, {SparkCatalogConfig.SPARK}
+    };
+  }
+
+  private static final Namespace NS = Namespace.of("db");
+  private final boolean isSessionCatalog;
+  private final String fullNamespace;
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.isSessionCatalog = "spark_catalog".equals(catalogName);
+    this.fullNamespace = (isSessionCatalog ? "" : catalogName + ".") + NS;

Review Comment:
   There should be no need to change behavior based on whether this is the session catalog. This can refer to the session catalog by name, `spark_catalog`.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933573030


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);

Review Comment:
   Can you also add a tests for functions that don't exist?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933577485


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** A function that gets registered to return the current Iceberg version in use from within SQL */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format("Cannot bind: %s does not accept arguments", name()));
+    }
+
+    return new IcebergVersionFunctionImpl();
+  }
+
+  @Override
+  public String description() {
+    return name() + " - Returns the runtime Iceberg version";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  /**
+   * Concrete implementation of {@link IcebergVersionFunction} that returns the short version string
+   * when called, e.g. {@code SELECT system.iceberg_version()} will produce results like "0.14.0" or
+   * "0.15.0-SNAPSHOT".
+   */
+  static class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {

Review Comment:
   Makes sense! Can you add a comment?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933462631


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/SparkTestBaseWithCatalog.java:
##########
@@ -80,6 +80,7 @@ public SparkTestBaseWithCatalog(
     spark.conf().set("spark.sql.catalog." + catalogName, implementation);
     config.forEach(
         (key, value) -> spark.conf().set("spark.sql.catalog." + catalogName + "." + key, value));
+    spark.conf().set("spark.sql.defaultCatalog", catalogName);

Review Comment:
   Why set this for all tests? If you need it set in the new one, isn't it better to set it only in that test?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933707745


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * A function for use in SQL that returns the current Iceberg version, e.g. e.g. {@code SELECT
+ * system.iceberg_version()} will return a value such as "0.14.0" or "0.15.0-SNAPSHOT"
+ */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format("Cannot bind: %s does not accept arguments", name()));
+    }
+
+    return new IcebergVersionFunctionImpl();
+  }
+
+  @Override
+  public String description() {
+    return name() + " - Returns the runtime Iceberg version";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  // class must be at least package private to be resolvable
+  static class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {
+    private static final UTF8String versionAsSparkString =

Review Comment:
   nit: Static final vars must use all capital letters. Also, what about shortening the name so that it fits on one line?
   
   ```
   private static final UTF8String VERSION = UTF8String.fromString(IcebergBuild.version());
   ```
   
   Any var name would work.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933707039


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * A function for use in SQL that returns the current Iceberg version, e.g. e.g. {@code SELECT

Review Comment:
   nit: twice `e.g.`?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#issuecomment-1200073253

   > Looks good to me. I had a few minor suggestions. Thanks for working on this, @kbendick!
   
   Thanks for the review @aokolnychyi! I made those changes and rebased. This will hopefully help unblock parallelizing work on storage partitioned joins and things.


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932783190


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {SparkCatalogConfig.HADOOP}, {SparkCatalogConfig.HIVE}, {SparkCatalogConfig.SPARK}
+    };
+  }
+
+  private static final Namespace NS = Namespace.of("db");
+  private final boolean isSessionCatalog;
+  private final String fullNamespace;
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.isSessionCatalog = "spark_catalog".equals(catalogName);
+    this.fullNamespace = (isSessionCatalog ? "" : catalogName + ".") + NS;
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", fullNamespace);
+  }
+
+  @After
+  public void cleanNamespaces() {
+    sql("DROP NAMESPACE IF EXISTS %s", fullNamespace);
+  }
+
+  @Test
+  public void testLoadListAndUseFunctionsFromSystemNamespace()
+      throws NoSuchFunctionException, NoSuchNamespaceException {
+    // Session catalog requires that the namespace actually exists
+    if (isSessionCatalog) {
+      sql("CREATE NAMESPACE IF NOT EXISTS system");
+    }
+
+    try {
+      String[] namespace = {"system"};
+      String name = "iceberg_version";
+      Identifier identifier = Identifier.of(namespace, name);
+
+      assertListingLoadingAndBindingFrom(identifier);
+    } finally {
+      if (isSessionCatalog) {
+        sql("DROP NAMESPACE IF EXISTS system");
+      }
+    }
+  }
+
+  @Test
+  public void testLoadListAndUseFunctionsFromEmptyNamespace()
+      throws NoSuchFunctionException, NoSuchNamespaceException {
+    String[] namespace = {};
+    String name = "iceberg_version";
+    Identifier identifier = Identifier.of(namespace, name);
+
+    assertListingLoadingAndBindingFrom(identifier);
+  }
+
+  @Test
+  public void testCannotLoadFunctionsFromInvalidNamespace() {
+    AssertHelpers.assertThrows(
+        "Function Catalog functions should only be accessible from the system namespace and empty namespace",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT %s.iceberg_version()", fullNamespace));
+  }
+
+  @Test
+  public void testCannotUseUndefinedFunction() {
+    AssertHelpers.assertThrows(
+        "Using an undefined function should throw",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT undefined_function(1, 2)"));
+  }
+
+  private void assertListingLoadingAndBindingFrom(Identifier identifier)

Review Comment:
   This is partially because of oddities between what can be done with V2 functions in the session catalog and elsewhere (non-session catalogs).
   
   I addded the tests in these styles because this is how the storage partitioned joins function resolution actually happens. The function is created with a "fake" input Schema to addd `numBuckets` and then called through this pass.
   
   And then not testing certain other things through plain SQL is because of limitations / differences in what can be done in the session catalog vs elsewhere. Per the Spark 3.3 tests for example, only the session catalog can use describe (and my own findings).
   
   I'll try to refactor to be more in line with your thought though as I don't disagree 👍 



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#issuecomment-1201575072

   Merged. @kbendick, can you backport this to 3.2 as well?


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue merged PR #5377:
URL: https://github.com/apache/iceberg/pull/5377


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933566180


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** A function that gets registered to return the current Iceberg version in use from within SQL */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format("Cannot bind: %s does not accept arguments", name()));
+    }
+
+    return new IcebergVersionFunctionImpl();
+  }
+
+  @Override
+  public String description() {
+    return name() + " - Returns the runtime Iceberg version";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  /**
+   * Concrete implementation of {@link IcebergVersionFunction} that returns the short version string
+   * when called, e.g. {@code SELECT system.iceberg_version()} will produce results like "0.14.0" or
+   * "0.15.0-SNAPSHOT".

Review Comment:
   I think you can just remove this Javadoc.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933604537


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func).isExactlyInstanceOf(IcebergVersionFunction.class);
+      Assertions.assertThat(func.name()).asString().isEqualTo("iceberg_version");
+      Assertions.assertThat(func.description())
+          .containsIgnoringCase("Returns the runtime Iceberg version");
+    }
+  }
+
+  @Test
+  public void testBindAndUseFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction unboundFunc = asFunctionCatalog.loadFunction(identifier);
+
+      BoundFunction boundFunc = unboundFunc.bind(new StructType());
+      Assertions.assertThat(boundFunc.getClass().toString())
+          .containsIgnoringCase("IcebergVersionFunctionImpl");
+      Assertions.assertThat(boundFunc.canonicalName()).isEqualTo("iceberg.iceberg_version");
+      Assertions.assertThat(boundFunc.isResultNullable()).isFalse();
+      Assertions.assertThat(boundFunc.isDeterministic()).isTrue();
+
+      // Ensure the bound result produces the expected UTF8String type.
+      ScalarFunction<UTF8String> scalarFunc = (ScalarFunction<UTF8String>) boundFunc;
+      Assert.assertEquals(
+          UTF8String.fromString(IcebergBuild.version()),
+          scalarFunc.produceResult(InternalRow.empty()));
+    }
+  }
+
+  @Test
+  public void testCannotLoadFunctionsFromInvalidNamespace() {
+    String dbNamespace = catalogName + ".db";
+    try {
+      sql("CREATE NAMESPACE IF NOT EXISTS %s", dbNamespace);
+      AssertHelpers.assertThrows(
+          "Function Catalog functions should only be accessible from the system namespace and empty namespace",
+          AnalysisException.class,
+          "Undefined function",
+          () -> sql("SELECT %s.iceberg_version()", dbNamespace));

Review Comment:
   That's in `TestIcebergVersionFunction`. There's calls to it in SQL in every possible way.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933598153


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);

Review Comment:
   I have one for functions that don't exist, but adding one in here that's not SQL based.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934036450


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * A function for use in SQL that returns the current Iceberg version, e.g. {@code SELECT
+ * system.iceberg_version()} will return a String such as "0.14.0" or "0.15.0-SNAPSHOT"
+ */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format("Cannot bind: %s does not accept arguments", name()));
+    }
+
+    return new IcebergVersionFunctionImpl();
+  }
+
+  @Override
+  public String description() {
+    return name() + " - Returns the runtime Iceberg version";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  // class must be at least package private to be resolvable

Review Comment:
   What does "to be resolvable" mean?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932743323


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));
+    }
+
+    return new IcebergVersion();
+  }
+
+  @Override
+  public String description() {
+    return "iceberg_version - Returns the runtime Iceberg version for debugging purposes";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  public static class IcebergVersion implements ScalarFunction<String> {
+
+    private static final UTF8String icebergBuildVersion =
+        UTF8String.fromString(IcebergBuild.version());
+
+    public static UTF8String invoke() {
+      return icebergBuildVersion;
+    }
+
+    @Override
+    public DataType[] inputTypes() {
+      return new DataType[0];
+    }
+
+    @Override
+    public DataType resultType() {
+      return DataTypes.StringType;
+    }
+
+    @Override
+    public String canonicalName() {
+      return "org.apache.iceberg.spark.functions.iceberg_version()";
+    }
+
+    @Override
+    public String name() {
+      return "iceberg_version";
+    }
+
+    @Override
+    public String produceResult(InternalRow input) {

Review Comment:
   For the non-magic function, it uses the decoded / deserialized type vs the primitive type or the built-in boxed type (for `UTF8String` etc).
   
   this will actually never call `produceResult`, it should always use the magic function. But I’ve implemented it anyway in case somebody disables magic functions etc.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933603492


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);

Review Comment:
   Added one that's not SQL based for functions that don't exist and functions that do exist, but with a namespace other than empty namespace or `system`.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933621859


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** A function that gets registered to return the current Iceberg version in use from within SQL */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format("Cannot bind: %s does not accept arguments", name()));
+    }
+
+    return new IcebergVersionFunctionImpl();
+  }
+
+  @Override
+  public String description() {
+    return name() + " - Returns the runtime Iceberg version";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  /**
+   * Concrete implementation of {@link IcebergVersionFunction} that returns the short version string
+   * when called, e.g. {@code SELECT system.iceberg_version()} will produce results like "0.14.0" or
+   * "0.15.0-SNAPSHOT".
+   */
+  static class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {

Review Comment:
   Added



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933706520


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java:
##########
@@ -47,4 +56,38 @@ public Procedure loadProcedure(Identifier ident) throws NoSuchProcedureException
 
     throw new NoSuchProcedureException(ident);
   }
+
+  @Override
+  public Identifier[] listFunctions(String[] namespace) throws NoSuchNamespaceException {
+    if (namespace.length == 0
+        || (namespace.length == 1 && namespace[0].equalsIgnoreCase("system"))) {

Review Comment:
   nit: What about defining a helper function `isSystemNamespace`? There are 3 places in this class where we can call it.
   
   ```
   private boolean isSystemNamespace(String[] namespace) {
     return namespace.length == 1 && namespace[0].equalsIgnoreCase("system");
   }
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933576335


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** A function that gets registered to return the current Iceberg version in use from within SQL */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format("Cannot bind: %s does not accept arguments", name()));
+    }
+
+    return new IcebergVersionFunctionImpl();
+  }
+
+  @Override
+  public String description() {
+    return name() + " - Returns the runtime Iceberg version";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  /**
+   * Concrete implementation of {@link IcebergVersionFunction} that returns the short version string
+   * when called, e.g. {@code SELECT system.iceberg_version()} will produce results like "0.14.0" or
+   * "0.15.0-SNAPSHOT".

Review Comment:
   Yeah removed.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933472964


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format("Invalid input type. %s does not except arguments.", name()));

Review Comment:
   Rather than "Invalid input type" how about "Cannot bind: %s does not accept arguments". Also, "except" is a typo and should be either "accept" or "expect".



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933548579


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));
+    }
+
+    return new IcebergVersion();
+  }
+
+  @Override
+  public String description() {
+    return "iceberg_version - Returns the runtime Iceberg version for debugging purposes";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  public static class IcebergVersion implements ScalarFunction<String> {
+
+    private static final UTF8String icebergBuildVersion =
+        UTF8String.fromString(IcebergBuild.version());
+
+    public static UTF8String invoke() {
+      return icebergBuildVersion;
+    }
+
+    @Override
+    public DataType[] inputTypes() {
+      return new DataType[0];
+    }
+
+    @Override
+    public DataType resultType() {
+      return DataTypes.StringType;
+    }
+
+    @Override
+    public String canonicalName() {
+      return "org.apache.iceberg.spark.functions.iceberg_version()";
+    }
+
+    @Override
+    public String name() {
+      return "iceberg_version";
+    }
+
+    @Override
+    public String produceResult(InternalRow input) {

Review Comment:
   Sure. It is tested in the FunctionCatalog suite, but I'll make it more explicit via refactor.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933584405


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {

Review Comment:
   There is a test in there for empty function namespace. The first lines are checking that.
   
   ```
       Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
           .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934810401


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
+  private static final String[] DB_NAMESPACE = new String[] {"db"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog() {
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createDefaultNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
+  }
+
+  @After
+  public void dropDefaultNamespace() {
+    sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));

Review Comment:
   > I updated the tests, but built-in functions are case-insensitive by default.
   
   That doesn't mean that Spark will change the case of names that are supplied by `FunctionCatalog`. The problem isn't resolution being case insensitive, it is that this test case allows Spark to change the function name. I don't see a reason to allow that.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934036855


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
+  private static final String[] DB_NAMESPACE = new String[] {"db"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog() {
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createDefaultNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
+  }
+
+  @After
+  public void dropDefaultNamespace() {
+    sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));

Review Comment:
   Nit: I would not be permissive here. This lets Spark change the case of the function name, which we don't expect and would be really odd.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#issuecomment-1201598332

   > Merged. @kbendick, can you backport this to 3.2 as well?
   
   Working on it now!


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#issuecomment-1201618089

   New PR for Spark 3.2: https://github.com/apache/iceberg/pull/5411


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933513326


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunctionImpl.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * {@link org.apache.spark.sql.connector.catalog.functions.BoundFunction} that implements {@link
+ * org.apache.iceberg.spark.functions.IcebergVersionFunction} by returning the short version string,
+ * e.g. "0.14.0" or "0.15.0-SNAPSHOT".
+ */
+public class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {
+
+  private static final String icebergVersionAsString = IcebergBuild.version();

Review Comment:
   Yeah removed.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunctionImpl.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * {@link org.apache.spark.sql.connector.catalog.functions.BoundFunction} that implements {@link
+ * org.apache.iceberg.spark.functions.IcebergVersionFunction} by returning the short version string,
+ * e.g. "0.14.0" or "0.15.0-SNAPSHOT".
+ */
+public class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {
+
+  private static final String icebergVersionAsString = IcebergBuild.version();
+  private static final UTF8String icebergVersionAsNative =

Review Comment:
   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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932743323


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));
+    }
+
+    return new IcebergVersion();
+  }
+
+  @Override
+  public String description() {
+    return "iceberg_version - Returns the runtime Iceberg version for debugging purposes";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  public static class IcebergVersion implements ScalarFunction<String> {
+
+    private static final UTF8String icebergBuildVersion =
+        UTF8String.fromString(IcebergBuild.version());
+
+    public static UTF8String invoke() {
+      return icebergBuildVersion;
+    }
+
+    @Override
+    public DataType[] inputTypes() {
+      return new DataType[0];
+    }
+
+    @Override
+    public DataType resultType() {
+      return DataTypes.StringType;
+    }
+
+    @Override
+    public String canonicalName() {
+      return "org.apache.iceberg.spark.functions.iceberg_version()";
+    }
+
+    @Override
+    public String name() {
+      return "iceberg_version";
+    }
+
+    @Override
+    public String produceResult(InternalRow input) {

Review Comment:
   For the non-magic function, it uses the decoded / deserialized type vs the primitive type or the built-in boxed type (for `UTF8String` etc).
   
   Spark shouldn’t actually ever call `produceResult` - it should always use the magic function as this function pretty trivially meets the requirements for the usage of magic functions and codegen.
   
   But I’ve implemented it anyway in case somebody disables magic functions etc.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932742081


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));
+    }
+
+    return new IcebergVersion();
+  }
+
+  @Override
+  public String description() {
+    return "iceberg_version - Returns the runtime Iceberg version for debugging purposes";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  public static class IcebergVersion implements ScalarFunction<String> {
+
+    private static final UTF8String icebergBuildVersion =
+        UTF8String.fromString(IcebergBuild.version());
+
+    public static UTF8String invoke() {
+      return icebergBuildVersion;
+    }
+
+    @Override
+    public DataType[] inputTypes() {
+      return new DataType[0];
+    }
+
+    @Override
+    public DataType resultType() {
+      return DataTypes.StringType;
+    }
+
+    @Override
+    public String canonicalName() {
+      return "org.apache.iceberg.spark.functions.iceberg_version()";
+    }
+
+    @Override
+    public String name() {
+      return "iceberg_version";
+    }
+
+    @Override
+    public String produceResult(InternalRow input) {

Review Comment:
   Why does this return `String` instead of `UTF8String`?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932748952


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));

Review Comment:
   +1.
   
   I took this general check from a test in Spark, but field seems very weird without any expected input fields.
   
   It also seems Spark checks the inputTypes field length on your behalf (against all known implementations I guess.
   
   But the check still seems worthwhile as we can't be sure all Spark paths will do that for us as we've seem differences in v1 session catalog vs v2 session catalog vs v2 non-session catalog already.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932749591


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {SparkCatalogConfig.HADOOP}, {SparkCatalogConfig.HIVE}, {SparkCatalogConfig.SPARK}
+    };
+  }
+
+  private static final Namespace NS = Namespace.of("db");
+  private final boolean isSessionCatalog;
+  private final String fullNamespace;
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.isSessionCatalog = "spark_catalog".equals(catalogName);
+    this.fullNamespace = (isSessionCatalog ? "" : catalogName + ".") + NS;
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", fullNamespace);
+  }
+
+  @After
+  public void cleanNamespaces() {
+    sql("DROP NAMESPACE IF EXISTS %s", fullNamespace);
+  }
+
+  @Test
+  public void testLoadListAndUseFunctionsFromSystemNamespace()
+      throws NoSuchFunctionException, NoSuchNamespaceException {
+    // Session catalog requires that the namespace actually exists
+    if (isSessionCatalog) {
+      sql("CREATE NAMESPACE IF NOT EXISTS system");
+    }
+
+    try {
+      String[] namespace = {"system"};
+      String name = "iceberg_version";
+      Identifier identifier = Identifier.of(namespace, name);
+
+      assertListingLoadingAndBindingFrom(identifier);
+    } finally {
+      if (isSessionCatalog) {
+        sql("DROP NAMESPACE IF EXISTS system");
+      }
+    }
+  }
+
+  @Test
+  public void testLoadListAndUseFunctionsFromEmptyNamespace()
+      throws NoSuchFunctionException, NoSuchNamespaceException {
+    String[] namespace = {};
+    String name = "iceberg_version";
+    Identifier identifier = Identifier.of(namespace, name);
+
+    assertListingLoadingAndBindingFrom(identifier);
+  }
+
+  @Test
+  public void testCannotLoadFunctionsFromInvalidNamespace() {
+    AssertHelpers.assertThrows(
+        "Function Catalog functions should only be accessible from the system namespace and empty namespace",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT %s.iceberg_version()", fullNamespace));
+  }
+
+  @Test
+  public void testCannotUseUndefinedFunction() {
+    AssertHelpers.assertThrows(
+        "Using an undefined function should throw",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT undefined_function(1, 2)"));
+  }
+
+  private void assertListingLoadingAndBindingFrom(Identifier identifier)

Review Comment:
   What is the value of testing all of these things directly? I would expect these tests to run SQL, like `SELECT iceberg_version()` and verify the result.
   
   Think about what you need to validate in generic catalog-level tests. You need to validate that you can load and call a function, assuming that it works properly. You don't need to test the function itself (unbound to bound, for example) because that isn't the goal of this suite.
   
   I think that this suite should verify, using SQL, that you can call the `iceberg_version` method and it shows up in listings. You can also verify that it is in both `system` and the empty namespace. Beyond that, I don't think this needs to do much.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933515919


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {SparkCatalogConfig.HADOOP}, {SparkCatalogConfig.HIVE}, {SparkCatalogConfig.SPARK}
+    };
+  }
+
+  private static final Namespace NS = Namespace.of("db");
+  private final boolean isSessionCatalog;
+  private final String fullNamespace;
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.isSessionCatalog = "spark_catalog".equals(catalogName);
+    this.fullNamespace = (isSessionCatalog ? "" : catalogName + ".") + NS;

Review Comment:
   I noticed that in other tests, but you're right it's not necessary.
   
   I'm cleaning up this class, as it's not possible to `SHOW FUNCTIONS` or `DESCRIBE FUNCTIONS` for v2 functions via SQL (which matches up with the tests in Spark too). Still seeing if I can get around that a bit. I think I can still just call `list` etc (which is what it already had).



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933572526


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func).isExactlyInstanceOf(IcebergVersionFunction.class);
+      Assertions.assertThat(func.name()).asString().isEqualTo("iceberg_version");
+      Assertions.assertThat(func.description())
+          .containsIgnoringCase("Returns the runtime Iceberg version");
+    }
+  }
+
+  @Test
+  public void testBindAndUseFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction unboundFunc = asFunctionCatalog.loadFunction(identifier);
+
+      BoundFunction boundFunc = unboundFunc.bind(new StructType());
+      Assertions.assertThat(boundFunc.getClass().toString())
+          .containsIgnoringCase("IcebergVersionFunctionImpl");
+      Assertions.assertThat(boundFunc.canonicalName()).isEqualTo("iceberg.iceberg_version");
+      Assertions.assertThat(boundFunc.isResultNullable()).isFalse();
+      Assertions.assertThat(boundFunc.isDeterministic()).isTrue();
+
+      // Ensure the bound result produces the expected UTF8String type.
+      ScalarFunction<UTF8String> scalarFunc = (ScalarFunction<UTF8String>) boundFunc;
+      Assert.assertEquals(
+          UTF8String.fromString(IcebergBuild.version()),
+          scalarFunc.produceResult(InternalRow.empty()));
+    }
+  }
+
+  @Test
+  public void testCannotLoadFunctionsFromInvalidNamespace() {

Review Comment:
   I think this test should use `loadFunction` directly, like the others. I don't see why this one uses SQL while the other load and list tests use the API.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933574704


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));
+    }
+
+    return new IcebergVersion();
+  }
+
+  @Override
+  public String description() {
+    return "iceberg_version - Returns the runtime Iceberg version for debugging purposes";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  public static class IcebergVersion implements ScalarFunction<String> {
+
+    private static final UTF8String icebergBuildVersion =
+        UTF8String.fromString(IcebergBuild.version());
+
+    public static UTF8String invoke() {
+      return icebergBuildVersion;
+    }
+
+    @Override
+    public DataType[] inputTypes() {
+      return new DataType[0];
+    }
+
+    @Override
+    public DataType resultType() {
+      return DataTypes.StringType;
+    }
+
+    @Override
+    public String canonicalName() {
+      return "org.apache.iceberg.spark.functions.iceberg_version()";
+    }
+
+    @Override
+    public String name() {
+      return "iceberg_version";
+    }
+
+    @Override
+    public String produceResult(InternalRow input) {

Review Comment:
   I made the test for it much more explicit, using `Assert.assertEquals(UTF8String.fromString(IcebergBuild.version()), func.produceResult())`. Which fails if it's not UTF8String.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933568763


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};

Review Comment:
   To test with a single catalog, extend `SparkTestBaseWithCatalog` instead of `SparkCatalogTestBase`.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933632719


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func).isExactlyInstanceOf(IcebergVersionFunction.class);
+      Assertions.assertThat(func.name()).asString().isEqualTo("iceberg_version");
+      Assertions.assertThat(func.description())
+          .containsIgnoringCase("Returns the runtime Iceberg version");

Review Comment:
   I'll just check that it's not null. I just needed some sort of check really.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933604928


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func).isExactlyInstanceOf(IcebergVersionFunction.class);
+      Assertions.assertThat(func.name()).asString().isEqualTo("iceberg_version");
+      Assertions.assertThat(func.description())
+          .containsIgnoringCase("Returns the runtime Iceberg version");
+    }
+  }
+
+  @Test
+  public void testBindAndUseFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction unboundFunc = asFunctionCatalog.loadFunction(identifier);
+
+      BoundFunction boundFunc = unboundFunc.bind(new StructType());
+      Assertions.assertThat(boundFunc.getClass().toString())
+          .containsIgnoringCase("IcebergVersionFunctionImpl");
+      Assertions.assertThat(boundFunc.canonicalName()).isEqualTo("iceberg.iceberg_version");
+      Assertions.assertThat(boundFunc.isResultNullable()).isFalse();
+      Assertions.assertThat(boundFunc.isDeterministic()).isTrue();
+
+      // Ensure the bound result produces the expected UTF8String type.
+      ScalarFunction<UTF8String> scalarFunc = (ScalarFunction<UTF8String>) boundFunc;
+      Assert.assertEquals(
+          UTF8String.fromString(IcebergBuild.version()),
+          scalarFunc.produceResult(InternalRow.empty()));
+    }
+  }
+
+  @Test
+  public void testCannotLoadFunctionsFromInvalidNamespace() {
+    String dbNamespace = catalogName + ".db";
+    try {
+      sql("CREATE NAMESPACE IF NOT EXISTS %s", dbNamespace);
+      AssertHelpers.assertThrows(
+          "Function Catalog functions should only be accessible from the system namespace and empty namespace",
+          AnalysisException.class,
+          "Undefined function",
+          () -> sql("SELECT %s.iceberg_version()", dbNamespace));

Review Comment:
   Oh I see your note about moving `TestIcebergVersionFunction` over. Will do.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934046314


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
+  private static final String[] DB_NAMESPACE = new String[] {"db"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog() {
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createDefaultNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
+  }
+
+  @After
+  public void dropDefaultNamespace() {
+    sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assert.assertArrayEquals(
+        "Listing functions in an existing namespace that's not system should not throw",
+        new Identifier[0],
+        asFunctionCatalog.listFunctions(DEFAULT_NAMESPACE));
+
+    AssertHelpers.assertThrows(
+        "Listing functions in a namespace that does not exist should throw",
+        NoSuchNamespaceException.class,
+        "Namespace 'db' not found",
+        () -> asFunctionCatalog.listFunctions(DB_NAMESPACE));
+  }
+
+  @Test
+  public void testLoadFunctions() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func)
+          .isNotNull()
+          .isInstanceOf(UnboundFunction.class)
+          .isExactlyInstanceOf(IcebergVersionFunction.class);
+    }
+
+    AssertHelpers.assertThrows(
+        "Cannot load a function if it's not used with the system namespace or the empty namespace",
+        NoSuchFunctionException.class,
+        "Undefined function: default.iceberg_version",
+        () -> asFunctionCatalog.loadFunction(Identifier.of(DEFAULT_NAMESPACE, "iceberg_version")));
+
+    AssertHelpers.assertThrows(
+        "Cannot load a function that does not exist",
+        NoSuchFunctionException.class,
+        "Undefined function: system.undefined_function",
+        () ->
+            asFunctionCatalog.loadFunction(Identifier.of(SYSTEM_NAMESPACE, "undefined_function")));
+
+    AssertHelpers.assertThrows(
+        "Using an undefined function from SQL should fail analysis",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT undefined_function(1, 2)"));
+  }
+
+  @Test
+  public void testIcebergVersionFunctionReturnsCorrectResult() {
+    String buildVersion = IcebergBuild.version();
+
+    // Check that value correctly loads from file, even in codegen etc
+    Assert.assertNotEquals(
+        "iceberg_version should not return undefined",
+        "undefined",
+        scalarSql("SELECT %s.system.iceberg_version()", catalogName));

Review Comment:
   I added this check because the `IcebergBuild.version()` falls back to `undefined` if the file cannot be resolved.
   
   I am essentially testing that the function implementation that gets bound at runtime to `iceberg_version` picks up the correct `IcebergBuild.version()`, though to really test what I want to test it would probably make more sense to add this test in a distributed integration test (i.e. that executors don't receive code that cannot access the value).
   
   Given we already have a test for `IcebergBuild.version()` elsewhere though, this is not necessary.
   
   I'll remove it.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933560982


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunctionImpl.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * {@link org.apache.spark.sql.connector.catalog.functions.BoundFunction} that implements {@link
+ * org.apache.iceberg.spark.functions.IcebergVersionFunction} by returning the short version string,
+ * e.g. "0.14.0" or "0.15.0-SNAPSHOT".
+ */
+public class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {

Review Comment:
   And then I'll see about the `public` or not after moving it back.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934036775


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
+  private static final String[] DB_NAMESPACE = new String[] {"db"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog() {
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createDefaultNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
+  }
+
+  @After
+  public void dropDefaultNamespace() {
+    sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assert.assertArrayEquals(
+        "Listing functions in an existing namespace that's not system should not throw",
+        new Identifier[0],
+        asFunctionCatalog.listFunctions(DEFAULT_NAMESPACE));
+
+    AssertHelpers.assertThrows(
+        "Listing functions in a namespace that does not exist should throw",
+        NoSuchNamespaceException.class,
+        "Namespace 'db' not found",
+        () -> asFunctionCatalog.listFunctions(DB_NAMESPACE));
+  }
+
+  @Test
+  public void testLoadFunctions() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func)
+          .isNotNull()
+          .isInstanceOf(UnboundFunction.class)
+          .isExactlyInstanceOf(IcebergVersionFunction.class);
+    }
+
+    AssertHelpers.assertThrows(
+        "Cannot load a function if it's not used with the system namespace or the empty namespace",
+        NoSuchFunctionException.class,
+        "Undefined function: default.iceberg_version",
+        () -> asFunctionCatalog.loadFunction(Identifier.of(DEFAULT_NAMESPACE, "iceberg_version")));
+
+    AssertHelpers.assertThrows(
+        "Cannot load a function that does not exist",
+        NoSuchFunctionException.class,
+        "Undefined function: system.undefined_function",
+        () ->
+            asFunctionCatalog.loadFunction(Identifier.of(SYSTEM_NAMESPACE, "undefined_function")));
+
+    AssertHelpers.assertThrows(
+        "Using an undefined function from SQL should fail analysis",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT undefined_function(1, 2)"));
+  }
+
+  @Test
+  public void testIcebergVersionFunctionReturnsCorrectResult() {

Review Comment:
   Nit: "ReturnsCorrectResult" is an assumption of most tests. Make sure names only include what is needed to understand the purpose of the test.
   
   In this case, the purpose of the test is to test function usage end-to-end, through SQL.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934048717


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
+  private static final String[] DB_NAMESPACE = new String[] {"db"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog() {
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createDefaultNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
+  }
+
+  @After
+  public void dropDefaultNamespace() {
+    sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));

Review Comment:
   I updated the tests, but built-in functions are case-insensitive by default.
   
   Tests done on Spark 3.3 with `spark.sql.caseSensitive` set to both `true` and `false` (made no difference either way).
   
   ```scala
   scala> spark.sql("SELECT uuid() as _uuid").show()
   +--------------------+
   |               _uuid|
   +--------------------+
   |3d52c2c7-225c-44c...|
   +--------------------+
   
   
   scala> spark.sql("SELECT UUID() as _uuid").show()
   +--------------------+
   |               _uuid|
   +--------------------+
   |1babdfb6-1f71-498...|
   +--------------------+
   
   
   scala> spark.sql("SELECT UuID() as _uuid").show()
   +--------------------+
   |               _uuid|
   +--------------------+
   |d63892a1-e2e6-49d...|
   +--------------------+
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933495868


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {

Review Comment:
   Yeah. I reverted it from 1 back to 3 momentarily to test refactoring this class. I'll revert it back once I'm done and ping you again. Sorry for the confusion.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933567123


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** A function that gets registered to return the current Iceberg version in use from within SQL */

Review Comment:
   What does "that gets registered" mean? Is this _always_ used from SQL?
   
   When writing docs, be brief. The Iceberg version function doesn't need to tell the reader how functions are exposed to Spark. It just needs to tell the reader what the purpose of this class is and any relevant things about using it directly (which won't happen).



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933620930


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {

Review Comment:
   I added that too inside this suite.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933721234


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java:
##########
@@ -47,4 +56,38 @@ public Procedure loadProcedure(Identifier ident) throws NoSuchProcedureException
 
     throw new NoSuchProcedureException(ident);
   }
+
+  @Override
+  public Identifier[] listFunctions(String[] namespace) throws NoSuchNamespaceException {
+    if (namespace.length == 0
+        || (namespace.length == 1 && namespace[0].equalsIgnoreCase("system"))) {

Review Comment:
   Cool. Updated all 3 usages in this file. This helps with the funky formatting too. Thank you!



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933441146


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));
+    }
+
+    return new IcebergVersion();
+  }
+
+  @Override
+  public String description() {
+    return "iceberg_version - Returns the runtime Iceberg version for debugging purposes";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  public static class IcebergVersion implements ScalarFunction<String> {
+
+    private static final UTF8String icebergBuildVersion =
+        UTF8String.fromString(IcebergBuild.version());
+
+    public static UTF8String invoke() {
+      return icebergBuildVersion;
+    }
+
+    @Override
+    public DataType[] inputTypes() {
+      return new DataType[0];
+    }
+
+    @Override
+    public DataType resultType() {
+      return DataTypes.StringType;
+    }
+
+    @Override
+    public String canonicalName() {
+      return "org.apache.iceberg.spark.functions.iceberg_version()";
+    }
+
+    @Override
+    public String name() {
+      return "iceberg_version";
+    }
+
+    @Override
+    public String produceResult(InternalRow input) {

Review Comment:
   > According to Javadoc, the [Spark data type to Java type mapping](https://spark.apache.org/docs/latest/api/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.html) applies to the invoke functions as well as the values returned by produceResult.
   
   Oh I see you're right. Let me look at the plan and see what it's doing. Based on the test implementations I saw, I thought that `String` was used. It would be easier if `UTF8String` is used.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933470643


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunctionImpl.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * {@link org.apache.spark.sql.connector.catalog.functions.BoundFunction} that implements {@link
+ * org.apache.iceberg.spark.functions.IcebergVersionFunction} by returning the short version string,

Review Comment:
   We prefer not to use fully qualified names in docs. And trying to link to classes outside of Iceberg will fail.
   
   Try to write javadoc so that it has a useful level of detail for someone trying to understand what the class does, rather than the details of how it happens. I don't think that mentioning the interfaces that this implements is useful to the reader, and if it is then it is available in other ways.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933590481


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func).isExactlyInstanceOf(IcebergVersionFunction.class);
+      Assertions.assertThat(func.name()).asString().isEqualTo("iceberg_version");
+      Assertions.assertThat(func.description())
+          .containsIgnoringCase("Returns the runtime Iceberg version");

Review Comment:
   I don't think we want to exercise bind ourselves. For that we should go through Spark and make sure Spark is binding correctly. This isn't something we need to do for the version function, too.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933468183


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/SparkFunctions.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.functions;
+
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+
+public class SparkFunctions {
+
+  private SparkFunctions() {}
+
+  private static final Map<String, UnboundFunction> FUNCTIONS =
+      ImmutableMap.of("iceberg_version", new IcebergVersionFunction());
+
+  private static final List<String> FUNCTION_NAMES = ImmutableList.copyOf(FUNCTIONS.keySet());
+
+  // Functions that are added to all Iceberg catalogs should be accessed with the `system`
+  // namespace. They can also be accessed with no namespace at all if qualified with the
+  // catalog name, e.g. my_hadoop_catalog.iceberg_version().
+  // As namespace resolution is handled by those rules in BaseCatalog, a list of names
+  // alone is returned.
+  public static List<String> list() {
+    return FUNCTION_NAMES;
+  }
+
+  public static UnboundFunction load(String name) {
+    // function resolution is case-insensitive to match the existing Spark behavior for functions
+    UnboundFunction func = FUNCTIONS.get(name.toLowerCase(Locale.ROOT));
+    return func;

Review Comment:
   Nit: unnecessary local variable, `func`.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r934045366


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFunctionCatalog extends SparkTestBaseWithCatalog {
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private static final String[] DEFAULT_NAMESPACE = new String[] {"default"};
+  private static final String[] DB_NAMESPACE = new String[] {"db"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog() {
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createDefaultNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default");
+  }
+
+  @After
+  public void dropDefaultNamespace() {
+    sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default");
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));

Review Comment:
   I chose this as function resolution is case-insensitive in Spark, at least according to the existing code for `Procedure`'s. I can update it though.
   
   https://github.com/apache/iceberg/blob/14f4bc1ebe021bce35b756685a88026d85dd6025/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java#L34-L37



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932764387


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {SparkCatalogConfig.HADOOP}, {SparkCatalogConfig.HIVE}, {SparkCatalogConfig.SPARK}
+    };
+  }
+
+  private static final Namespace NS = Namespace.of("db");
+  private final boolean isSessionCatalog;
+  private final String fullNamespace;
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.isSessionCatalog = "spark_catalog".equals(catalogName);
+    this.fullNamespace = (isSessionCatalog ? "" : catalogName + ".") + NS;
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", fullNamespace);
+  }
+
+  @After
+  public void cleanNamespaces() {
+    sql("DROP NAMESPACE IF EXISTS %s", fullNamespace);
+  }
+
+  @Test
+  public void testLoadListAndUseFunctionsFromSystemNamespace()
+      throws NoSuchFunctionException, NoSuchNamespaceException {
+    // Session catalog requires that the namespace actually exists
+    if (isSessionCatalog) {

Review Comment:
   Sure.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932746687


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {

Review Comment:
   Sure.
   
   I wanted to test session catalog vs non session catalog as non-session catalog is somewhat different, mostly because calling functions in SQL on the session catalog uses v1 behavior and ensures the namespace exists.
   
   But I’ve added a check if in session catalog and create `session` namespace block and will leave that in as documentation & run then run this code as session catalog.
   
   3 is too many for sure.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932750478


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestIcebergVersionFunction.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.iceberg.spark.sql;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestIcebergVersionFunction extends SparkTestBaseWithCatalog {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}, {SparkCatalogConfig.HIVE}};

Review Comment:
   I think this is a bit too much for the version function. Think about what you need to test here. This suite is validating the behavior of the `iceberg_version` function. It doesn't need to test across multiple catalogs, especially when those catalogs use the same Spark wrapper.
   
   Testing both SparkCatalog and SparkSessionCatalog is a good idea, but only for the generic FunctionCatalog tests... and we don't even need to do that to begin with.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932784374


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {SparkCatalogConfig.HADOOP}, {SparkCatalogConfig.HIVE}, {SparkCatalogConfig.SPARK}
+    };
+  }
+
+  private static final Namespace NS = Namespace.of("db");
+  private final boolean isSessionCatalog;
+  private final String fullNamespace;
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.isSessionCatalog = "spark_catalog".equals(catalogName);
+    this.fullNamespace = (isSessionCatalog ? "" : catalogName + ".") + NS;
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", fullNamespace);
+  }
+
+  @After
+  public void cleanNamespaces() {
+    sql("DROP NAMESPACE IF EXISTS %s", fullNamespace);
+  }
+
+  @Test
+  public void testLoadListAndUseFunctionsFromSystemNamespace()
+      throws NoSuchFunctionException, NoSuchNamespaceException {
+    // Session catalog requires that the namespace actually exists
+    if (isSessionCatalog) {
+      sql("CREATE NAMESPACE IF NOT EXISTS system");
+    }
+
+    try {
+      String[] namespace = {"system"};
+      String name = "iceberg_version";
+      Identifier identifier = Identifier.of(namespace, name);
+
+      assertListingLoadingAndBindingFrom(identifier);
+    } finally {
+      if (isSessionCatalog) {
+        sql("DROP NAMESPACE IF EXISTS system");
+      }
+    }
+  }
+
+  @Test
+  public void testLoadListAndUseFunctionsFromEmptyNamespace()
+      throws NoSuchFunctionException, NoSuchNamespaceException {
+    String[] namespace = {};
+    String name = "iceberg_version";
+    Identifier identifier = Identifier.of(namespace, name);
+
+    assertListingLoadingAndBindingFrom(identifier);
+  }
+
+  @Test
+  public void testCannotLoadFunctionsFromInvalidNamespace() {
+    AssertHelpers.assertThrows(
+        "Function Catalog functions should only be accessible from the system namespace and empty namespace",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT %s.iceberg_version()", fullNamespace));
+  }
+
+  @Test
+  public void testCannotUseUndefinedFunction() {
+    AssertHelpers.assertThrows(
+        "Using an undefined function should throw",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT undefined_function(1, 2)"));
+  }
+
+  private void assertListingLoadingAndBindingFrom(Identifier identifier)

Review Comment:
   In Spark's `DataSourceV2FunctionSuite` for example, they have the following test (which is similar to the results I get) https://github.com/apache/spark/blob/144d4c546f7023b20e07619134feca1a46017a5f/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala#L157-L169
   
   But I'll refactor to be more like that suite and what you mentioned. I don't disagree by any means with what you say.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932784374


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+      {SparkCatalogConfig.HADOOP}, {SparkCatalogConfig.HIVE}, {SparkCatalogConfig.SPARK}
+    };
+  }
+
+  private static final Namespace NS = Namespace.of("db");
+  private final boolean isSessionCatalog;
+  private final String fullNamespace;
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.isSessionCatalog = "spark_catalog".equals(catalogName);
+    this.fullNamespace = (isSessionCatalog ? "" : catalogName + ".") + NS;
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Before
+  public void createNamespace() {
+    sql("CREATE NAMESPACE IF NOT EXISTS %s", fullNamespace);
+  }
+
+  @After
+  public void cleanNamespaces() {
+    sql("DROP NAMESPACE IF EXISTS %s", fullNamespace);
+  }
+
+  @Test
+  public void testLoadListAndUseFunctionsFromSystemNamespace()
+      throws NoSuchFunctionException, NoSuchNamespaceException {
+    // Session catalog requires that the namespace actually exists
+    if (isSessionCatalog) {
+      sql("CREATE NAMESPACE IF NOT EXISTS system");
+    }
+
+    try {
+      String[] namespace = {"system"};
+      String name = "iceberg_version";
+      Identifier identifier = Identifier.of(namespace, name);
+
+      assertListingLoadingAndBindingFrom(identifier);
+    } finally {
+      if (isSessionCatalog) {
+        sql("DROP NAMESPACE IF EXISTS system");
+      }
+    }
+  }
+
+  @Test
+  public void testLoadListAndUseFunctionsFromEmptyNamespace()
+      throws NoSuchFunctionException, NoSuchNamespaceException {
+    String[] namespace = {};
+    String name = "iceberg_version";
+    Identifier identifier = Identifier.of(namespace, name);
+
+    assertListingLoadingAndBindingFrom(identifier);
+  }
+
+  @Test
+  public void testCannotLoadFunctionsFromInvalidNamespace() {
+    AssertHelpers.assertThrows(
+        "Function Catalog functions should only be accessible from the system namespace and empty namespace",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT %s.iceberg_version()", fullNamespace));
+  }
+
+  @Test
+  public void testCannotUseUndefinedFunction() {
+    AssertHelpers.assertThrows(
+        "Using an undefined function should throw",
+        AnalysisException.class,
+        "Undefined function",
+        () -> sql("SELECT undefined_function(1, 2)"));
+  }
+
+  private void assertListingLoadingAndBindingFrom(Identifier identifier)

Review Comment:
   In Spark's `DataSourceV2FunctionSuite` for example, they have the [following test ](https://github.com/apache/spark/blob/144d4c546f7023b20e07619134feca1a46017a5f/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala#L157-L169) showing that `DESCRIBE` (and list) only work on the session catalog which mirrors the results I get.
   
   ````
     test("DESCRIBE FUNCTION: only support session catalog") {
       addFunction(Identifier.of(Array.empty, "abc"), new JavaStrLen(new JavaStrLenNoImpl))
   
       val e = intercept[AnalysisException] {
         sql("DESCRIBE FUNCTION testcat.abc")
       }
       assert(e.message.contains("Catalog testcat does not support functions"))
   
       val e1 = intercept[AnalysisException] {
         sql("DESCRIBE FUNCTION default.ns1.ns2.fun")
       }
       assert(e1.message.contains("requires a single-part namespace"))
     }
   ````
   
   But I'll refactor to be more like that suite and what you mentioned. I don't disagree by any means with what you say.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933573865


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func).isExactlyInstanceOf(IcebergVersionFunction.class);
+      Assertions.assertThat(func.name()).asString().isEqualTo("iceberg_version");
+      Assertions.assertThat(func.description())
+          .containsIgnoringCase("Returns the runtime Iceberg version");
+    }
+  }
+
+  @Test
+  public void testBindAndUseFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction unboundFunc = asFunctionCatalog.loadFunction(identifier);
+
+      BoundFunction boundFunc = unboundFunc.bind(new StructType());
+      Assertions.assertThat(boundFunc.getClass().toString())
+          .containsIgnoringCase("IcebergVersionFunctionImpl");
+      Assertions.assertThat(boundFunc.canonicalName()).isEqualTo("iceberg.iceberg_version");
+      Assertions.assertThat(boundFunc.isResultNullable()).isFalse();
+      Assertions.assertThat(boundFunc.isDeterministic()).isTrue();
+
+      // Ensure the bound result produces the expected UTF8String type.
+      ScalarFunction<UTF8String> scalarFunc = (ScalarFunction<UTF8String>) boundFunc;
+      Assert.assertEquals(
+          UTF8String.fromString(IcebergBuild.version()),
+          scalarFunc.produceResult(InternalRow.empty()));
+    }
+  }
+
+  @Test
+  public void testCannotLoadFunctionsFromInvalidNamespace() {
+    String dbNamespace = catalogName + ".db";
+    try {
+      sql("CREATE NAMESPACE IF NOT EXISTS %s", dbNamespace);
+      AssertHelpers.assertThrows(
+          "Function Catalog functions should only be accessible from the system namespace and empty namespace",
+          AnalysisException.class,
+          "Undefined function",
+          () -> sql("SELECT %s.iceberg_version()", dbNamespace));

Review Comment:
   I'd also like to have a SQL test that calls `iceberg_version()` because we need to test the end-to-end behavior, not just the catalog implementation directly.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933572002


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {

Review Comment:
   I think there should also be a case for an empty function namespace.
   
   You may want to test with coverage turned on in IntelliJ. That can help you spot code that is untested.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933574715


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestIcebergVersionFunction.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.iceberg.spark.sql;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.spark.SparkCatalogConfig;
+import org.apache.iceberg.spark.SparkTestBaseWithCatalog;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestIcebergVersionFunction extends SparkTestBaseWithCatalog {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.SPARK}};
+  }
+
+  private final String fullyQualifiedSystemNamespace;
+
+  public TestIcebergVersionFunction(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.fullyQualifiedSystemNamespace = catalogConfig.catalogName() + ".system";
+  }
+
+  @Test
+  public void testIcebergVersionFunctionReturnsCorrectResult() {
+    // Check that value correctly loads from file, even in codegen etc
+    String buildVersion = IcebergBuild.version();
+    Assert.assertNotEquals(
+        "iceberg_version should not return undefined",
+        "undefined",
+        scalarSql("SELECT %s.iceberg_version()", fullyQualifiedSystemNamespace));
+
+    Assert.assertEquals(
+        "Should be able to use the Iceberg version function from the fully qualified system namespace",
+        buildVersion,
+        scalarSql("SELECT %s.iceberg_version()", fullyQualifiedSystemNamespace));

Review Comment:
   Why not `SELECT %s.system.iceberg_version()`? I don't see a need to have a separate variable for the fully-qualified namespace.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933472261


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));
+    }
+
+    return new IcebergVersion();
+  }
+
+  @Override
+  public String description() {
+    return "iceberg_version - Returns the runtime Iceberg version for debugging purposes";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  public static class IcebergVersion implements ScalarFunction<String> {
+
+    private static final UTF8String icebergBuildVersion =
+        UTF8String.fromString(IcebergBuild.version());
+
+    public static UTF8String invoke() {
+      return icebergBuildVersion;
+    }
+
+    @Override
+    public DataType[] inputTypes() {
+      return new DataType[0];
+    }
+
+    @Override
+    public DataType resultType() {
+      return DataTypes.StringType;
+    }
+
+    @Override
+    public String canonicalName() {
+      return "org.apache.iceberg.spark.functions.iceberg_version()";
+    }
+
+    @Override
+    public String name() {
+      return "iceberg_version";
+    }
+
+    @Override
+    public String produceResult(InternalRow input) {

Review Comment:
   Can you also make sure that this is tested?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933470963


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunctionImpl.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * {@link org.apache.spark.sql.connector.catalog.functions.BoundFunction} that implements {@link
+ * org.apache.iceberg.spark.functions.IcebergVersionFunction} by returning the short version string,
+ * e.g. "0.14.0" or "0.15.0-SNAPSHOT".
+ */
+public class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {

Review Comment:
   Why does this need to be public? I think this could easily be a private static class inside of `IcebergVersionFunction`.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932764137


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));
+    }
+
+    return new IcebergVersion();
+  }
+
+  @Override
+  public String description() {
+    return "iceberg_version - Returns the runtime Iceberg version for debugging purposes";

Review Comment:
   Updated and using `name()`



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933581517


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.iceberg.spark;
+
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.functions.IcebergVersionFunction;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {
+
+  @Parameterized.Parameters(name = "catalogConfig = {0}")
+  public static Object[][] parameters() {
+    return new Object[][] {{SparkCatalogConfig.HADOOP}};
+  }
+
+  private static final String[] EMPTY_NAMESPACE = new String[] {};
+  private static final String[] SYSTEM_NAMESPACE = new String[] {"system"};
+  private final FunctionCatalog asFunctionCatalog;
+
+  public TestFunctionCatalog(SparkCatalogConfig catalogConfig) {
+    super(catalogConfig);
+    this.asFunctionCatalog = castToFunctionCatalog(catalogName);
+  }
+
+  @Test
+  public void testListFunctionsViaCatalog() throws NoSuchNamespaceException {
+    Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+
+    Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE))
+        .anyMatch(func -> "iceberg_version".equalsIgnoreCase(func.name()));
+  }
+
+  @Test
+  public void testLoadAndDescribeUnboundFunctionsViaCatalog() throws NoSuchFunctionException {
+    for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) {
+      Identifier identifier = Identifier.of(namespace, "iceberg_version");
+      UnboundFunction func = asFunctionCatalog.loadFunction(identifier);
+
+      Assertions.assertThat(func).isExactlyInstanceOf(IcebergVersionFunction.class);
+      Assertions.assertThat(func.name()).asString().isEqualTo("iceberg_version");
+      Assertions.assertThat(func.description())
+          .containsIgnoringCase("Returns the runtime Iceberg version");

Review Comment:
   Yeah I can remove these and maybe move them over to `TestIcebergVersionFunction`.
   
   What I really wanted was to test `DESCRIBE FUNCTION ...`, which arguably would be testing more the catalog. But that _doesn't_ work. So I settled on this. I can remove them or move them to `TestIcebergVersionFunction` if we'd like.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933556145


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {

Review Comment:
   Ok it's back at just one.
   
   A lot of the `FunctionCatalog` / v2 functions cannot use many DDL statements like `SHOW FUNCTIONS` and `DESCRIBE FUNCTION`.
   
   This is in-line with what's in the V2CatalogSuite in Spark as well, so I've added the tests via just calling the code directly.
   
   Given that's generally how Spark uses the function catalog functions, in addition to resolving them for usage in SQL, that is the best we can do.
   
   However, I scrapped the old large test and broke it up so it should be much more readable now.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933560292


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunctionImpl.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * {@link org.apache.spark.sql.connector.catalog.functions.BoundFunction} that implements {@link
+ * org.apache.iceberg.spark.functions.IcebergVersionFunction} by returning the short version string,

Review Comment:
   Updated this. I think it's better now. Doesn't reference interfaces and only mentions what it does.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
kbendick commented on PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#issuecomment-1200528737

   @rdblue I addressed your most recent comments. Can you please take a look?


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932743281


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));
+    }
+
+    return new IcebergVersion();
+  }
+
+  @Override
+  public String description() {
+    return "iceberg_version - Returns the runtime Iceberg version for debugging purposes";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  public static class IcebergVersion implements ScalarFunction<String> {
+
+    private static final UTF8String icebergBuildVersion =
+        UTF8String.fromString(IcebergBuild.version());
+
+    public static UTF8String invoke() {
+      return icebergBuildVersion;
+    }
+
+    @Override
+    public DataType[] inputTypes() {
+      return new DataType[0];
+    }
+
+    @Override
+    public DataType resultType() {
+      return DataTypes.StringType;
+    }
+
+    @Override
+    public String canonicalName() {
+      return "org.apache.iceberg.spark.functions.iceberg_version()";
+    }
+
+    @Override
+    public String name() {
+      return "iceberg_version";
+    }
+
+    @Override
+    public String produceResult(InternalRow input) {

Review Comment:
   According to Javadoc, the [Spark data type to Java type mapping](https://spark.apache.org/docs/latest/api/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.html) applies to the `invoke` functions as well as the values returned by `produceResult`.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r932742201


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.BoundFunction;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/** Implementation of {@link UnboundFunction} that prints the current Iceberg version in use. */
+public class IcebergVersionFunction implements UnboundFunction {
+  @Override
+  public BoundFunction bind(StructType inputType) {
+    if (inputType.fields().length > 0) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Invalid input type. Expected 0 fields but found %s", inputType.fields().length));
+    }
+
+    return new IcebergVersion();
+  }
+
+  @Override
+  public String description() {
+    return "iceberg_version - Returns the runtime Iceberg version for debugging purposes";
+  }
+
+  @Override
+  public String name() {
+    return "iceberg_version";
+  }
+
+  public static class IcebergVersion implements ScalarFunction<String> {
+
+    private static final UTF8String icebergBuildVersion =
+        UTF8String.fromString(IcebergBuild.version());
+
+    public static UTF8String invoke() {
+      return icebergBuildVersion;
+    }
+
+    @Override
+    public DataType[] inputTypes() {
+      return new DataType[0];
+    }
+
+    @Override
+    public DataType resultType() {
+      return DataTypes.StringType;
+    }
+
+    @Override
+    public String canonicalName() {
+      return "org.apache.iceberg.spark.functions.iceberg_version()";

Review Comment:
   How about `iceberg.iceberg_version`?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933475400


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunctionImpl.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.iceberg.spark.functions;
+
+import org.apache.iceberg.IcebergBuild;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * {@link org.apache.spark.sql.connector.catalog.functions.BoundFunction} that implements {@link
+ * org.apache.iceberg.spark.functions.IcebergVersionFunction} by returning the short version string,
+ * e.g. "0.14.0" or "0.15.0-SNAPSHOT".
+ */
+public class IcebergVersionFunctionImpl implements ScalarFunction<UTF8String> {
+
+  private static final String icebergVersionAsString = IcebergBuild.version();

Review Comment:
   I don't think that this is needed. It's only used to create the Spark string, and it is available from `IcebergBuild`. Plus, this isn't converting the value to another representation so I don't think you need an "as string" form.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #5377: Spark - Add Spark FunctionCatalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #5377:
URL: https://github.com/apache/iceberg/pull/5377#discussion_r933476019


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.iceberg.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.iceberg.AssertHelpers;
+import org.apache.iceberg.IcebergBuild;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.spark.functions.SparkFunctions;
+import org.apache.spark.sql.AnalysisException;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.connector.catalog.FunctionCatalog;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.functions.ScalarFunction;
+import org.apache.spark.sql.connector.catalog.functions.UnboundFunction;
+import org.apache.spark.sql.types.StructType;
+import org.assertj.core.api.Assertions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TestFunctionCatalog extends SparkCatalogTestBase {

Review Comment:
   @kbendick, can you update this test?



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org