|
21 | 21 |
|
22 | 22 | import java.util.List;
|
23 | 23 | import java.util.Map;
|
| 24 | +import java.util.Set; |
24 | 25 | import org.apache.hadoop.conf.Configuration;
|
25 | 26 | import org.apache.iceberg.CachingCatalog;
|
26 | 27 | import org.apache.iceberg.Schema;
|
27 | 28 | import org.apache.iceberg.Table;
|
28 | 29 | import org.apache.iceberg.Transaction;
|
29 | 30 | import org.apache.iceberg.catalog.Catalog;
|
30 | 31 | import org.apache.iceberg.catalog.Namespace;
|
| 32 | +import org.apache.iceberg.catalog.SupportsNamespaces; |
31 | 33 | import org.apache.iceberg.catalog.TableIdentifier;
|
32 | 34 | import org.apache.iceberg.exceptions.AlreadyExistsException;
|
33 | 35 | import org.apache.iceberg.hadoop.HadoopCatalog;
|
34 | 36 | import org.apache.iceberg.hive.HiveCatalog;
|
35 | 37 | import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
|
| 38 | +import org.apache.iceberg.relocated.com.google.common.base.Splitter; |
| 39 | +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; |
| 40 | +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; |
36 | 41 | import org.apache.iceberg.relocated.com.google.common.collect.Lists;
|
37 | 42 | import org.apache.iceberg.spark.source.SparkTable;
|
38 | 43 | import org.apache.iceberg.spark.source.StagedSparkTable;
|
39 | 44 | import org.apache.spark.sql.SparkSession;
|
| 45 | +import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; |
| 46 | +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; |
40 | 47 | import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
|
41 | 48 | import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException;
|
42 | 49 | import org.apache.spark.sql.connector.catalog.Identifier;
|
| 50 | +import org.apache.spark.sql.connector.catalog.NamespaceChange; |
43 | 51 | import org.apache.spark.sql.connector.catalog.StagedTable;
|
44 | 52 | import org.apache.spark.sql.connector.catalog.StagingTableCatalog;
|
45 | 53 | import org.apache.spark.sql.connector.catalog.TableCatalog;
|
|
50 | 58 | import org.apache.spark.sql.connector.expressions.Transform;
|
51 | 59 | import org.apache.spark.sql.types.StructType;
|
52 | 60 | import org.apache.spark.sql.util.CaseInsensitiveStringMap;
|
| 61 | +import org.glassfish.jersey.internal.guava.Sets; |
| 62 | +import org.sparkproject.guava.collect.Maps; |
53 | 63 |
|
54 | 64 | /**
|
55 |
| - * A Spark TableCatalog implementation that wraps Iceberg's {@link Catalog} interface. |
| 65 | + * A Spark TableCatalog implementation that wraps an Iceberg {@link Catalog}. |
| 66 | + * <p> |
| 67 | + * This supports the following catalog configuration options: |
| 68 | + * <ul> |
| 69 | + * <li><tt>type</tt> - catalog type, "hive" or "hadoop"</li> |
| 70 | + * <li><tt>uri</tt> - the Hive Metastore URI (Hive catalog only)</li> |
| 71 | + * <li><tt>warehouse</tt> - the warehouse path (Hadoop catalog only)</li> |
| 72 | + * <li><tt>default-namespace</tt> - a namespace to use as the default</li> |
| 73 | + * </ul> |
| 74 | + * <p> |
| 75 | + * To use a custom catalog that is not a Hive or Hadoop catalog, extend this class and override |
| 76 | + * {@link #buildIcebergCatalog(String, CaseInsensitiveStringMap)}. |
56 | 77 | */
|
57 |
| -public class SparkCatalog implements StagingTableCatalog { |
| 78 | +public class SparkCatalog implements StagingTableCatalog, org.apache.spark.sql.connector.catalog.SupportsNamespaces { |
| 79 | + private static final Set<String> DEFAULT_NS_KEYS = ImmutableSet.of(TableCatalog.PROP_OWNER); |
| 80 | + |
58 | 81 | private String catalogName = null;
|
59 | 82 | private Catalog icebergCatalog = null;
|
| 83 | + private SupportsNamespaces asNamespaceCatalog = null; |
| 84 | + private String[] defaultNamespace = null; |
60 | 85 |
|
61 | 86 | /**
|
62 | 87 | * Build an Iceberg {@link Catalog} to be used by this Spark catalog adapter.
|
@@ -93,12 +118,6 @@ protected TableIdentifier buildIdentifier(Identifier identifier) {
|
93 | 118 | return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name());
|
94 | 119 | }
|
95 | 120 |
|
96 |
| - @Override |
97 |
| - public Identifier[] listTables(String[] namespace) { |
98 |
| - // TODO: handle namespaces |
99 |
| - return new Identifier[0]; |
100 |
| - } |
101 |
| - |
102 | 121 | @Override
|
103 | 122 | public SparkTable loadTable(Identifier ident) throws NoSuchTableException {
|
104 | 123 | try {
|
@@ -227,13 +246,141 @@ public void invalidateTable(Identifier ident) {
|
227 | 246 | }
|
228 | 247 | }
|
229 | 248 |
|
| 249 | + @Override |
| 250 | + public Identifier[] listTables(String[] namespace) { |
| 251 | + return icebergCatalog.listTables(Namespace.of(namespace)).stream() |
| 252 | + .map(ident -> Identifier.of(ident.namespace().levels(), ident.name())) |
| 253 | + .toArray(Identifier[]::new); |
| 254 | + } |
| 255 | + |
| 256 | + @Override |
| 257 | + public String[] defaultNamespace() { |
| 258 | + if (defaultNamespace != null) { |
| 259 | + return defaultNamespace; |
| 260 | + } |
| 261 | + |
| 262 | + return new String[0]; |
| 263 | + } |
| 264 | + |
| 265 | + @Override |
| 266 | + public String[][] listNamespaces() { |
| 267 | + if (asNamespaceCatalog != null) { |
| 268 | + return asNamespaceCatalog.listNamespaces().stream() |
| 269 | + .map(Namespace::levels) |
| 270 | + .toArray(String[][]::new); |
| 271 | + } |
| 272 | + |
| 273 | + return new String[0][]; |
| 274 | + } |
| 275 | + |
| 276 | + @Override |
| 277 | + public String[][] listNamespaces(String[] namespace) throws NoSuchNamespaceException { |
| 278 | + if (asNamespaceCatalog != null) { |
| 279 | + try { |
| 280 | + return asNamespaceCatalog.listNamespaces(Namespace.of(namespace)).stream() |
| 281 | + .map(Namespace::levels) |
| 282 | + .toArray(String[][]::new); |
| 283 | + } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) { |
| 284 | + throw new NoSuchNamespaceException(namespace); |
| 285 | + } |
| 286 | + } |
| 287 | + |
| 288 | + throw new NoSuchNamespaceException(namespace); |
| 289 | + } |
| 290 | + |
| 291 | + @Override |
| 292 | + public Map<String, String> loadNamespaceMetadata(String[] namespace) throws NoSuchNamespaceException { |
| 293 | + if (asNamespaceCatalog != null) { |
| 294 | + try { |
| 295 | + return asNamespaceCatalog.loadNamespaceMetadata(Namespace.of(namespace)); |
| 296 | + } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) { |
| 297 | + throw new NoSuchNamespaceException(namespace); |
| 298 | + } |
| 299 | + } |
| 300 | + |
| 301 | + throw new NoSuchNamespaceException(namespace); |
| 302 | + } |
| 303 | + |
| 304 | + @Override |
| 305 | + public void createNamespace(String[] namespace, Map<String, String> metadata) throws NamespaceAlreadyExistsException { |
| 306 | + if (asNamespaceCatalog != null) { |
| 307 | + try { |
| 308 | + if (asNamespaceCatalog instanceof HadoopCatalog && DEFAULT_NS_KEYS.equals(metadata.keySet())) { |
| 309 | + // Hadoop catalog will reject metadata properties, but Spark automatically adds "owner". |
| 310 | + // If only the automatic properties are present, replace metadata with an empty map. |
| 311 | + asNamespaceCatalog.createNamespace(Namespace.of(namespace), ImmutableMap.of()); |
| 312 | + } else { |
| 313 | + asNamespaceCatalog.createNamespace(Namespace.of(namespace), metadata); |
| 314 | + } |
| 315 | + } catch (AlreadyExistsException e) { |
| 316 | + throw new NamespaceAlreadyExistsException(namespace); |
| 317 | + } |
| 318 | + } else { |
| 319 | + throw new UnsupportedOperationException("Namespaces are not supported by catalog: " + catalogName); |
| 320 | + } |
| 321 | + } |
| 322 | + |
| 323 | + @Override |
| 324 | + public void alterNamespace(String[] namespace, NamespaceChange... changes) throws NoSuchNamespaceException { |
| 325 | + if (asNamespaceCatalog != null) { |
| 326 | + Map<String, String> updates = Maps.newHashMap(); |
| 327 | + Set<String> removals = Sets.newHashSet(); |
| 328 | + for (NamespaceChange change : changes) { |
| 329 | + if (change instanceof NamespaceChange.SetProperty) { |
| 330 | + NamespaceChange.SetProperty set = (NamespaceChange.SetProperty) change; |
| 331 | + updates.put(set.property(), set.value()); |
| 332 | + } else if (change instanceof NamespaceChange.RemoveProperty) { |
| 333 | + removals.add(((NamespaceChange.RemoveProperty) change).property()); |
| 334 | + } else { |
| 335 | + throw new UnsupportedOperationException("Cannot apply unknown namespace change: " + change); |
| 336 | + } |
| 337 | + } |
| 338 | + |
| 339 | + try { |
| 340 | + if (!updates.isEmpty()) { |
| 341 | + asNamespaceCatalog.setProperties(Namespace.of(namespace), updates); |
| 342 | + } |
| 343 | + |
| 344 | + if (!removals.isEmpty()) { |
| 345 | + asNamespaceCatalog.removeProperties(Namespace.of(namespace), removals); |
| 346 | + } |
| 347 | + |
| 348 | + } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) { |
| 349 | + throw new NoSuchNamespaceException(namespace); |
| 350 | + } |
| 351 | + } else { |
| 352 | + throw new NoSuchNamespaceException(namespace); |
| 353 | + } |
| 354 | + } |
| 355 | + |
| 356 | + @Override |
| 357 | + public boolean dropNamespace(String[] namespace) throws NoSuchNamespaceException { |
| 358 | + if (asNamespaceCatalog != null) { |
| 359 | + try { |
| 360 | + return asNamespaceCatalog.dropNamespace(Namespace.of(namespace)); |
| 361 | + } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) { |
| 362 | + throw new NoSuchNamespaceException(namespace); |
| 363 | + } |
| 364 | + } |
| 365 | + |
| 366 | + return false; |
| 367 | + } |
| 368 | + |
230 | 369 | @Override
|
231 | 370 | public final void initialize(String name, CaseInsensitiveStringMap options) {
|
232 | 371 | boolean cacheEnabled = Boolean.parseBoolean(options.getOrDefault("cache-enabled", "true"));
|
233 | 372 | Catalog catalog = buildIcebergCatalog(name, options);
|
234 | 373 |
|
235 | 374 | this.catalogName = name;
|
236 | 375 | this.icebergCatalog = cacheEnabled ? CachingCatalog.wrap(catalog) : catalog;
|
| 376 | + if (catalog instanceof SupportsNamespaces) { |
| 377 | + this.asNamespaceCatalog = (SupportsNamespaces) catalog; |
| 378 | + if (options.containsKey("default-namespace")) { |
| 379 | + this.defaultNamespace = Splitter.on('.') |
| 380 | + .splitToList(options.get("default-namespace")) |
| 381 | + .toArray(new String[0]); |
| 382 | + } |
| 383 | + } |
237 | 384 | }
|
238 | 385 |
|
239 | 386 | @Override
|
|
0 commit comments