You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/03/05 09:58:06 UTC

[GitHub] [incubator-pinot] snleee opened a new pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

snleee opened a new pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS
URL: https://github.com/apache/incubator-pinot/pull/5116
 
 
   1. Testing have been done by attaching ADLS Gen2 to the local deployment.
   2. move() is implemented by copy & delete because of azure sdk issue with rename() API.
      https://github.com/Azure/azure-sdk-for-java/issues/8761

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r389157324
 
 

 ##########
 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
 
 Review comment:
    the comment already have the issue linked.
   
   As for atomic moves, pinotFS interface doesn't require us to implement `move()` as an atomic operation. 
   
   But, copy & delete approach will perform very bad on moving large files (pinot segments) which is quite often. Before we go to the production on this, we should make sure Azure team fix this.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee merged pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
snleee merged pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS
URL: https://github.com/apache/incubator-pinot/pull/5116
 
 
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mayankshriv commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
mayankshriv 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_r388457374
 
 

 ##########
 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);
 
 Review comment:
   Not needed for the PR, but we should think about supporting encrypted ACCESS_KEY in the config, as opposed to plain text.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r392512787
 
 

 ##########
 File path: pinot-plugins/pinot-file-system/pinot-adls/src/main/java/org/apache/pinot/plugin/filesystem/AzureGen2PinotFS.java
 ##########
 @@ -0,0 +1,460 @@
