[
https://issues.apache.org/jira/browse/HADOOP-19348?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17927810#comment-17927810
]
ASF GitHub Bot commented on HADOOP-19348:
-----------------------------------------
steveloughran commented on code in PR #7334:
URL: https://github.com/apache/hadoop/pull/7334#discussion_r1958458978
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStreamFactory.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.hadoop.fs.s3a.impl.streams;
+
+import org.apache.hadoop.conf.Configuration;
Review Comment:
usual nit: import ordering, and I'd prefer an explicit import of those
Constants which are being used
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java:
##########
@@ -115,7 +115,7 @@ public class RequestFactoryImpl implements RequestFactory {
/**
* Callback to prepare requests.
*/
- private final PrepareRequest requestPreparer;
+ private PrepareRequest requestPreparer;
Review Comment:
this doesn't need to be non-final any more, I shall fix in my PR
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStreamFactory.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.hadoop.fs.s3a.impl.streams;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.s3a.VectoredIOContext;
+import org.apache.hadoop.util.functional.CallableRaisingIOE;
+import org.apache.hadoop.util.functional.LazyAutoCloseableReference;
+
+import software.amazon.awssdk.services.s3.S3AsyncClient;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.s3.analyticsaccelerator.S3SdkObjectClient;
+import
software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamConfiguration;
+import software.amazon.s3.analyticsaccelerator.S3SeekableInputStreamFactory;
+import software.amazon.s3.analyticsaccelerator.common.ConnectorConfiguration;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.fs.s3a.Constants.*;
+import static
org.apache.hadoop.fs.s3a.impl.streams.StreamIntegration.populateVectoredIOContext;
+
+/**
+ * A factory for {@link AnalyticsStream}. This class is instantiated during
initialization of
+ * {@code S3AStore}, if fs.s3a.input.stream.type is set to Analytics.
+ */
+public class AnalyticsStreamFactory extends AbstractObjectInputStreamFactory {
+
+ private S3SeekableInputStreamConfiguration
seekableInputStreamConfiguration;
+ private LazyAutoCloseableReference<S3SeekableInputStreamFactory>
s3SeekableInputStreamFactory;
+ private boolean requireCrt;
+
+ public AnalyticsStreamFactory() {
+ super("AnalyticsStreamFactory");
+ }
+
+ @Override
+ protected void serviceInit(final Configuration conf) throws Exception {
+ super.serviceInit(conf);
+ ConnectorConfiguration configuration = new ConnectorConfiguration(conf,
+ ANALYTICS_ACCELERATOR_CONFIGURATION_PREFIX);
+ this.seekableInputStreamConfiguration =
+
S3SeekableInputStreamConfiguration.fromConfiguration(configuration);
+ this.requireCrt = false;
+ }
+
+ @Override
+ public void bind(final FactoryBindingParameters factoryBindingParameters)
throws IOException {
+ super.bind(factoryBindingParameters);
+ this.s3SeekableInputStreamFactory = new
LazyAutoCloseableReference<>(createS3SeekableInputStreamFactory());
Review Comment:
can you chop this line down..it's too wide fo side-by-side reviews
##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/streams/AnalyticsStream.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.hadoop.fs.s3a.impl.streams;
+
+import java.io.EOFException;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FSExceptionMessages;
Review Comment:
nit, imports are out of order
> S3A: Add initial support for analytics-accelerator-s3
> -----------------------------------------------------
>
> Key: HADOOP-19348
> URL: https://issues.apache.org/jira/browse/HADOOP-19348
> Project: Hadoop Common
> Issue Type: Sub-task
> Components: fs/s3
> Affects Versions: 3.4.2
> Reporter: Ahmar Suhail
> Priority: Major
> Labels: pull-request-available
>
> S3 recently released [Analytics Accelerator Library for Amazon
> S3|https://github.com/awslabs/analytics-accelerator-s3] as an Alpha release,
> which is an input stream, with an initial goal of improving performance for
> Apache Spark workloads on Parquet datasets.
> For example, it implements optimisations such as footer prefetching, and so
> avoids the multiple GETS S3AInputStream currently makes for the footer bytes
> and PageIndex structures.
> The library also tracks columns currently being read by a query using the
> parquet metadata, and then prefetches these bytes when parquet files with the
> same schema are opened.
> This ticket tracks the work required for the basic initial integration. There
> is still more work to be done, such as VectoredIO support etc, which we will
> identify and follow up with.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]