[
https://issues.apache.org/jira/browse/HADOOP-11867?focusedWorklogId=666852&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-666852
]
ASF GitHub Bot logged work on HADOOP-11867:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 19/Oct/21 10:23
Start Date: 19/Oct/21 10:23
Worklog Time Spent: 10m
Work Description: bogthe commented on a change in pull request #3499:
URL: https://github.com/apache/hadoop/pull/3499#discussion_r731698772
##########
File path:
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
##########
@@ -286,8 +295,123 @@ protected int readChunk(long pos, byte[] buf, int offset,
int len,
public IOStatistics getIOStatistics() {
return IOStatisticsSupport.retrieveIOStatistics(datas);
}
+
+ public static long findChecksumOffset(long dataOffset,
Review comment:
Looks like it can either be `1` (if constructor init fails) or another
int.
##########
File path:
hadoop-common-project/benchmark/src/main/java/org/apache/hadoop/benchmark/AsyncBenchmark.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.benchmark;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileRange;
+import org.apache.hadoop.fs.FileRangeImpl;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.infra.Blackhole;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+import java.io.EOFException;
+import java.io.IOException;
+
+import java.nio.ByteBuffer;
+import java.nio.channels.AsynchronousFileChannel;
+import java.nio.channels.CompletionHandler;
+import java.nio.file.FileSystems;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.function.IntFunction;
+
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MICROSECONDS)
+public class AsyncBenchmark {
+
+ static final Path DATA_PATH = getTestDataPath();
+ static final String DATA_PATH_PROPERTY = "bench.data";
+
+ static Path getTestDataPath() {
+ String value = System.getProperty(DATA_PATH_PROPERTY);
+ return new Path(value == null ? "/tmp/taxi.orc" : value);
+ }
+
+ @State(Scope.Thread)
+ public static class FileSystemChoice {
+
+ @Param({"local", "raw"})
+ String fileSystemKind;
+
+ Configuration conf;
+ FileSystem fs;
+
+ @Setup(Level.Trial)
+ public void setup() {
+ conf = new Configuration();
+ try {
+ LocalFileSystem local = FileSystem.getLocal(conf);
+ fs = "raw".equals(fileSystemKind) ? local.getRaw() : local;
+ } catch (IOException e) {
+ throw new IllegalArgumentException("Can't get filesystem", e);
+ }
+ }
+ }
+
+ @State(Scope.Thread)
+ public static class BufferChoice {
+ @Param({"direct", "array"})
+ String bufferKind;
+
+ IntFunction<ByteBuffer> allocate;
+ @Setup(Level.Trial)
+ public void setup() {
+ allocate = "array".equals(bufferKind)
+ ? ByteBuffer::allocate : ByteBuffer::allocateDirect;
+ }
+ }
+
+ @Benchmark
+ public void asyncRead(FileSystemChoice fsChoice,
+ BufferChoice bufferChoice,
+ Blackhole blackhole) throws Exception {
+ FSDataInputStream stream = fsChoice.fs.open(DATA_PATH);
+ List<FileRange> ranges = new ArrayList<>();
+ for(int m=0; m < 100; ++m) {
+ FileRangeImpl range = new FileRangeImpl(m * 1024L * 1024, 64 * 1024);
Review comment:
Won't this move the offset every `1024*1024` bytes and only have a
length of `64*1024`? Meaning you'll read `64` and move to `1024.` The same code
is used 3 times in this file `m * 1024` so if it should be `64` make sure to
replace all 3 of them.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 666852)
Time Spent: 8h (was: 7h 50m)
> FS API: Add a high-performance vectored Read to FSDataInputStream API
> ---------------------------------------------------------------------
>
> Key: HADOOP-11867
> URL: https://issues.apache.org/jira/browse/HADOOP-11867
> Project: Hadoop Common
> Issue Type: New Feature
> Components: fs, fs/azure, fs/s3, hdfs-client
> Affects Versions: 3.0.0
> Reporter: Gopal Vijayaraghavan
> Assignee: Owen O'Malley
> Priority: Major
> Labels: performance, pull-request-available
> Time Spent: 8h
> Remaining Estimate: 0h
>
> The most significant way to read from a filesystem in an efficient way is to
> let the FileSystem implementation handle the seek behaviour underneath the
> API to be the most efficient as possible.
> A better approach to the seek problem is to provide a sequence of read
> locations as part of a single call, while letting the system schedule/plan
> the reads ahead of time.
> This is exceedingly useful for seek-heavy readers on HDFS, since this allows
> for potentially optimizing away the seek-gaps within the FSDataInputStream
> implementation.
> For seek+read systems with even more latency than locally-attached disks,
> something like a {{readFully(long[] offsets, ByteBuffer[] chunks)}} would
> take of the seeks internally while reading chunk.remaining() bytes into each
> chunk (which may be {{slice()}}ed off a bigger buffer).
> The base implementation can stub in this as a sequence of seeks + read() into
> ByteBuffers, without forcing each FS implementation to override this in any
> way.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]