wang-x-xia commented on a change in pull request #4221: URL: https://github.com/apache/iceberg/pull/4221#discussion_r818267986
########## File path: dell/src/main/java/org/apache/iceberg/dell/ecs/EcsCatalog.java ########## @@ -0,0 +1,517 @@ +/* + * 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.dell.ecs; + +import com.emc.object.s3.S3Client; +import com.emc.object.s3.S3Exception; +import com.emc.object.s3.S3ObjectMetadata; +import com.emc.object.s3.bean.GetObjectResult; +import com.emc.object.s3.bean.ListObjectsResult; +import com.emc.object.s3.bean.S3Object; +import com.emc.object.s3.request.ListObjectsRequest; +import com.emc.object.s3.request.PutObjectRequest; +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseMetastoreCatalog; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.dell.DellClientFactories; +import org.apache.iceberg.dell.DellProperties; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NamespaceNotEmptyException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.hadoop.Configurable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class EcsCatalog extends BaseMetastoreCatalog + implements Closeable, SupportsNamespaces, Configurable<Object> { + + /** + * Suffix of table metadata object + */ + private static final String TABLE_OBJECT_SUFFIX = ".table"; + /** + * Suffix of namespace metadata object + */ + private static final String NAMESPACE_OBJECT_SUFFIX = ".namespace"; + /** + * Key of properties version in ECS object user metadata. + */ + private static final String PROPERTIES_VERSION_USER_METADATA_KEY = "iceberg_properties_version"; + + private static final Logger LOG = LoggerFactory.getLogger(EcsCatalog.class); + + private S3Client client; + private Object hadoopConf; + private String catalogName; + /** + * Warehouse is unified with other catalog that without delimiter. + */ + private String warehouseLocation; + private DellProperties dellProperties; + private PropertiesSerDes propertiesSerDes; + private FileIO fileIO; + + /** + * No-arg constructor to load the catalog dynamically. + * <p> + * All fields are initialized by calling {@link EcsCatalog#initialize(String, Map)} later. + */ + public EcsCatalog() { + } + + @Override + public void initialize(String name, Map<String, String> properties) { + this.catalogName = name; + this.dellProperties = new DellProperties(properties); + this.warehouseLocation = + cleanWarehouse(properties.get(CatalogProperties.WAREHOUSE_LOCATION), dellProperties.ecsCatalogDelimiter()); + this.client = DellClientFactories.from(properties).ecsS3(); + this.propertiesSerDes = PropertiesSerDes.current(); + this.fileIO = initializeFileIO(properties); + } + + private String cleanWarehouse(String path, String delimiter) { + Preconditions.checkArgument( + path != null && path.length() > 0, + "Cannot initialize EcsCatalog because warehousePath must not be null"); + int len = path.length(); + if (path.endsWith(delimiter)) { + return path.substring(0, len - delimiter.length()); + } else { + return path; + } + } + + private FileIO initializeFileIO(Map<String, String> properties) { + String fileIOImpl = properties.get(CatalogProperties.FILE_IO_IMPL); + if (fileIOImpl == null) { + FileIO io = new EcsFileIO(); + io.initialize(properties); + return io; + } else { + return CatalogUtil.loadFileIO(fileIOImpl, properties, hadoopConf); + } + } + + @Override + protected TableOperations newTableOps(TableIdentifier tableIdentifier) { + return new EcsTableOperations(catalogName + "." + tableIdentifier, + tableURI(tableIdentifier), fileIO, this); + } + + @Override + protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) { + StringBuilder builder = new StringBuilder(); + builder.append(warehouseLocation); + for (String level : tableIdentifier.namespace().levels()) { + builder.append(dellProperties.ecsCatalogDelimiter()); + builder.append(level); + } + + builder.append(dellProperties.ecsCatalogDelimiter()); + builder.append(tableIdentifier.name()); + return builder.toString(); + } + + /** + * Iterate all table objects with the namespace prefix. + */ + @Override + public List<TableIdentifier> listTables(Namespace namespace) { + if (!namespace.isEmpty() && !namespaceExists(namespace)) { + throw new NoSuchNamespaceException("Namespace %s does not exist", namespace); + } + + String marker = null; + List<TableIdentifier> results = Lists.newArrayList(); + EcsURI prefix = namespacePrefix(namespace); + do { + ListObjectsResult listObjectsResult = client.listObjects( + new ListObjectsRequest(prefix.bucket()) + .withDelimiter(dellProperties.ecsCatalogDelimiter()) + .withPrefix(prefix.name()) + .withMarker(marker)); + marker = listObjectsResult.getNextMarker(); + results.addAll(listObjectsResult.getObjects().stream() + .filter(s3Object -> s3Object.getKey().endsWith(TABLE_OBJECT_SUFFIX)) + .map(object -> parseTableId(namespace, prefix, object)) + .collect(Collectors.toList())); + } while (marker != null); Review comment: The table needs an object to store the latest table snapshot location. So I reuse this object to list tables. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
