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/06/20 22:10:39 UTC

[GitHub] [iceberg] rdblue commented on a diff in pull request #5094: Spark: Add Spark 3.2 copy on top of Spark 3.3

rdblue commented on code in PR #5094:
URL: https://github.com/apache/iceberg/pull/5094#discussion_r902004836


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java:
##########
@@ -0,0 +1,651 @@
+/*
+ * 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.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.CachingCatalog;
+import org.apache.iceberg.CatalogProperties;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.Transaction;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.SupportsNamespaces;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Splitter;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.spark.actions.SparkActions;
+import org.apache.iceberg.spark.source.SparkTable;
+import org.apache.iceberg.spark.source.StagedSparkTable;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.catalyst.analysis.NonEmptyNamespaceException;
+import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.NamespaceChange;
+import org.apache.spark.sql.connector.catalog.StagedTable;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.catalog.TableChange;
+import org.apache.spark.sql.connector.catalog.TableChange.ColumnChange;
+import org.apache.spark.sql.connector.catalog.TableChange.RemoveProperty;
+import org.apache.spark.sql.connector.catalog.TableChange.SetProperty;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+import static org.apache.iceberg.TableProperties.GC_ENABLED;
+import static org.apache.iceberg.TableProperties.GC_ENABLED_DEFAULT;
+
+/**
+ * A Spark TableCatalog implementation that wraps an Iceberg {@link Catalog}.
+ * <p>
+ * This supports the following catalog configuration options:
+ * <ul>
+ *   <li><code>type</code> - catalog type, "hive" or "hadoop"</li>
+ *   <li><code>uri</code> - the Hive Metastore URI (Hive catalog only)</li>
+ *   <li><code>warehouse</code> - the warehouse path (Hadoop catalog only)</li>
+ *   <li><code>default-namespace</code> - a namespace to use as the default</li>
+ *   <li><code>cache-enabled</code> - whether to enable catalog cache</li>
+ *   <li><code>cache.expiration-interval-ms</code> - interval in millis before expiring tables from catalog cache.
+ *       Refer to {@link CatalogProperties#CACHE_EXPIRATION_INTERVAL_MS} for further details and significant values.
+ *   </li>
+ * </ul>
+ * <p>
+ * To use a custom catalog that is not a Hive or Hadoop catalog, extend this class and override
+ * {@link #buildIcebergCatalog(String, CaseInsensitiveStringMap)}.
+ */
+public class SparkCatalog extends BaseCatalog {
+  private static final Set<String> DEFAULT_NS_KEYS = ImmutableSet.of(TableCatalog.PROP_OWNER);
+  private static final Splitter COMMA = Splitter.on(",");
+  private static final Pattern AT_TIMESTAMP = Pattern.compile("at_timestamp_(\\d+)");
+  private static final Pattern SNAPSHOT_ID = Pattern.compile("snapshot_id_(\\d+)");
+
+  private String catalogName = null;
+  private Catalog icebergCatalog = null;
+  private boolean cacheEnabled = CatalogProperties.CACHE_ENABLED_DEFAULT;
+  private SupportsNamespaces asNamespaceCatalog = null;
+  private String[] defaultNamespace = null;
+  private HadoopTables tables;
+  private boolean useTimestampsWithoutZone;
+
+  /**
+   * Build an Iceberg {@link Catalog} to be used by this Spark catalog adapter.
+   *
+   * @param name Spark's catalog name
+   * @param options Spark's catalog options
+   * @return an Iceberg catalog
+   */
+  protected Catalog buildIcebergCatalog(String name, CaseInsensitiveStringMap options) {
+    Configuration conf = SparkUtil.hadoopConfCatalogOverrides(SparkSession.active(), name);
+    Map<String, String> optionsMap = new TreeMap<>(options.asCaseSensitiveMap());
+    optionsMap.put(CatalogProperties.APP_ID, SparkSession.active().sparkContext().applicationId());
+    optionsMap.put(CatalogProperties.USER, SparkSession.active().sparkContext().sparkUser());
+    return CatalogUtil.buildIcebergCatalog(name, optionsMap, conf);
+  }
+
+  /**
+   * Build an Iceberg {@link TableIdentifier} for the given Spark identifier.
+   *
+   * @param identifier Spark's identifier
+   * @return an Iceberg identifier
+   */
+  protected TableIdentifier buildIdentifier(Identifier identifier) {
+    return Spark3Util.identifierToTableIdentifier(identifier);
+  }
+
+  @Override
+  public SparkTable loadTable(Identifier ident) throws NoSuchTableException {
+    try {
+      Pair<Table, Long> icebergTable = load(ident);
+      return new SparkTable(icebergTable.first(), icebergTable.second(), !cacheEnabled);
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new NoSuchTableException(ident);
+    }
+  }
+
+  @Override
+  public SparkTable loadTable(Identifier ident, String version) throws NoSuchTableException {
+    try {
+      Pair<Table, Long> icebergTable = load(ident);
+      return new SparkTable(icebergTable.first(), Long.parseLong(version), !cacheEnabled);
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new NoSuchTableException(ident);
+    }
+  }
+
+  @Override
+  public SparkTable loadTable(Identifier ident, long timestamp) throws NoSuchTableException {
+    try {
+      Pair<Table, Long> icebergTable = load(ident);
+      return new SparkTable(icebergTable.first(), SnapshotUtil.snapshotIdAsOfTime(icebergTable.first(), timestamp),
+          !cacheEnabled);
+    } catch (org.apache.iceberg.exceptions.NoSuchTableException e) {
+      throw new NoSuchTableException(ident);
+    }
+  }
+
+  @Override
+  public SparkTable createTable(Identifier ident, StructType schema,
+      Transform[] transforms,
+      Map<String, String> properties) throws TableAlreadyExistsException {

Review Comment:
   Can you revert the whitespace changes in this file? It looks like something autoformatted these and removed the alignment. These should be aligned with the start of `Identifier ident, ...`. There are other examples of this in the gist, but I won't note them all here.



-- 
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