+/**
+ * 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.DataLakeDirectoryClient;
+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.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 ENABLE_CHECKSUM = "enableChecksum";
+
+  private static final String HTTPS_URL_PREFIX = "https://";
+
+  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;
+
+  // If enabled, pinotFS implementation will guarantee that the bits you've read are the same as the ones you wrote.
+  // However, there's some overhead in computing hash. (Adds roughly 3 seconds for 1GB file)
+  private boolean _enableChecksum;
+
+  @Override
+  public void init(Configuration config) {
+    _enableChecksum = config.getBoolean(ENABLE_CHECKSUM, false);
+
+    // Azure storage account name
+    String accountName = config.getString(ACCOUNT_NAME);
+
+    // TODO: consider to add the encryption of the following config
+    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.error("AzureGen2PinotFS is initialized (accountName={}, fileSystemName={}, dfsServiceEndpointUrl={}, "
+            + "blobServiceEndpointUrl={}, enableChecksum={})", accountName, fileSystemName, dfsServiceEndpointUrl,
+        blobServiceEndpointUrl, _enableChecksum);
+  }
+
+  @Override
+  public boolean mkdir(URI uri) throws IOException {
+    LOGGER.debug("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(AzurePinotFSUtil.convertUriToUrlEncodedAzureStylePath(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 (uri = {})", uri, e);
 
 Review comment:
   I'm including `e`, which is exception object to the log. This should include the status code information as part of the exception stack. Do you think it's better to add status code explicitly along with `uri`?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on issue #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
snleee commented on issue #5116: Add Azure Data Lake Gen2 connector for PinotFS
URL: https://github.com/apache/incubator-pinot/pull/5116#issuecomment-597961687
 
 
   md5 computation time benchmark (on macbook with ssd):
   
   (md5 compute/ total time) <- total time = IO(read contents from file) + md5 hash computation
   1 mb: 17 / 20 ms
   10 mb: 54 / 62 ms
   100 mb: 402 / 462 ms
   1000 mb: 2998 / 3497 ms
   
   So, `computing md5 hash on 1GB file adds about 3 seconds.` Since this adds non-trivial overheads (depending on the use case), I updated the code to make md5 check configurable.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mayankshriv commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
mayankshriv 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_r388459234
 
 

 ##########
 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) {
 
 Review comment:
   What's the expected performance of listFiles? We just need to ensure there's at least one file, and not list all.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r389153432
 
 

 ##########
 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);
 
 Review comment:
   added uri path as part of this log

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] codecov-io edited a comment on issue #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
codecov-io edited a comment on issue #5116: Add Azure Data Lake Gen2 connector for PinotFS
URL: https://github.com/apache/incubator-pinot/pull/5116#issuecomment-596944096
 
 
   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/5116?src=pr&el=h1) Report
   > Merging [#5116](https://codecov.io/gh/apache/incubator-pinot/pull/5116?src=pr&el=desc) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/d5c8398c4bf95d31e20bcac2946ca2ed8b3d7a82?src=pr&el=desc) will **decrease** coverage by `0.76%`.
   > The diff coverage is `66.66%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/5116/graphs/tree.svg?width=650&token=4ibza2ugkz&height=150&src=pr)](https://codecov.io/gh/apache/incubator-pinot/pull/5116?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #5116      +/-   ##
   ============================================
   - Coverage     58.32%   57.55%   -0.77%     
     Complexity       12       12              
   ============================================
     Files          1209     1184      -25     
     Lines         64541    62424    -2117     
     Branches       9484     9143     -341     
   ============================================
   - Hits          37643    35929    -1714     
   + Misses        24143    23847     -296     
   + Partials       2755     2648     -107
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/5116?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [.../java/org/apache/pinot/spi/filesystem/PinotFS.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZmlsZXN5c3RlbS9QaW5vdEZTLmphdmE=) | `72.22% <66.66%> (-6.35%)` | `0 <0> (ø)` | |
   | [...ation/function/AggregationFunctionVisitorBase.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9hZ2dyZWdhdGlvbi9mdW5jdGlvbi9BZ2dyZWdhdGlvbkZ1bmN0aW9uVmlzaXRvckJhc2UuamF2YQ==) | `0% <0%> (-96%)` | `0% <0%> (ø)` | |
   | [...ommon/lineage/SegmentMergeLineageAccessHelper.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbGluZWFnZS9TZWdtZW50TWVyZ2VMaW5lYWdlQWNjZXNzSGVscGVyLmphdmE=) | `0% <0%> (-80%)` | `0% <0%> (ø)` | |
   | [...n/java/org/apache/pinot/common/utils/LLCUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvTExDVXRpbHMuamF2YQ==) | `0% <0%> (-75%)` | `0% <0%> (ø)` | |
   | [.../org/apache/pinot/common/config/RoutingConfig.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vY29uZmlnL1JvdXRpbmdDb25maWcuamF2YQ==) | `0% <0%> (-70%)` | `0% <0%> (ø)` | |
   | [...org/apache/pinot/common/metrics/BrokerMetrics.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJNZXRyaWNzLmphdmE=) | `44.44% <0%> (-44.45%)` | `0% <0%> (ø)` | |
   | [.../org/apache/pinot/client/PinotClientException.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L1Bpbm90Q2xpZW50RXhjZXB0aW9uLmphdmE=) | `33.33% <0%> (-33.34%)` | `0% <0%> (ø)` | |
   | [...che/pinot/core/startree/v2/StarTreeV2Metadata.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9zdGFydHJlZS92Mi9TdGFyVHJlZVYyTWV0YWRhdGEuamF2YQ==) | `62.5% <0%> (-29.17%)` | `0% <0%> (ø)` | |
   | [...a/manager/realtime/RealtimeSegmentDataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVTZWdtZW50RGF0YU1hbmFnZXIuamF2YQ==) | `50% <0%> (-25%)` | `0% <0%> (ø)` | |
   | [...rg/apache/pinot/core/transport/ServerInstance.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS90cmFuc3BvcnQvU2VydmVySW5zdGFuY2UuamF2YQ==) | `53.57% <0%> (-21.43%)` | `0% <0%> (ø)` | |
   | ... and [239 more](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/5116?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/5116?src=pr&el=footer). Last update [d5c8398...67feb10](https://codecov.io/gh/apache/incubator-pinot/pull/5116?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on issue #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
snleee commented on issue #5116: Add Azure Data Lake Gen2 connector for PinotFS
URL: https://github.com/apache/incubator-pinot/pull/5116#issuecomment-599814490
 
 
   @jackjlli Thanks for the comment. I added `TODO` comment on the test. I will address it with the separate PR.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mayankshriv commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
mayankshriv 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_r388464861
 
 

 ##########
 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 {
 
 Review comment:
   Would be good to publish some performance numbers, for such operations as copy, listFiles, 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] jackjlli commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
jackjlli 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_r391853187
 
 

 ##########
 File path: pinot-plugins/pinot-file-system/pinot-adls/src/main/java/org/apache/pinot/plugin/filesystem/AzureGen2PinotFS.java
 ##########
 @@ -0,0 +1,460 @@
+/**
+ * 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.DataLakeDirectoryClient;
+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.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 ENABLE_CHECKSUM = "enableChecksum";
+
+  private static final String HTTPS_URL_PREFIX = "https://";
+
+  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;
+
+  // If enabled, pinotFS implementation will guarantee that the bits you've read are the same as the ones you wrote.
+  // However, there's some overhead in computing hash. (Adds roughly 3 seconds for 1GB file)
+  private boolean _enableChecksum;
+
+  @Override
+  public void init(Configuration config) {
+    _enableChecksum = config.getBoolean(ENABLE_CHECKSUM, false);
+
+    // Azure storage account name
+    String accountName = config.getString(ACCOUNT_NAME);
+
+    // TODO: consider to add the encryption of the following config
+    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.error("AzureGen2PinotFS is initialized (accountName={}, fileSystemName={}, dfsServiceEndpointUrl={}, "
+            + "blobServiceEndpointUrl={}, enableChecksum={})", accountName, fileSystemName, dfsServiceEndpointUrl,
+        blobServiceEndpointUrl, _enableChecksum);
+  }
+
+  @Override
+  public boolean mkdir(URI uri) throws IOException {
+    LOGGER.debug("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(AzurePinotFSUtil.convertUriToUrlEncodedAzureStylePath(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 (uri = {})", uri, e);
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public boolean delete(URI segmentUri, boolean forceDelete) throws IOException {
+    LOGGER.debug("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 = AzurePinotFSUtil.convertUriToUrlEncodedAzureStylePath(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.debug("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(AzurePinotFSUtil.convertUriToUrlEncodedAzureStylePath(srcUri));
+      directoryClient.rename(null, AzurePinotFSUtil.convertUriToUrlEncodedAzureStylePath(dstUri));
+      return true;
+    } catch (DataLakeStorageException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public boolean copy(URI srcUri, URI dstUri) throws IOException {
+    LOGGER.debug("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(), path, null);
+
+          // Compute the destination path for the current path.
+          String relativeSrcPath = srcPath.relativize(Paths.get(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.
+            mkdir(newDst);
+          } else {
+            // If src is a file, we need to copy.
+            copySucceeded |= copySrcToDst(currentSrc, newDst);
 
 Review comment:
   What if part of the files failed? It will still return true, right?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
mcvsubbu 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_r391792039
 
 

 ##########
 File path: pinot-plugins/pinot-file-system/pinot-adls/src/main/java/org/apache/pinot/plugin/filesystem/AzureGen2PinotFS.java
 ##########
 @@ -0,0 +1,460 @@
+/**
+ * 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.DataLakeDirectoryClient;
+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.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 ENABLE_CHECKSUM = "enableChecksum";
+
+  private static final String HTTPS_URL_PREFIX = "https://";
+
+  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;
+
+  // If enabled, pinotFS implementation will guarantee that the bits you've read are the same as the ones you wrote.
+  // However, there's some overhead in computing hash. (Adds roughly 3 seconds for 1GB file)
+  private boolean _enableChecksum;
+
+  @Override
+  public void init(Configuration config) {
+    _enableChecksum = config.getBoolean(ENABLE_CHECKSUM, false);
+
+    // Azure storage account name
+    String accountName = config.getString(ACCOUNT_NAME);
+
+    // TODO: consider to add the encryption of the following config
+    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.error("AzureGen2PinotFS is initialized (accountName={}, fileSystemName={}, dfsServiceEndpointUrl={}, "
 
 Review comment:
   why is this an error log?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mayankshriv commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
mayankshriv 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_r388462934
 
 

 ##########
 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);
 
 Review comment:
   Same here, let's try to consolidate logging per segment push.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] jackjlli commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
jackjlli 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_r392514077
 
 

 ##########
 File path: pinot-plugins/pinot-file-system/pinot-adls/src/main/java/org/apache/pinot/plugin/filesystem/AzureGen2PinotFS.java
 ##########
 @@ -0,0 +1,460 @@
+/**
+ * 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.DataLakeDirectoryClient;
+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.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 ENABLE_CHECKSUM = "enableChecksum";
+
+  private static final String HTTPS_URL_PREFIX = "https://";
+
+  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;
+
+  // If enabled, pinotFS implementation will guarantee that the bits you've read are the same as the ones you wrote.
+  // However, there's some overhead in computing hash. (Adds roughly 3 seconds for 1GB file)
+  private boolean _enableChecksum;
+
+  @Override
+  public void init(Configuration config) {
+    _enableChecksum = config.getBoolean(ENABLE_CHECKSUM, false);
+
+    // Azure storage account name
+    String accountName = config.getString(ACCOUNT_NAME);
+
+    // TODO: consider to add the encryption of the following config
+    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.error("AzureGen2PinotFS is initialized (accountName={}, fileSystemName={}, dfsServiceEndpointUrl={}, "
+            + "blobServiceEndpointUrl={}, enableChecksum={})", accountName, fileSystemName, dfsServiceEndpointUrl,
+        blobServiceEndpointUrl, _enableChecksum);
+  }
+
+  @Override
+  public boolean mkdir(URI uri) throws IOException {
+    LOGGER.debug("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(AzurePinotFSUtil.convertUriToUrlEncodedAzureStylePath(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 (uri = {})", uri, e);
 
 Review comment:
   Not sure whether the status code is included in the exception, but it'd be good to show it in the log. :)

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mayankshriv commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
mayankshriv 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_r388457892
 
 

 ##########
 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;
 
 Review comment:
   Should this be true or false? Don't recall what the interface expects, but good to check.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r389151305
 
 

 ##########
 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);
 
 Review comment:
   i will change all the logs to `debug` level. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r389155589
 
 

 ##########
 File path: pom.xml
 ##########
 @@ -394,14 +399,43 @@
       <dependency>
         <groupId>io.netty</groupId>
         <artifactId>netty-all</artifactId>
-        <version>4.1.28.Final</version>
+        <version>4.1.42.Final</version>
 
 Review comment:
   thanks i changed

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] jackjlli commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
jackjlli 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_r391856425
 
 

 ##########
 File path: pinot-spi/src/main/java/org/apache/pinot/spi/filesystem/PinotFS.java
 ##########
 @@ -103,8 +106,13 @@ public boolean move(URI srcUri, URI dstUri, boolean overwrite)
       }
     } else {
       // ensures the parent path of dst exists.
-      URI parentUri = Paths.get(dstUri).getParent().toUri();
-      mkdir(parentUri);
+      try {
+        Path parentPath = Paths.get(dstUri.getPath()).getParent();
 
 Review comment:
   Can you add a test for testing the case when the scheme doesn't match then throw the exception?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mayankshriv commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
mayankshriv 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_r388495313
 
 

 ##########
 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);
+        }
+      }
+    }
+
+    // If MD5 hash is available as part of path properties, verify it with the local file
+    DataLakeFileClient fileClient = _fileSystemClient.getFileClient(convertUriToAzureStylePath(srcUri));
+    byte[] md5FromMetadata = fileClient.getProperties().getContentMd5();
+    if (md5FromMetadata != null && md5FromMetadata.length > 0) {
+      byte[] md5FromLocalFile = md5File.digest();
+      if (!Arrays.equals(md5FromLocalFile, md5FromMetadata)) {
+        // Clean up the corrupted file
+        FileUtils.deleteQuietly(dstFile);
+        throw new IOException("Computed MD5 and MD5 from metadata do not match");
+      }
+    }
+  }
+
+  @Override
+  public void copyFromLocalFile(File srcFile, URI dstUri) throws Exception {
+    LOGGER.info("copyFromLocalFile is called with srcFile='{}', dstUri='{}'", srcFile, dstUri);
+    try (InputStream fileInputStream = new FileInputStream(srcFile)) {
+      copyInputStreamToDst(fileInputStream, dstUri);
+    }
+  }
+
+  @Override
+  public boolean isDirectory(URI uri) throws IOException {
+    try {
+      PathProperties pathProperties = getPathProperties(uri);
+      Map<String, String> metadata = pathProperties.getMetadata();
+      // TODO: need to find the other ways to check the directory if it becomes available. listFiles API returns
+      // PathInfo, which includes "isDirectory" field; however, there's no API available for fetching PathInfo directly
+      // from target uri.
+      return Boolean.valueOf(metadata.get(IS_DIRECTORY_KEY));
+    } catch (DataLakeStorageException e) {
+      throw new IOException("Failed while checking isDirectory for : " + uri, e);
+    }
+  }
+
+  @Override
+  public long lastModified(URI uri) throws IOException {
+    try {
+      PathProperties pathProperties = getPathProperties(uri);
+      OffsetDateTime offsetDateTime = pathProperties.getLastModified();
+      Timestamp timestamp = Timestamp.valueOf(offsetDateTime.atZoneSameInstant(ZoneOffset.UTC).toLocalDateTime());
+      return timestamp.getTime();
+    } catch (DataLakeStorageException e) {
+      throw new IOException("Failed while checking lastModified time for : " + uri, e);
+    }
+  }
+
+  @Override
+  public boolean touch(URI uri) throws IOException {
+    // The following data lake gen2 API provides a way to update file properties including last modified time.
+    // https://docs.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/update
+    // However, action = "setProperties" is available in REST API but not available in Java SDK yet.
+    //
+    // For now, directly use Blob service's API to get the same effect.
+    // https://docs.microsoft.com/en-us/rest/api/storageservices/set-file-properties
+    try {
+      DataLakeFileClient fileClient = _fileSystemClient.getFileClient(convertUriToAzureStylePath(uri));
+      PathProperties pathProperties = fileClient.getProperties();
+      fileClient.setHttpHeaders(getPathHttpHeaders(pathProperties));
+      return true;
+    } catch (DataLakeStorageException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public InputStream open(URI uri) throws IOException {
+    // Use Blob API since read() function from Data Lake Client currently takes "OutputStream" as an input and
+    // flush bytes to an output stream. This needs to be piped back into input stream to implement this function.
+    // On the other hand, Blob API directly allow you to open the input stream.
+    BlobClient blobClient = _blobServiceClient.getBlobContainerClient(_fileSystemClient.getFileSystemName())
+        .getBlobClient(convertUriToAzureStylePath(uri));
+
+    return blobClient.openInputStream();
+    // Another approach is to download the file to the local disk to a temp path and return the file input stream. In
+    // this case, we need to override "close()" and delete temp file.
+  }
+
+  private boolean copySrcToDst(URI srcUri, URI dstUri) throws IOException {
+    try (InputStream inputStream = open(srcUri)) {
+      return copyInputStreamToDst(inputStream, dstUri);
+    }
+  }
+
+  /**
+   * Helper function to copy input stream to destination URI.
+   *
+   * NOTE: the caller has to close the input stream.
+   *
+   * @param inputStream input stream that will be written to dstUri
+   * @param dstUri destination URI
+   * @return true if the copy succeeds
+   */
+  private boolean copyInputStreamToDst(InputStream inputStream, URI dstUri) throws IOException {
+    int bytesRead;
+    long totalBytesRead = 0;
+    byte[] buffer = new byte[BUFFER_SIZE];
+    DataLakeFileClient fileClient = _fileSystemClient.createFile(convertUriToAzureStylePath(dstUri));
+    try {
+      MessageDigest md5File = MessageDigest.getInstance("MD5");
 
 Review comment:
   Seems like this part of the code can be refactored, since there is one more place doing something similar?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r389155545
 
 

 ##########
 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)) {
 
 Review comment:
   changed to `mkdir(newDst)`

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r392520612
 
 

 ##########
 File path: pinot-plugins/pinot-file-system/pinot-adls/src/main/java/org/apache/pinot/plugin/filesystem/AzureGen2PinotFS.java
 ##########
 @@ -0,0 +1,460 @@
+/**
+ * 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.DataLakeDirectoryClient;
+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.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 {
 
 Review comment:
   I don't see a good way to test. I've been testing this by hooking this up to the live ADLS Gen2. One way is to mock every single Azure SDK API that i'm calling using Mockhito but this doesn't really check much.
   
   Another potential approach is to create the integration test by incorporating Azurite https://github.com/Azure/Azurite, which is Azure storage service emulator. But, this doesn't support Azure Datalake Gen2.
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] jackjlli commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
jackjlli 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_r388572373
 
 

 ##########
 File path: pom.xml
 ##########
 @@ -394,14 +399,43 @@
       <dependency>
         <groupId>io.netty</groupId>
         <artifactId>netty-all</artifactId>
-        <version>4.1.28.Final</version>
+        <version>4.1.42.Final</version>
 
 Review comment:
   We can extract the netty version altogether and put it to one place.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r388504180
 
 

 ##########
 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;
 
 Review comment:
   I'm explicitly following `pinotFS` interface.
   
   ```
     /**
      * Creates a new directory. If parent directories are not created, it will create them.
      * If the directory exists, it will return true without doing anything.
      * @return true if mkdir is successful
      * @throws IOException on IO failure
      */
   ```

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] codecov-io commented on issue #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
codecov-io commented on issue #5116: Add Azure Data Lake Gen2 connector for PinotFS
URL: https://github.com/apache/incubator-pinot/pull/5116#issuecomment-596944096
 
 
   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/5116?src=pr&el=h1) Report
   > Merging [#5116](https://codecov.io/gh/apache/incubator-pinot/pull/5116?src=pr&el=desc) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/d5c8398c4bf95d31e20bcac2946ca2ed8b3d7a82?src=pr&el=desc) will **decrease** coverage by `0.76%`.
   > The diff coverage is `66.66%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/5116/graphs/tree.svg?width=650&token=4ibza2ugkz&height=150&src=pr)](https://codecov.io/gh/apache/incubator-pinot/pull/5116?src=pr&el=tree)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #5116      +/-   ##
   ============================================
   - Coverage     58.32%   57.55%   -0.77%     
     Complexity       12       12              
   ============================================
     Files          1209     1184      -25     
     Lines         64541    62424    -2117     
     Branches       9484     9143     -341     
   ============================================
   - Hits          37643    35929    -1714     
   + Misses        24143    23847     -296     
   + Partials       2755     2648     -107
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/5116?src=pr&el=tree) | Coverage Δ | Complexity Δ | |
   |---|---|---|---|
   | [.../java/org/apache/pinot/spi/filesystem/PinotFS.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3Qtc3BpL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9zcGkvZmlsZXN5c3RlbS9QaW5vdEZTLmphdmE=) | `72.22% <66.66%> (-6.35%)` | `0 <0> (ø)` | |
   | [...ation/function/AggregationFunctionVisitorBase.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9xdWVyeS9hZ2dyZWdhdGlvbi9mdW5jdGlvbi9BZ2dyZWdhdGlvbkZ1bmN0aW9uVmlzaXRvckJhc2UuamF2YQ==) | `0% <0%> (-96%)` | `0% <0%> (ø)` | |
   | [...ommon/lineage/SegmentMergeLineageAccessHelper.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbGluZWFnZS9TZWdtZW50TWVyZ2VMaW5lYWdlQWNjZXNzSGVscGVyLmphdmE=) | `0% <0%> (-80%)` | `0% <0%> (ø)` | |
   | [...n/java/org/apache/pinot/common/utils/LLCUtils.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vdXRpbHMvTExDVXRpbHMuamF2YQ==) | `0% <0%> (-75%)` | `0% <0%> (ø)` | |
   | [.../org/apache/pinot/common/config/RoutingConfig.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vY29uZmlnL1JvdXRpbmdDb25maWcuamF2YQ==) | `0% <0%> (-70%)` | `0% <0%> (ø)` | |
   | [...org/apache/pinot/common/metrics/BrokerMetrics.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vbWV0cmljcy9Ccm9rZXJNZXRyaWNzLmphdmE=) | `44.44% <0%> (-44.45%)` | `0% <0%> (ø)` | |
   | [.../org/apache/pinot/client/PinotClientException.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY2xpZW50cy9waW5vdC1qYXZhLWNsaWVudC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY2xpZW50L1Bpbm90Q2xpZW50RXhjZXB0aW9uLmphdmE=) | `33.33% <0%> (-33.34%)` | `0% <0%> (ø)` | |
   | [...che/pinot/core/startree/v2/StarTreeV2Metadata.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9zdGFydHJlZS92Mi9TdGFyVHJlZVYyTWV0YWRhdGEuamF2YQ==) | `62.5% <0%> (-29.17%)` | `0% <0%> (ø)` | |
   | [...a/manager/realtime/RealtimeSegmentDataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVTZWdtZW50RGF0YU1hbmFnZXIuamF2YQ==) | `50% <0%> (-25%)` | `0% <0%> (ø)` | |
   | [...rg/apache/pinot/core/transport/ServerInstance.java](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS90cmFuc3BvcnQvU2VydmVySW5zdGFuY2UuamF2YQ==) | `53.57% <0%> (-21.43%)` | `0% <0%> (ø)` | |
   | ... and [239 more](https://codecov.io/gh/apache/incubator-pinot/pull/5116/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/5116?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/5116?src=pr&el=footer). Last update [d5c8398...4d2a8ff](https://codecov.io/gh/apache/incubator-pinot/pull/5116?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mayankshriv commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
mayankshriv 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_r388461881
 
 

 ##########
 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
 
 Review comment:
   Do we need atomic moves? That was one of the key benefits of ADLS Gen2 over ABS, if I am not mistaken.
   Would be good to link the issue you created, 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r392512306
 
 

 ##########
 File path: pinot-plugins/pinot-file-system/pinot-adls/src/main/java/org/apache/pinot/plugin/filesystem/AzureGen2PinotFS.java
 ##########
 @@ -0,0 +1,460 @@
+/**
+ * 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.DataLakeDirectoryClient;
+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.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 ENABLE_CHECKSUM = "enableChecksum";
+
+  private static final String HTTPS_URL_PREFIX = "https://";
+
+  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;
+
+  // If enabled, pinotFS implementation will guarantee that the bits you've read are the same as the ones you wrote.
+  // However, there's some overhead in computing hash. (Adds roughly 3 seconds for 1GB file)
+  private boolean _enableChecksum;
+
+  @Override
+  public void init(Configuration config) {
+    _enableChecksum = config.getBoolean(ENABLE_CHECKSUM, false);
+
+    // Azure storage account name
+    String accountName = config.getString(ACCOUNT_NAME);
+
+    // TODO: consider to add the encryption of the following config
+    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.error("AzureGen2PinotFS is initialized (accountName={}, fileSystemName={}, dfsServiceEndpointUrl={}, "
+            + "blobServiceEndpointUrl={}, enableChecksum={})", accountName, fileSystemName, dfsServiceEndpointUrl,
+        blobServiceEndpointUrl, _enableChecksum);
+  }
+
+  @Override
+  public boolean mkdir(URI uri) throws IOException {
+    LOGGER.debug("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(AzurePinotFSUtil.convertUriToUrlEncodedAzureStylePath(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 (uri = {})", uri, e);
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public boolean delete(URI segmentUri, boolean forceDelete) throws IOException {
+    LOGGER.debug("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 = AzurePinotFSUtil.convertUriToUrlEncodedAzureStylePath(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.debug("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(AzurePinotFSUtil.convertUriToUrlEncodedAzureStylePath(srcUri));
+      directoryClient.rename(null, AzurePinotFSUtil.convertUriToUrlEncodedAzureStylePath(dstUri));
+      return true;
+    } catch (DataLakeStorageException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public boolean copy(URI srcUri, URI dstUri) throws IOException {
+    LOGGER.debug("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(), path, null);
+
+          // Compute the destination path for the current path.
+          String relativeSrcPath = srcPath.relativize(Paths.get(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.
+            mkdir(newDst);
+          } else {
+            // If src is a file, we need to copy.
+            copySucceeded |= copySrcToDst(currentSrc, newDst);
 
 Review comment:
   good catch :) I updated. I also made the change to `GcsPinotFS`, which had the similar issue.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r391915420
 
 

 ##########
 File path: pinot-plugins/pinot-file-system/pinot-adls/src/main/java/org/apache/pinot/plugin/filesystem/AzureGen2PinotFS.java
 ##########
 @@ -0,0 +1,460 @@
+/**
+ * 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.DataLakeDirectoryClient;
+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.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 ENABLE_CHECKSUM = "enableChecksum";
+
+  private static final String HTTPS_URL_PREFIX = "https://";
+
+  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;
+
+  // If enabled, pinotFS implementation will guarantee that the bits you've read are the same as the ones you wrote.
+  // However, there's some overhead in computing hash. (Adds roughly 3 seconds for 1GB file)
+  private boolean _enableChecksum;
+
+  @Override
+  public void init(Configuration config) {
+    _enableChecksum = config.getBoolean(ENABLE_CHECKSUM, false);
+
+    // Azure storage account name
+    String accountName = config.getString(ACCOUNT_NAME);
+
+    // TODO: consider to add the encryption of the following config
+    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.error("AzureGen2PinotFS is initialized (accountName={}, fileSystemName={}, dfsServiceEndpointUrl={}, "
 
 Review comment:
   good catch. I was debugging and forgot to turn 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] jackjlli commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
jackjlli 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_r391857536
 
 

 ##########
 File path: pinot-plugins/pinot-file-system/pinot-adls/src/main/java/org/apache/pinot/plugin/filesystem/AzureGen2PinotFS.java
 ##########
 @@ -0,0 +1,460 @@
+/**
+ * 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.DataLakeDirectoryClient;
+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.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 {
 
 Review comment:
   Do we have any test for this class? We've noticed the code coverage is only 50% 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r389157324
 
 

 ##########
 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
 
 Review comment:
    the comment already have the issue linked.
   
   As for atomic moves, pinotFS interface doesn't require us to implement `move()` as an atomic operation. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r389151569
 
 

 ##########
 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) {
 
 Review comment:
   I will run some testing on all the functions and will share the number

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r392519581
 
 

 ##########
 File path: pinot-spi/src/main/java/org/apache/pinot/spi/filesystem/PinotFS.java
 ##########
 @@ -103,8 +106,13 @@ public boolean move(URI srcUri, URI dstUri, boolean overwrite)
       }
     } else {
       // ensures the parent path of dst exists.
-      URI parentUri = Paths.get(dstUri).getParent().toUri();
-      mkdir(parentUri);
+      try {
+        Path parentPath = Paths.get(dstUri.getPath()).getParent();
 
 Review comment:
   added a test to cover this case to `LocalPinotFSTest`

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] jackjlli commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
jackjlli 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_r391851562
 
 

 ##########
 File path: pinot-plugins/pinot-file-system/pinot-adls/src/main/java/org/apache/pinot/plugin/filesystem/AzureGen2PinotFS.java
 ##########
 @@ -0,0 +1,460 @@
+/**
+ * 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.DataLakeDirectoryClient;
+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.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 ENABLE_CHECKSUM = "enableChecksum";
+
+  private static final String HTTPS_URL_PREFIX = "https://";
+
+  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;
+
+  // If enabled, pinotFS implementation will guarantee that the bits you've read are the same as the ones you wrote.
+  // However, there's some overhead in computing hash. (Adds roughly 3 seconds for 1GB file)
+  private boolean _enableChecksum;
+
+  @Override
+  public void init(Configuration config) {
+    _enableChecksum = config.getBoolean(ENABLE_CHECKSUM, false);
+
+    // Azure storage account name
+    String accountName = config.getString(ACCOUNT_NAME);
+
+    // TODO: consider to add the encryption of the following config
+    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.error("AzureGen2PinotFS is initialized (accountName={}, fileSystemName={}, dfsServiceEndpointUrl={}, "
+            + "blobServiceEndpointUrl={}, enableChecksum={})", accountName, fileSystemName, dfsServiceEndpointUrl,
+        blobServiceEndpointUrl, _enableChecksum);
+  }
+
+  @Override
+  public boolean mkdir(URI uri) throws IOException {
+    LOGGER.debug("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(AzurePinotFSUtil.convertUriToUrlEncodedAzureStylePath(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 (uri = {})", uri, e);
 
 Review comment:
   It'd be good to print the error status code 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r389151771
 
 

 ##########
 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 {
 
 Review comment:
   same as above

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r392520612
 
 

 ##########
 File path: pinot-plugins/pinot-file-system/pinot-adls/src/main/java/org/apache/pinot/plugin/filesystem/AzureGen2PinotFS.java
 ##########
 @@ -0,0 +1,460 @@
+/**
+ * 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.DataLakeDirectoryClient;
+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.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 {
 
 Review comment:
   I don't see a good way to test. I've been testing this by hooking this up to the live ADLS Gen2. One way is to mock every single Azure SDK API that i'm calling using Mockhito but this doesn't really check much.
   
   Another potential approach is to create the integration test by incorporating Azurite https://github.com/Azure/Azurite, which is Azure storage service emulator. But, this doesn't support Azure Datalake Gen2.
   
   By the way, I did verify all the functions by hooking up the live Data Lake Gen2.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mayankshriv commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
mayankshriv 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_r388458107
 
 

 ##########
 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);
 
 Review comment:
   Given that rename is implemented as copy and delete, this message will be logged for each segment push (we already have too many messages per segment push). Consider if this can be a `debug` message, or merged with other messages (in case of errors).
   
   Nit: s/delete/'Delete'

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] jackjlli commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
jackjlli 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_r388569903
 
 

 ##########
 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)) {
 
 Review comment:
   According to the interface description of mkdir, it will either return true or throw an exception. Thus, it won't return false, right?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] jackjlli commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
jackjlli 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_r388567274
 
 

 ##########
 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);
 
 Review comment:
   It'd be good to log the path here in error level.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r388502577
 
 

 ##########
 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);
 
 Review comment:
   Existing AzurePinotFS also gets the config as a string. I guess that our internal restli framework handles encryption of the `ACCESS_KEY`. I will add `TODO` 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mayankshriv commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
mayankshriv 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_r388493178
 
 

 ##########
 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:
   Would be good to have perf check here, given that we don't do such checks for other PinotFS implementations.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mayankshriv commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
mayankshriv 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_r388458107
 
 

 ##########
 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);
 
 Review comment:
   Given that rename is implemented as copy and delete, this message will be logged for each segment push (we already have too many messages per segment push). Consider if this can be a `debug` message, or merged with other messages (in case of errors).
   
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r389151693
 
 

 ##########
 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);
 
 Review comment:
   changed to `debug`

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r389159338
 
 

 ##########
 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);
+        }
+      }
+    }
+
+    // If MD5 hash is available as part of path properties, verify it with the local file
+    DataLakeFileClient fileClient = _fileSystemClient.getFileClient(convertUriToAzureStylePath(srcUri));
+    byte[] md5FromMetadata = fileClient.getProperties().getContentMd5();
+    if (md5FromMetadata != null && md5FromMetadata.length > 0) {
+      byte[] md5FromLocalFile = md5File.digest();
+      if (!Arrays.equals(md5FromLocalFile, md5FromMetadata)) {
+        // Clean up the corrupted file
+        FileUtils.deleteQuietly(dstFile);
+        throw new IOException("Computed MD5 and MD5 from metadata do not match");
+      }
+    }
+  }
+
+  @Override
+  public void copyFromLocalFile(File srcFile, URI dstUri) throws Exception {
+    LOGGER.info("copyFromLocalFile is called with srcFile='{}', dstUri='{}'", srcFile, dstUri);
+    try (InputStream fileInputStream = new FileInputStream(srcFile)) {
+      copyInputStreamToDst(fileInputStream, dstUri);
+    }
+  }
+
+  @Override
+  public boolean isDirectory(URI uri) throws IOException {
+    try {
+      PathProperties pathProperties = getPathProperties(uri);
+      Map<String, String> metadata = pathProperties.getMetadata();
+      // TODO: need to find the other ways to check the directory if it becomes available. listFiles API returns
+      // PathInfo, which includes "isDirectory" field; however, there's no API available for fetching PathInfo directly
+      // from target uri.
+      return Boolean.valueOf(metadata.get(IS_DIRECTORY_KEY));
+    } catch (DataLakeStorageException e) {
+      throw new IOException("Failed while checking isDirectory for : " + uri, e);
+    }
+  }
+
+  @Override
+  public long lastModified(URI uri) throws IOException {
+    try {
+      PathProperties pathProperties = getPathProperties(uri);
+      OffsetDateTime offsetDateTime = pathProperties.getLastModified();
+      Timestamp timestamp = Timestamp.valueOf(offsetDateTime.atZoneSameInstant(ZoneOffset.UTC).toLocalDateTime());
+      return timestamp.getTime();
+    } catch (DataLakeStorageException e) {
+      throw new IOException("Failed while checking lastModified time for : " + uri, e);
+    }
+  }
+
+  @Override
+  public boolean touch(URI uri) throws IOException {
+    // The following data lake gen2 API provides a way to update file properties including last modified time.
+    // https://docs.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/update
+    // However, action = "setProperties" is available in REST API but not available in Java SDK yet.
+    //
+    // For now, directly use Blob service's API to get the same effect.
+    // https://docs.microsoft.com/en-us/rest/api/storageservices/set-file-properties
+    try {
+      DataLakeFileClient fileClient = _fileSystemClient.getFileClient(convertUriToAzureStylePath(uri));
+      PathProperties pathProperties = fileClient.getProperties();
+      fileClient.setHttpHeaders(getPathHttpHeaders(pathProperties));
+      return true;
+    } catch (DataLakeStorageException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public InputStream open(URI uri) throws IOException {
+    // Use Blob API since read() function from Data Lake Client currently takes "OutputStream" as an input and
+    // flush bytes to an output stream. This needs to be piped back into input stream to implement this function.
+    // On the other hand, Blob API directly allow you to open the input stream.
+    BlobClient blobClient = _blobServiceClient.getBlobContainerClient(_fileSystemClient.getFileSystemName())
+        .getBlobClient(convertUriToAzureStylePath(uri));
+
+    return blobClient.openInputStream();
+    // Another approach is to download the file to the local disk to a temp path and return the file input stream. In
+    // this case, we need to override "close()" and delete temp file.
+  }
+
+  private boolean copySrcToDst(URI srcUri, URI dstUri) throws IOException {
+    try (InputStream inputStream = open(srcUri)) {
+      return copyInputStreamToDst(inputStream, dstUri);
+    }
+  }
+
+  /**
+   * Helper function to copy input stream to destination URI.
+   *
+   * NOTE: the caller has to close the input stream.
+   *
+   * @param inputStream input stream that will be written to dstUri
+   * @param dstUri destination URI
+   * @return true if the copy succeeds
+   */
+  private boolean copyInputStreamToDst(InputStream inputStream, URI dstUri) throws IOException {
+    int bytesRead;
+    long totalBytesRead = 0;
+    byte[] buffer = new byte[BUFFER_SIZE];
+    DataLakeFileClient fileClient = _fileSystemClient.createFile(convertUriToAzureStylePath(dstUri));
+    try {
+      MessageDigest md5File = MessageDigest.getInstance("MD5");
 
 Review comment:
   discussed offline. resolving this.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] snleee commented on a change in pull request #5116: Add Azure Data Lake Gen2 connector for PinotFS

Posted by GitBox <gi...@apache.org>.
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_r392512787
 
 

 ##########
 File path: pinot-plugins/pinot-file-system/pinot-adls/src/main/java/org/apache/pinot/plugin/filesystem/AzureGen2PinotFS.java
 ##########
 @@ -0,0 +1,460 @@
+/**
+ * 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.DataLakeDirectoryClient;
+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.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 ENABLE_CHECKSUM = "enableChecksum";
+
+  private static final String HTTPS_URL_PREFIX = "https://";
+
+  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;
+
+  // If enabled, pinotFS implementation will guarantee that the bits you've read are the same as the ones you wrote.
+  // However, there's some overhead in computing hash. (Adds roughly 3 seconds for 1GB file)
+  private boolean _enableChecksum;
+
+  @Override
+  public void init(Configuration config) {
+    _enableChecksum = config.getBoolean(ENABLE_CHECKSUM, false);
+
+    // Azure storage account name
+    String accountName = config.getString(ACCOUNT_NAME);
+
+    // TODO: consider to add the encryption of the following config
+    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.error("AzureGen2PinotFS is initialized (accountName={}, fileSystemName={}, dfsServiceEndpointUrl={}, "
+            + "blobServiceEndpointUrl={}, enableChecksum={})", accountName, fileSystemName, dfsServiceEndpointUrl,
+        blobServiceEndpointUrl, _enableChecksum);
+  }
+
+  @Override
+  public boolean mkdir(URI uri) throws IOException {
+    LOGGER.debug("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(AzurePinotFSUtil.convertUriToUrlEncodedAzureStylePath(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 (uri = {})", uri, e);
 
 Review comment:
   I'm including `e`, which is exception object to the log. This should include the status code information as part of the exception stack. Do you think it's better to add status code explicitly along with \uri`?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org