snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 
connector for PinotFS
URL: https://github.com/apache/incubator-pinot/pull/5116#discussion_r389151923
 
 

 ##########
 File path: 
pinot-plugins/pinot-file-system/pinot-adls/src/main/java/org/apache/pinot/plugin/filesystem/AzureGen2PinotFS.java
 ##########
 @@ -0,0 +1,447 @@
+/**
+ * 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.pinot.plugin.filesystem;
+
+import com.azure.core.http.rest.PagedIterable;
+import com.azure.core.util.Context;
+import com.azure.storage.blob.BlobClient;
+import com.azure.storage.blob.BlobServiceClient;
+import com.azure.storage.blob.BlobServiceClientBuilder;
+import com.azure.storage.common.StorageSharedKeyCredential;
+import com.azure.storage.common.Utility;
+import com.azure.storage.file.datalake.DataLakeFileClient;
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.DataLakeServiceClientBuilder;
+import com.azure.storage.file.datalake.models.DataLakeRequestConditions;
+import com.azure.storage.file.datalake.models.DataLakeStorageException;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import com.azure.storage.file.datalake.models.PathHttpHeaders;
+import com.azure.storage.file.datalake.models.PathItem;
+import com.azure.storage.file.datalake.models.PathProperties;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URLDecoder;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.sql.Timestamp;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.io.FileUtils;
+import org.apache.pinot.spi.filesystem.PinotFS;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Azure Data Lake Storage Gen2 implementation for the PinotFS interface.
+ */
+public class AzureGen2PinotFS extends PinotFS {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(AzureGen2PinotFS.class);
+
+  private static final String ACCOUNT_NAME = "accountName";
+  private static final String ACCESS_KEY = "accessKey";
+  private static final String FILE_SYSTEM_NAME = "fileSystemName";
+
+  private static final String HTTPS_URL_PREFIX = "https://";;
+  private static final String DIRECTORY_DELIMITER = "/";
+
+  private static final String AZURE_STORAGE_DNS_SUFFIX = 
".dfs.core.windows.net";
+  private static final String AZURE_BLOB_DNS_SUFFIX = ".blob.core.windows.net";
+  private static final String PATH_ALREADY_EXISTS_ERROR_CODE = 
"PathAlreadyExists";
+  private static final String IS_DIRECTORY_KEY = "hdi_isfolder";
+
+  private static final int NOT_FOUND_STATUS_CODE = 404;
+  private static final int ALREADY_EXISTS_STATUS_CODE = 409;
+
+  // Azure Data Lake Gen2's block size is 4MB
+  private static final int BUFFER_SIZE = 4 * 1024 * 1024;
+
+  private DataLakeFileSystemClient _fileSystemClient;
+  private BlobServiceClient _blobServiceClient;
+
+  @Override
+  public void init(Configuration config) {
+    // Azure storage account name
+    String accountName = config.getString(ACCOUNT_NAME);
+    String accessKey = config.getString(ACCESS_KEY);
+    String fileSystemName = config.getString(FILE_SYSTEM_NAME);
+    String dfsServiceEndpointUrl = HTTPS_URL_PREFIX + accountName + 
AZURE_STORAGE_DNS_SUFFIX;
+    String blobServiceEndpointUrl = HTTPS_URL_PREFIX + accountName + 
AZURE_BLOB_DNS_SUFFIX;
+
+    StorageSharedKeyCredential sharedKeyCredential = new 
StorageSharedKeyCredential(accountName, accessKey);
+
+    DataLakeServiceClient serviceClient = new 
DataLakeServiceClientBuilder().credential(sharedKeyCredential)
+        .endpoint(dfsServiceEndpointUrl)
+        .buildClient();
+
+    _blobServiceClient =
+        new 
BlobServiceClientBuilder().credential(sharedKeyCredential).endpoint(blobServiceEndpointUrl).buildClient();
+    _fileSystemClient = serviceClient.getFileSystemClient(fileSystemName);
+    LOGGER.info("AzureGen2PinotFS is initialized (accountName={}, 
fileSystemName={}, dfsServiceEndpointUrl={}, "
+        + "blobServiceEndpointUrl={})", accountName, fileSystemName, 
dfsServiceEndpointUrl, blobServiceEndpointUrl);
+  }
+
+  @Override
+  public boolean mkdir(URI uri) throws IOException {
+    LOGGER.info("mkdir is called with uri='{}'", uri);
+    try {
+      // By default, create directory call will overwrite if the path already 
exists. Setting IfNoneMatch = "*" to
+      // prevent overwrite. 
https://docs.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/create
+      DataLakeRequestConditions requestConditions = new 
DataLakeRequestConditions().setIfNoneMatch("*");
+      
_fileSystemClient.createDirectoryWithResponse(convertUriToAzureStylePath(uri), 
null, null, null, null,
+          requestConditions, null, null);
+      return true;
+    } catch (DataLakeStorageException e) {
+      // If the path already exists, doing nothing and return true
+      if (e.getStatusCode() == ALREADY_EXISTS_STATUS_CODE && 
e.getErrorCode().equals(PATH_ALREADY_EXISTS_ERROR_CODE)) {
+        return true;
+      }
+      LOGGER.error("Exception thrown while calling mkdir.", e);
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public boolean delete(URI segmentUri, boolean forceDelete) throws 
IOException {
+    LOGGER.info("delete is called with segmentUri='{}', forceDelete='{}'", 
segmentUri, forceDelete);
+    try {
+      boolean isDirectory = isDirectory(segmentUri);
+      if (isDirectory && listFiles(segmentUri, false).length > 0 && 
!forceDelete) {
+        return false;
+      }
+
+      String path = convertUriToAzureStylePath(segmentUri);
+      if (isDirectory) {
+        _fileSystemClient.deleteDirectoryWithResponse(path, true, null, null, 
Context.NONE).getValue();
+      } else {
+        _fileSystemClient.deleteFile(path);
+      }
+      return true;
+    } catch (DataLakeStorageException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public boolean doMove(URI srcUri, URI dstUri) throws IOException {
+    LOGGER.info("doMove is called with srcUri='{}', dstUri='{}'", srcUri, 
dstUri);
+    try {
+      // TODO: currently, azure-sdk has a bug in "rename" when the path 
includes some special characters that gets
+      // changed during the url encoding (e.g '%' -> '%25', ' ' -> '%20')
+      // https://github.com/Azure/azure-sdk-for-java/issues/8761
+//      DataLakeDirectoryClient directoryClient =
+//          
_fileSystemClient.getDirectoryClient(convertUriToAzureStylePath(srcUri));
+//      directoryClient.rename(null, convertUriToAzureStylePath(dstUri));
+      copy(srcUri, dstUri);
+      delete(srcUri, true);
+      return true;
+    } catch (DataLakeStorageException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public boolean copy(URI srcUri, URI dstUri) throws IOException {
+    LOGGER.info("copy is called with srcUri='{}', dstUri='{}'", srcUri, 
dstUri);
+    // If src and dst are the same, do nothing.
+    if (srcUri.equals(dstUri)) {
+      return true;
+    }
+
+    // Remove the destination directory or file
+    if (exists(dstUri)) {
+      delete(dstUri, true);
+    }
+
+    if (!isDirectory(srcUri)) {
+      // If source is a file, we can simply copy the file from src to dst
+      return copySrcToDst(srcUri, dstUri);
+    } else {
+      // In case we are copying a directory, we need to recursively look into 
the directory and copy all the files and
+      // directories accordingly
+      try {
+        boolean copySucceeded = false;
+        Path srcPath = Paths.get(srcUri.getPath());
+        for (String path : listFiles(srcUri, true)) {
+          // Compute the src path for the given path
+          URI currentSrc =
+              new URI(srcUri.getScheme(), srcUri.getHost(), 
convertAzureStylePathToUriStylePath(path), null);
+
+          // Compute the destination path for the current path.
+          String relativeSrcPath = 
srcPath.relativize(Paths.get(convertAzureStylePathToUriStylePath(path))).toString();
+          String newDstPath = Paths.get(dstUri.getPath(), 
relativeSrcPath).toString();
+          URI newDst = new URI(dstUri.getScheme(), dstUri.getHost(), 
newDstPath, null);
+
+          if (isDirectory(currentSrc)) {
+            // If src is directory, create one.
+            if (!mkdir(newDst)) {
+              return false;
+            }
+          } else {
+            // If src is a file, we need to copy.
+            copySucceeded |= copySrcToDst(currentSrc, newDst);
+          }
+        }
+        return copySucceeded;
+      } catch (DataLakeStorageException | URISyntaxException e) {
+        throw new IOException(e);
+      }
+    }
+  }
+
+  @Override
+  public boolean exists(URI fileUri) throws IOException {
+    try {
+      
_fileSystemClient.getDirectoryClient(convertUriToAzureStylePath(fileUri)).getProperties();
+      return true;
+    } catch (DataLakeStorageException e) {
+      if (e.getStatusCode() == NOT_FOUND_STATUS_CODE) {
+        return false;
+      }
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public long length(URI fileUri) throws IOException {
+    try {
+      PathProperties pathProperties =
+          
_fileSystemClient.getDirectoryClient(convertUriToAzureStylePath(fileUri)).getProperties();
+      return pathProperties.getFileSize();
+    } catch (DataLakeStorageException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public String[] listFiles(URI fileUri, boolean recursive) throws IOException 
{
+    LOGGER.info("listFiles is called with fileUri='{}', recursive='{}'", 
fileUri, recursive);
+    try {
+      ListPathsOptions options =
+          new 
ListPathsOptions().setPath(convertUriToAzureStylePath(fileUri)).setRecursive(recursive);
+      PagedIterable<PathItem> iter = _fileSystemClient.listPaths(options, 
null);
+      return iter.stream().map(PathItem::getName).toArray(String[]::new);
+    } catch (DataLakeStorageException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public void copyToLocalFile(URI srcUri, File dstFile) throws Exception {
+    LOGGER.info("copyToLocalFile is called with srcUri='{}', dstFile='{}'", 
srcUri, dstFile);
+    if (dstFile.exists()) {
+      if (dstFile.isDirectory()) {
+        FileUtils.deleteDirectory(dstFile);
+      } else {
+        FileUtils.deleteQuietly(dstFile);
+      }
+    }
+
+    MessageDigest md5File = MessageDigest.getInstance("MD5");
+    int bytesRead;
+    byte[] buffer = new byte[BUFFER_SIZE];
+    try (InputStream inputStream = open(srcUri)) {
+      try (OutputStream outputStream = new FileOutputStream(dstFile)) {
+        while ((bytesRead = inputStream.read(buffer)) != -1) {
+          outputStream.write(buffer, 0, bytesRead);
+          md5File.update(buffer, 0, bytesRead);
 
 Review comment:
   will check md5 hash compute time but i think that this should be very quick

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to