From 8eba32919f2a1e467ea443ea0171d379a521853c Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Mon, 6 Sep 2021 19:25:48 +0200 Subject: [PATCH 01/19] finish asynchronous s3 reader. --- pixels-common/pom.xml | 7 + .../pixels/common/lock/EtcdAutoIncrement.java | 6 +- .../common/physical/PhysicalReader.java | 14 +- .../common/physical/PhysicalReaderUtil.java | 31 ++- .../common/physical/PhysicalWriter.java | 1 + .../common/physical/PhysicalWriterUtil.java | 29 +-- .../pixels/common/physical/Storage.java | 1 - .../physical/impl/PhysicalHDFSReader.java | 29 ++- .../physical/impl/PhysicalLocalReader.java | 52 +++- .../physical/impl/PhysicalLocalWriter.java | 23 ++ .../physical/impl/PhysicalS3Reader.java | 236 ++++++++++++++++++ .../physical/impl/PhysicalS3Writer.java | 101 ++++++++ .../pixels/common/physical/impl/S3.java | 236 +++++++++++++++++- .../pixels/common/utils/Constants.java | 9 +- .../pixels/common/utils/EtcdUtil.java | 2 +- .../src/main/resources/pixels.properties | 1 + pom.xml | 10 + 17 files changed, 722 insertions(+), 66 deletions(-) create mode 100644 pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java create mode 100644 pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java diff --git a/pixels-common/pom.xml b/pixels-common/pom.xml index e940ac77b..2563e91fc 100644 --- a/pixels-common/pom.xml +++ b/pixels-common/pom.xml @@ -48,6 +48,13 @@ true + + + software.amazon.awssdk + s3 + true + + com.google.protobuf diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/lock/EtcdAutoIncrement.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/lock/EtcdAutoIncrement.java index f23a334f2..f6e33302c 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/lock/EtcdAutoIncrement.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/lock/EtcdAutoIncrement.java @@ -24,7 +24,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import static io.pixelsdb.pixels.common.utils.Constants.LOCK_PATH_PREFIX; +import static io.pixelsdb.pixels.common.utils.Constants.AI_LOCK_PATH_PREFIX; /** * Created at: 8/29/21 @@ -45,7 +45,7 @@ public static void InitId(String idKey) { EtcdUtil etcd = EtcdUtil.Instance(); EtcdReadWriteLock readWriteLock = new EtcdReadWriteLock(etcd.getClient(), - LOCK_PATH_PREFIX + idKey); + AI_LOCK_PATH_PREFIX + idKey); EtcdMutex writeLock = readWriteLock.writeLock(); try { @@ -77,7 +77,7 @@ public static long GenerateId(String idKey) long id = 0; EtcdUtil etcd = EtcdUtil.Instance(); EtcdReadWriteLock readWriteLock = new EtcdReadWriteLock(etcd.getClient(), - LOCK_PATH_PREFIX + idKey); + AI_LOCK_PATH_PREFIX + idKey); EtcdMutex writeLock = readWriteLock.writeLock(); try { diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalReader.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalReader.java index 88705262d..db47c3832 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalReader.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalReader.java @@ -21,9 +21,12 @@ import java.io.Closeable; import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.concurrent.CompletableFuture; /** * @author guodong + * @author hank */ public interface PhysicalReader extends Closeable @@ -32,14 +35,19 @@ public interface PhysicalReader void seek(long desired) throws IOException; - int read(byte[] buffer) throws IOException; - - int read(byte[] buffer, int offset, int length) throws IOException; + ByteBuffer readFully(int length) throws IOException; void readFully(byte[] buffer) throws IOException; void readFully(byte[] buffer, int offset, int length) throws IOException; + /** + * @return true if readAsync is supported. + */ + boolean supportsAsync(); + + CompletableFuture readAsync(int length) throws IOException; + long readLong() throws IOException; int readInt() throws IOException; diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalReaderUtil.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalReaderUtil.java index 4d601ac56..f5dbd3cf8 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalReaderUtil.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalReaderUtil.java @@ -21,6 +21,7 @@ import io.pixelsdb.pixels.common.physical.impl.PhysicalHDFSReader; import io.pixelsdb.pixels.common.physical.impl.PhysicalLocalReader; +import io.pixelsdb.pixels.common.physical.impl.PhysicalS3Reader; import java.io.IOException; @@ -40,23 +41,21 @@ public static PhysicalReader newPhysicalReader(Storage storage, String path) thr { checkArgument(storage != null, "storage should not be null"); checkArgument(path != null, "path should not be null"); - PhysicalReader reader = null; - try - { - switch (storage.getScheme()) - { - case hdfs: - reader = new PhysicalHDFSReader(storage, path); - break; - case file: - reader = new PhysicalLocalReader(storage, path); - break; - case s3: - throw new IOException("S3 storage is not supported"); - } - } catch (IOException e) + + PhysicalReader reader; + switch (storage.getScheme()) { - throw e; + case hdfs: + reader = new PhysicalHDFSReader(storage, path); + break; + case file: + reader = new PhysicalLocalReader(storage, path); + break; + case s3: + reader = new PhysicalS3Reader(storage, path); + break; + default: + throw new IOException("Storage scheme '" + storage.getScheme() + "' is not supported."); } return reader; diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalWriter.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalWriter.java index 496a95795..9294d4f2a 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalWriter.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalWriter.java @@ -25,6 +25,7 @@ /** * @author guodong + * @author hank */ public interface PhysicalWriter extends Closeable diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalWriterUtil.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalWriterUtil.java index 1f4ff7f11..6b5a37a58 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalWriterUtil.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalWriterUtil.java @@ -21,6 +21,7 @@ import io.pixelsdb.pixels.common.physical.impl.PhysicalHDFSWriter; import io.pixelsdb.pixels.common.physical.impl.PhysicalLocalWriter; +import io.pixelsdb.pixels.common.physical.impl.PhysicalS3Writer; import java.io.IOException; @@ -41,21 +42,21 @@ public static PhysicalWriter newPhysicalWriter(Storage storage, String path, lon { checkArgument(storage != null, "storage should not be null"); checkArgument(path != null, "path should not be null"); - PhysicalWriter writer = null; - try - { - switch (storage.getScheme()) - { - case hdfs: - writer = new PhysicalHDFSWriter(storage, path, replication, addBlockPadding, blockSize); - case file: - writer = new PhysicalLocalWriter(storage, path); - case s3: - throw new IOException("S3 storage is not supported."); - } - } catch (IOException e) + + PhysicalWriter writer; + switch (storage.getScheme()) { - throw e; + case hdfs: + writer = new PhysicalHDFSWriter(storage, path, replication, addBlockPadding, blockSize); + break; + case file: + writer = new PhysicalLocalWriter(storage, path); + break; + case s3: + writer = new PhysicalS3Writer(storage, path); + break; + default: + throw new IOException("Storage scheme '" + storage.getScheme() + "' is not supported."); } return writer; diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Storage.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Storage.java index 9c7b7b4ff..962a31d51 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Storage.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Storage.java @@ -129,7 +129,6 @@ public boolean equals(Scheme other) */ DataInputStream open(String path) throws IOException; - /** * For local fs, path is considered as local. * @param path diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalHDFSReader.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalHDFSReader.java index 64a0aa751..96ea5a604 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalHDFSReader.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalHDFSReader.java @@ -28,10 +28,12 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.concurrent.CompletableFuture; /** * @author guodong @@ -172,15 +174,11 @@ public int readInt() throws IOException } @Override - public int read(byte[] buffer) throws IOException + public ByteBuffer readFully(int length) throws IOException { - return rawReader.read(buffer); - } - - @Override - public int read(byte[] buffer, int offset, int length) throws IOException - { - return rawReader.read(buffer, offset, length); + ByteBuffer buffer = ByteBuffer.allocate(length); + rawReader.readFully(buffer.array()); + return buffer; } @Override @@ -195,6 +193,21 @@ public void readFully(byte[] buffer, int offset, int length) throws IOException rawReader.readFully(buffer, offset, length); } + /** + * @return true if readAsync is supported. + */ + @Override + public boolean supportsAsync() + { + return false; + } + + @Override + public CompletableFuture readAsync(int length) throws IOException + { + throw new IOException("Asynchronous read is not supported for HDFS."); + } + @Override public void close() throws IOException { diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalReader.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalReader.java index f9013dd87..6bd7805da 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalReader.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalReader.java @@ -1,3 +1,22 @@ +/* + * Copyright 2021 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ package io.pixelsdb.pixels.common.physical.impl; import io.pixelsdb.pixels.common.physical.PhysicalReader; @@ -5,6 +24,8 @@ import java.io.IOException; import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.util.concurrent.CompletableFuture; /** * Created at: 30/08/2021 @@ -27,6 +48,10 @@ public PhysicalLocalReader(Storage storage, String path) throws IOException { throw new IOException("Storage is not LocalFS."); } + if (path.startsWith("file://")) + { + path = path.substring(7); + } this.path = path; this.raf = this.local.openRaf(path); this.id = this.local.getFileId(path); @@ -45,15 +70,11 @@ public void seek(long desired) throws IOException } @Override - public int read(byte[] buffer) throws IOException + public ByteBuffer readFully(int length) throws IOException { - return raf.read(buffer); - } - - @Override - public int read(byte[] buffer, int offset, int length) throws IOException - { - return raf.read(buffer, offset, length); + ByteBuffer buffer = ByteBuffer.allocate(length); + raf.readFully(buffer.array()); + return buffer; } @Override @@ -68,6 +89,21 @@ public void readFully(byte[] buffer, int offset, int length) throws IOException raf.readFully(buffer, offset, length); } + /** + * @return true if readAsync is supported. + */ + @Override + public boolean supportsAsync() + { + return false; + } + + @Override + public CompletableFuture readAsync(int length) throws IOException + { + throw new IOException("Asynchronous read is not supported for local fs."); + } + @Override public long readLong() throws IOException { diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalWriter.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalWriter.java index eace3fa06..bd81d44b8 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalWriter.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalWriter.java @@ -1,3 +1,22 @@ +/* + * Copyright 2021 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ package io.pixelsdb.pixels.common.physical.impl; import io.pixelsdb.pixels.common.physical.PhysicalWriter; @@ -29,6 +48,10 @@ public PhysicalLocalWriter(Storage storage, String path) throws IOException { throw new IOException("Storage is not LocalFS."); } + if (path.startsWith("file://")) + { + path = path.substring(7); + } this.path = path; this.position = 0; this.rawWriter = this.local.create(path, false, Constants.LOCAL_BUFFER_SIZE, (short) 1); diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java new file mode 100644 index 000000000..2cd8e6a0a --- /dev/null +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java @@ -0,0 +1,236 @@ +/* + * Copyright 2021 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ +package io.pixelsdb.pixels.common.physical.impl; + +import io.pixelsdb.pixels.common.physical.PhysicalReader; +import io.pixelsdb.pixels.common.physical.Storage; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import software.amazon.awssdk.core.ResponseBytes; +import software.amazon.awssdk.core.async.AsyncResponseTransformer; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Created at: 06/09/2021 + * Author: hank + */ +public class PhysicalS3Reader implements PhysicalReader +{ + private static Logger logger = LogManager.getLogger(PhysicalS3Reader.class); + + private S3 s3; + private S3.Path path; + private String pathStr; + private long id; + private AtomicLong position; + private long length; + private S3AsyncClient client; + + public PhysicalS3Reader(Storage storage, String path) throws IOException + { + if (storage instanceof LocalFS) + { + this.s3 = (S3) storage; + } + else + { + throw new IOException("Storage is not S3."); + } + if (path.startsWith("s3://")) + { + path = path.substring(5); + } + this.path = new S3.Path(path); + this.pathStr = path; + this.id = this.s3.getFileId(path); + this.length = this.s3.getStatus(path).getLength(); + this.position = new AtomicLong(0); + this.client = s3.getClient(); + } + + private String toRange(long start, int length) + { + StringBuilder builder = new StringBuilder("bytes="); + builder.append(start).append('-').append(start+length); + return builder.toString(); + } + + @Override + public long getFileLength() throws IOException + { + return length; + } + + @Override + public void seek(long desired) throws IOException + { + if (0 <= desired && desired < length) + { + position.set(desired); + return; + } + throw new IOException("Desired offset is out of bound."); + } + + @Override + public ByteBuffer readFully(int len) throws IOException + { + if (this.position.get() + len > this.length) + { + throw new IOException("Current position " + this.position.get() + " plus " + + len + " exceeds object length " + this.length + "."); + } + GetObjectRequest request = GetObjectRequest.builder().bucket(path.bucket) + .key(path.key).range(toRange(position.get(), len)).build(); + CompletableFuture> future = + client.getObject(request, AsyncResponseTransformer.toBytes()); + try + { + this.position.addAndGet(len); + return ByteBuffer.wrap(future.get().asByteArray()); + } catch (Exception e) + { + throw new IOException("Failed to read object.", e); + } + } + + @Override + public void readFully(byte[] buffer) throws IOException + { + ByteBuffer byteBuffer = readFully(buffer.length); + System.arraycopy(byteBuffer.array(), 0, buffer, 0, buffer.length); + } + + @Override + public void readFully(byte[] buffer, int off, int len) throws IOException + { + ByteBuffer byteBuffer = readFully(len); + System.arraycopy(byteBuffer.array(), 0, buffer, off, len); + } + + /** + * @return true if readAsync is supported. + */ + @Override + public boolean supportsAsync() + { + return true; + } + + @Override + public CompletableFuture readAsync(int len) throws IOException + { + if (this.position.get() + len > this.length) + { + throw new IOException("Current position " + this.position.get() + " plus " + + len + " exceeds object length " + this.length + "."); + } + GetObjectRequest request = GetObjectRequest.builder().bucket(path.bucket) + .key(path.key).range(toRange(position.get(), len)).build(); + CompletableFuture> future = + client.getObject(request, AsyncResponseTransformer.toBytes()); + try + { + CompletableFuture futureBuffer = new CompletableFuture<>(); + future.whenComplete((resp, err) -> + { + if (resp != null) + { + futureBuffer.complete(ByteBuffer.wrap(resp.asByteArray())); + this.position.addAndGet(len); + } + else + { + logger.error("Failed complete the asynchronous read.", err); + err.printStackTrace(); + } + }); + return futureBuffer; + } catch (Exception e) + { + throw new IOException("Failed to read object.", e); + } + } + + @Override + public long readLong() throws IOException + { + ByteBuffer buffer = readFully(Long.BYTES); + return buffer.getLong(); + } + + @Override + public int readInt() throws IOException + { + ByteBuffer buffer = readFully(Integer.BYTES); + return buffer.getInt(); + } + + @Override + public void close() throws IOException + { + this.client.close(); + } + + @Override + public String getPath() + { + return pathStr; + } + + /** + * Get the last domain in path. + * + * @return + */ + @Override + public String getName() + { + return path.key; + } + + /** + * For a file or object in the storage, it may have one or more + * blocks. Each block has its unique id. This method returns the + * block id of the current block that is been reading. + *

+ * For local fs, each file has only one block id, which is also + * the file id. + * + *

Note: Storage.getFileId() assumes that each file or object + * only has one block. In this case, the file id is the same as + * the block id here.

+ * + * @return + * @throws IOException + */ + @Override + public long getBlockId() throws IOException + { + return s3.getFileId(pathStr); + } +} diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java new file mode 100644 index 000000000..5504cac69 --- /dev/null +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java @@ -0,0 +1,101 @@ +/* + * Copyright 2021 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ +package io.pixelsdb.pixels.common.physical.impl; + +import io.pixelsdb.pixels.common.physical.PhysicalWriter; +import io.pixelsdb.pixels.common.physical.Storage; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Created at: 06/09/2021 + * Author: hank + */ +public class PhysicalS3Writer implements PhysicalWriter +{ + public PhysicalS3Writer(Storage storage, String path) throws IOException + { + + } + + /** + * Prepare the writer to ensure the length can fit into current block. + * + * @param length length of content + * @return starting offset after preparing. If -1, means prepare has failed, + * due to the specified length cannot fit into current block. + */ + @Override + public long prepare(int length) throws IOException + { + return 0; + } + + /** + * Append content to the file. + * + * @param buffer content buffer + * @return start offset of content in the file. + */ + @Override + public long append(ByteBuffer buffer) throws IOException + { + return 0; + } + + /** + * Append content to the file. + * + * @param buffer content buffer container + * @param offset start offset of actual content buffer + * @param length length of actual content buffer + * @return start offset of content in the file. + */ + @Override + public long append(byte[] buffer, int offset, int length) throws IOException + { + return 0; + } + + /** + * Close writer. + */ + @Override + public void close() throws IOException + { + + } + + /** + * Flush writer. + */ + @Override + public void flush() throws IOException + { + + } + + @Override + public String getPath() + { + return null; + } +} diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java index 1f89f26f2..fa562fc59 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java @@ -19,24 +19,110 @@ */ package io.pixelsdb.pixels.common.physical.impl; +import com.google.common.collect.ImmutableList; +import io.etcd.jetcd.KeyValue; import io.pixelsdb.pixels.common.physical.Location; import io.pixelsdb.pixels.common.physical.Status; import io.pixelsdb.pixels.common.physical.Storage; +import io.pixelsdb.pixels.common.utils.EtcdUtil; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import software.amazon.awssdk.core.ResponseBytes; +import software.amazon.awssdk.core.async.AsyncResponseTransformer; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3AsyncClientBuilder; +import software.amazon.awssdk.services.s3.model.*; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.List; +import static io.pixelsdb.pixels.common.lock.EtcdAutoIncrement.GenerateId; +import static io.pixelsdb.pixels.common.lock.EtcdAutoIncrement.InitId; +import static io.pixelsdb.pixels.common.utils.Constants.*; +import static java.util.Objects.requireNonNull; + /** + *

+ * For S3, we assume that each table is stores as one bucket. + * And all the pixels files in this table are store as separate objects + * in the bucket. We do not use object key prefix for performance + * considerations. + *

+ *
* Created at: 20/08/2021 * Author: hank */ public class S3 implements Storage { + private static Logger logger = LogManager.getLogger(S3.class); + + private static S3AsyncClient s3; + + static + { + InitId(S3_ID_KEY); + + S3AsyncClientBuilder builder = S3AsyncClient.builder(); + // TODO: config builder. + s3 = builder.build(); + } + + private String[] allHosts; + private int hostIndex = 0; + public S3() { + List kvs = EtcdUtil.Instance().getKeyValuesByPrefix(CACHE_NODE_STATUS_LITERAL); + allHosts = new String[kvs.size()]; + for (int i = 0; i < kvs.size(); ++i) + { + String key = kvs.get(i).getKey().toString(StandardCharsets.UTF_8); + allHosts[i] = key.substring(CACHE_NODE_STATUS_LITERAL.length()); + } + } + + private String getPathKey(String path) + { + return S3_META_PREFIX + path; + } + + private String getPathFrom(String key) + { + if (key.startsWith(S3_META_PREFIX)) + { + key.substring(S3_META_PREFIX.length()); + } + return null; + } + + public static class Path + { + public String bucket = null; + public String key = null; + boolean valid = false; + boolean isBucket = false; + public Path(String path) + { + requireNonNull(path); + int slash = path.indexOf("/"); + if (slash > 0) + { + this.bucket = path.substring(0, slash); + if (slash < path.length()-1) + { + this.key = path.substring(slash + 1); + } + else + { + isBucket = true; + } + this.valid = true; + } + } } @Override @@ -57,63 +143,191 @@ public List listPaths(String path) throws IOException return null; } + /** + * For S3, the replication is always 1. + * @param path + * @return + * @throws IOException + */ @Override - public Status getStatus(String path) + public Status getStatus(String path) throws IOException { - return null; + Path p = new Path(path); + if (!p.valid) + { + throw new IOException("Path '" + path + "' is not valid."); + } + if (!this.exists(path)) + { + throw new IOException("Path '" + path + "' does not exist."); + } + if (p.isBucket) + { + return new Status(path, 0, true, 1); + } + HeadObjectRequest request = HeadObjectRequest.builder().bucket(p.bucket).key(p.key).build(); + try + { + HeadObjectResponse response = s3.headObject(request).get(); + return new Status(path, response.contentLength(), false, 1); + } catch (Exception e) + { + throw new IOException("Failed to get object head of '" + path + "'", e); + } } @Override public long getFileId(String path) throws IOException { - return 0; + KeyValue kv = EtcdUtil.Instance().getKeyValue(getPathKey(path)); + return Long.parseLong(kv.getValue().toString(StandardCharsets.UTF_8)); } @Override public List getLocations(String path) { - return null; + String host = allHosts[hostIndex++]; + if (hostIndex >= allHosts.length) + { + hostIndex = 0; + } + return ImmutableList.of(new Location(new String[]{host})); } + /** + * For S3, we do not have the concept host. + * When S3 is used as the storage, disable enable.absolute.balancer. + * @param path + * @return + * @throws IOException + */ @Override public String[] getHosts(String path) throws IOException { - return new String[0]; + return allHosts; } + /** + * For S3, this open method is only used to read the data object + * fully and sequentially. And it will load the whole object into + * memory, so be careful for large objects. + * @param path + * @return + * @throws IOException + */ @Override - public DataInputStream open(String path) + public DataInputStream open(String path) throws IOException { - return null; + Path p = new Path(path); + if (!p.valid) + { + throw new IOException("Path '" + path + "' is not valid."); + } + if (!this.exists(path)) + { + throw new IOException("Path '" + path + "' does not exist."); + } + GetObjectRequest request = GetObjectRequest.builder().bucket(p.bucket).key(p.key).build(); + try + { + ResponseBytes get = + s3.getObject(request, AsyncResponseTransformer.toBytes()).get(); + return new DataInputStream(get.asInputStream()); + } catch (Exception e) + { + throw new IOException("Failed to get object '" + path + "'.", e); + } } + /** + * As S3 does not support append, we do not create object, + * only create the file id and metadata in etcd. + * @param path + * @param overwrite + * @param bufferSize + * @param replication + * @return always return null. + * @throws IOException + */ @Override public DataOutputStream create(String path, boolean overwrite, int bufferSize, short replication) throws IOException { + Path p = new Path(path); + if (!p.valid) + { + throw new IOException("Path '" + path + "' is not valid."); + } + if (this.exists(path)) + { + throw new IOException("Path '" + path + "' already exists."); + } + long id = GenerateId(S3_ID_KEY); + EtcdUtil.Instance().putKeyValue(getPathKey(path), Long.toString(id)); return null; } @Override public boolean delete(String path, boolean recursive) throws IOException { - return false; + Path p = new Path(path); + if (!p.valid) + { + throw new IOException("Path '" + path + "' is not valid."); + } + if (!this.exists(path)) + { + throw new IOException("Path '" + path + "' does not exist."); + } + if (p.isBucket) + { + List kvs = EtcdUtil.Instance().getKeyValuesByPrefix(getPathKey(p.bucket)); + for (KeyValue kv : kvs) + { + Path sub = new Path(getPathFrom(kv.getKey().toString(StandardCharsets.UTF_8))); + DeleteObjectRequest request = DeleteObjectRequest.builder().bucket(sub.bucket).key(sub.key).build(); + try + { + s3.deleteObject(request).get(); + } catch (Exception e) + { + throw new IOException("Failed to delete object '" + sub.bucket + "/" + sub.key + "' from S3.", e); + } + } + } + else + { + DeleteObjectRequest request = DeleteObjectRequest.builder().bucket(p.bucket).key(p.key).build(); + try + { + s3.deleteObject(request).get(); + } catch (Exception e) + { + throw new IOException("Failed to delete object '" + p.bucket + "/" + p.key + "' from S3.", e); + } + } + return true; } @Override public boolean exists(String path) { - return false; + return EtcdUtil.Instance().getKeyValue(getPathKey(path)) != null; } @Override public boolean isFile(String path) { - return false; + return !(new Path(path).isBucket); } @Override public boolean isDirectory(String path) { - return false; + return new Path(path).isBucket; + } + + public S3AsyncClient getClient() + { + return s3; } } diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/Constants.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/Constants.java index 721a8c02a..3b70d24e2 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/Constants.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/Constants.java @@ -48,7 +48,14 @@ public final class Constants * Issue #108: * The prefix for read-write lock used in etcd auto-increment id. */ - public static final String LOCK_PATH_PREFIX = "/pixels_lock/"; + public static final String AI_LOCK_PATH_PREFIX = "/pixels_ai_lock/"; + public static final String LOCAL_FS_ID_KEY = "pixels_storage_local_id"; + // the prefix for keys of local fs metadata (i.e. file path -> file id). public static final String LOCAL_FS_META_PREFIX = "pixels_storage_local_meta:"; + + public static final String S3_ID_KEY = "pixels_storage_s3_id"; + // the prefix for keys of s3 metadata (i.e. file path -> file id). + public static final String S3_META_PREFIX = "pixels_storage_s3_meta:"; + } diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/EtcdUtil.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/EtcdUtil.java index ad047503e..8a104da91 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/EtcdUtil.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/EtcdUtil.java @@ -80,7 +80,7 @@ public Client getClient() * you should ensure that there is only one value for this key. * * @param key etcdKey - * @return + * @return null if the key is not found. */ public KeyValue getKeyValue(String key) { diff --git a/pixels-common/src/main/resources/pixels.properties b/pixels-common/src/main/resources/pixels.properties index 3797a724e..9f29997aa 100755 --- a/pixels-common/src/main/resources/pixels.properties +++ b/pixels-common/src/main/resources/pixels.properties @@ -70,6 +70,7 @@ storage.scheme=hdfs cache.schema=pixels cache.table=test_105 lease.ttl.seconds=30 +# set to false if storage.scheme is S3 enable.absolute.balancer=false cache.enabled=true cache.read.direct=false diff --git a/pom.xml b/pom.xml index c744d3ead..7978ea4cf 100644 --- a/pom.xml +++ b/pom.xml @@ -85,6 +85,7 @@ 4.13.1 1.6.2 0.5.0-beta + 2.16.60 @@ -224,6 +225,15 @@ ${dep.hadoop.version}
+ + + software.amazon.awssdk + bom + ${dep.awssdk.version} + pom + import + + io.airlift From 1bc12e16513f90c5dde7c92cc128569d11f8f796 Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Wed, 8 Sep 2021 13:33:04 +0200 Subject: [PATCH 02/19] fix s3 reader. --- .../physical/impl/PhysicalLocalReader.java | 1 + .../physical/impl/PhysicalLocalWriter.java | 1 + .../physical/impl/PhysicalS3Reader.java | 3 +- .../physical/impl/PhysicalS3Writer.java | 34 ++++++++++++++++++- 4 files changed, 37 insertions(+), 2 deletions(-) diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalReader.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalReader.java index 6bd7805da..d7c03f996 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalReader.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalReader.java @@ -50,6 +50,7 @@ public PhysicalLocalReader(Storage storage, String path) throws IOException } if (path.startsWith("file://")) { + // remove the scheme. path = path.substring(7); } this.path = path; diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalWriter.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalWriter.java index bd81d44b8..28e0cc53e 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalWriter.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalWriter.java @@ -50,6 +50,7 @@ public PhysicalLocalWriter(Storage storage, String path) throws IOException } if (path.startsWith("file://")) { + // remove the scheme. path = path.substring(7); } this.path = path; diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java index 2cd8e6a0a..98c395b79 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java @@ -52,7 +52,7 @@ public class PhysicalS3Reader implements PhysicalReader public PhysicalS3Reader(Storage storage, String path) throws IOException { - if (storage instanceof LocalFS) + if (storage instanceof S3) { this.s3 = (S3) storage; } @@ -62,6 +62,7 @@ public PhysicalS3Reader(Storage storage, String path) throws IOException } if (path.startsWith("s3://")) { + // remove the scheme. path = path.substring(5); } this.path = new S3.Path(path); diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java index 5504cac69..5e58f6472 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java @@ -21,9 +21,13 @@ import io.pixelsdb.pixels.common.physical.PhysicalWriter; import io.pixelsdb.pixels.common.physical.Storage; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import software.amazon.awssdk.services.s3.S3AsyncClient; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicLong; /** * Created at: 06/09/2021 @@ -31,9 +35,37 @@ */ public class PhysicalS3Writer implements PhysicalWriter { + private static Logger logger = LogManager.getLogger(PhysicalS3Writer.class); + + private S3 s3; + private S3.Path path; + private String pathStr; + private long id; + private AtomicLong position; + private long length; + private S3AsyncClient client; + public PhysicalS3Writer(Storage storage, String path) throws IOException { - + if (storage instanceof S3) + { + this.s3 = (S3) storage; + } + else + { + throw new IOException("Storage is not S3."); + } + if (path.startsWith("s3://")) + { + // remove the scheme. + path = path.substring(5); + } + this.path = new S3.Path(path); + this.pathStr = path; + this.id = this.s3.getFileId(path); + this.length = this.s3.getStatus(path).getLength(); + this.position = new AtomicLong(0); + this.client = s3.getClient(); } /** From f5ac661323a9ac2e15cfc588e87cf18e12ab33f3 Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Wed, 8 Sep 2021 21:49:10 +0200 Subject: [PATCH 03/19] finish s3 writer. --- .../physical/impl/PhysicalS3Writer.java | 48 ++++++++++++++----- .../pixels/common/utils/Constants.java | 1 + .../io/pixelsdb/pixels/common/TestS3.java | 46 ++++++++++++++++++ 3 files changed, 82 insertions(+), 13 deletions(-) create mode 100644 pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java index 5e58f6472..1a460fa17 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java @@ -24,10 +24,15 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import java.io.File; +import java.io.FileOutputStream; import java.io.IOException; +import java.io.OutputStream; import java.nio.ByteBuffer; -import java.util.concurrent.atomic.AtomicLong; + +import static io.pixelsdb.pixels.common.utils.Constants.S3_BUFFER_SIZE; /** * Created at: 06/09/2021 @@ -40,10 +45,10 @@ public class PhysicalS3Writer implements PhysicalWriter private S3 s3; private S3.Path path; private String pathStr; - private long id; - private AtomicLong position; - private long length; + private long position; private S3AsyncClient client; + private File tempFile; + private OutputStream out; public PhysicalS3Writer(Storage storage, String path) throws IOException { @@ -62,10 +67,11 @@ public PhysicalS3Writer(Storage storage, String path) throws IOException } this.path = new S3.Path(path); this.pathStr = path; - this.id = this.s3.getFileId(path); - this.length = this.s3.getStatus(path).getLength(); - this.position = new AtomicLong(0); + this.position = 0L; this.client = s3.getClient(); + this.s3.create(path, false, S3_BUFFER_SIZE, (short)1); + this.tempFile = File.createTempFile("pixels-s3-", ".tmp"); + this.out = new FileOutputStream(tempFile); } /** @@ -78,7 +84,7 @@ public PhysicalS3Writer(Storage storage, String path) throws IOException @Override public long prepare(int length) throws IOException { - return 0; + return position; } /** @@ -90,7 +96,9 @@ public long prepare(int length) throws IOException @Override public long append(ByteBuffer buffer) throws IOException { - return 0; + buffer.flip(); + int length = buffer.remaining(); + return append(buffer.array(), buffer.arrayOffset() + buffer.position(), length); } /** @@ -104,7 +112,10 @@ public long append(ByteBuffer buffer) throws IOException @Override public long append(byte[] buffer, int offset, int length) throws IOException { - return 0; + long start = position; + this.out.write(buffer, offset, length); + position += length; + return start; } /** @@ -113,7 +124,18 @@ public long append(byte[] buffer, int offset, int length) throws IOException @Override public void close() throws IOException { - + this.out.close(); + PutObjectRequest request = PutObjectRequest.builder() + .bucket(path.bucket).key(path.key).build(); + try + { + this.client.putObject(request, this.tempFile.toPath()).get(); + } catch (Exception e) + { + throw new IOException("Failed to put local temp file to S3.", e); + } + this.tempFile.deleteOnExit(); + this.client.close(); } /** @@ -122,12 +144,12 @@ public void close() throws IOException @Override public void flush() throws IOException { - + this.out.flush(); } @Override public String getPath() { - return null; + return pathStr; } } diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/Constants.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/Constants.java index 3b70d24e2..2d5e3beb9 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/Constants.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/Constants.java @@ -30,6 +30,7 @@ public final class Constants public static final int DEFAULT_HDFS_BLOCK_SIZE = 256 * 1024 * 1024; public static final int HDFS_BUFFER_SIZE = 256 * 1024; public static final int LOCAL_BUFFER_SIZE = 256 * 1024; + public static final int S3_BUFFER_SIZE = 256 * 1024; public static final int MIN_REPEAT = 3; public static final int MAX_SCOPE = 512; diff --git a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java new file mode 100644 index 000000000..4977ada1e --- /dev/null +++ b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java @@ -0,0 +1,46 @@ +package io.pixelsdb.pixels.common; + +import io.pixelsdb.pixels.common.physical.*; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.concurrent.CompletableFuture; + +/** + * Created at: 9/8/21 + * Author: hank + */ +public class TestS3 +{ + @Test + public void testS3Writer() throws IOException + { + PhysicalWriter writer = PhysicalWriterUtil.newPhysicalWriter(Storage.Scheme.s3, "pixels-01/object-3", + 0, (short) 1, false); + ByteBuffer buffer = ByteBuffer.allocate(1024); + buffer.putLong(1); + writer.append(buffer); + writer.flush(); + writer.close(); + } + + @Test + public void testS3Reader() throws IOException + { + PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(Storage.Scheme.s3, "pixels-01/object-0"); + CompletableFuture future = reader.readAsync(1024); + future.whenComplete((resp, err) -> + { + if (resp != null) + { + System.out.println(resp.getLong()); + } + else + { + err.printStackTrace(); + } + }); + future.join(); + } +} From 5eb24c92f7de8ad7e3d94906d2ef5f8601bd0c3b Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Sun, 12 Sep 2021 05:02:31 +0200 Subject: [PATCH 04/19] implement read scheduler. --- .../pixels/common/physical/Scheduler.java | 120 ++++++++++ .../common/physical/SchedulerFactory.java | 63 ++++++ .../impl/scheduler/NoopScheduler.java | 83 +++++++ .../impl/scheduler/SortMergeScheduler.java | 208 ++++++++++++++++++ .../pixels/common/utils/ConfigFactory.java | 40 +++- .../src/main/resources/pixels.properties | 3 + 6 files changed, 514 insertions(+), 3 deletions(-) create mode 100644 pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Scheduler.java create mode 100644 pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/SchedulerFactory.java create mode 100644 pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java create mode 100644 pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Scheduler.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Scheduler.java new file mode 100644 index 000000000..4159da546 --- /dev/null +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Scheduler.java @@ -0,0 +1,120 @@ +/* + * Copyright 2021 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ +package io.pixelsdb.pixels.common.physical; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.concurrent.CompletableFuture; + +/** + * The interface for the reading request schedulers. + * From each thread, call the addRequest() method to add read + * requests, and then execute them in a + * Created at: 9/10/21 + * Author: hank + */ +public interface Scheduler +{ + CompletableFuture executeBatch(PhysicalReader reader, RequestBatch batch) throws IOException; + + class Request implements Comparable + { + public long start; + public int length; + + public Request(long start, int length) + { + this.start = start; + this.length = length; + } + + @Override + public int hashCode() + { + return (int) ((this.start<<32)>>32); + } + + @Override + public boolean equals(Object obj) + { + if (obj instanceof Request) + { + Request r = (Request) obj; + return this.start == r.start && + this.length == r.length; + } + return false; + } + + @Override + public int compareTo(Request o) + { + return (int) (this.start - o.start); + } + } + + class RequestBatch + { + private int pos; + private int size; + private Request[] requests; + private CompletableFuture[] futures; + + @SuppressWarnings("unchecked") + public RequestBatch(int size) + { + if (size <= 0) + { + throw new IllegalArgumentException("Request batch size: " + size); + } + this.requests = new Request[size]; + this.futures = new CompletableFuture[size]; + this.pos = 0; + } + + public CompletableFuture add(Request request) + { + if (pos >= size) + { + throw new IndexOutOfBoundsException("pos: " + pos); + } + CompletableFuture future = new CompletableFuture<>(); + requests[pos] = request; + futures[pos] = future; + pos++; + return future; + } + + public int size() + { + return size; + } + + public Request[] getRequests() + { + return requests; + } + + public CompletableFuture[] getFutures() + { + return futures; + } + } +} diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/SchedulerFactory.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/SchedulerFactory.java new file mode 100644 index 000000000..ff2b5d4ce --- /dev/null +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/SchedulerFactory.java @@ -0,0 +1,63 @@ +/* + * Copyright 2021 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ +package io.pixelsdb.pixels.common.physical; + +import io.pixelsdb.pixels.common.physical.impl.scheduler.NoopScheduler; +import io.pixelsdb.pixels.common.utils.ConfigFactory; + +/** + * Created at: 9/10/21 + * Author: hank + */ +public class SchedulerFactory +{ + private static SchedulerFactory instance; + + public static SchedulerFactory Instance() + { + if (instance == null) + { + instance = new SchedulerFactory(); + } + return instance; + } + + private Scheduler scheduler; + + private SchedulerFactory() + { + String name = ConfigFactory.Instance().getProperty("read.request.scheduler"); + switch (name) + { + // Add more schedulers here. + case "noop": + scheduler = new NoopScheduler(); + break; + default: + throw new UnsupportedOperationException("The read request scheduler '" + + name + "' is unsupported."); + } + } + + public Scheduler getScheduler() + { + return scheduler; + } +} diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java new file mode 100644 index 000000000..572c36243 --- /dev/null +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java @@ -0,0 +1,83 @@ +/* + * Copyright 2021 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ +package io.pixelsdb.pixels.common.physical.impl.scheduler; + +import io.pixelsdb.pixels.common.physical.PhysicalReader; +import io.pixelsdb.pixels.common.physical.Scheduler; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.concurrent.CompletableFuture; + +/** + * Noop scheduler submits all the read requests in the batch one by one + * in a FIFO manner. + * If the reader supports asynchronous read, noop scheduler will do asynchronous read. + * Request sort and request merge are not implemented in noop. + * + * Created at: 9/10/21 + * Author: hank + */ +public class NoopScheduler implements Scheduler +{ + private static Logger logger = LogManager.getLogger(NoopScheduler.class); + + @Override + public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch batch) throws IOException + { + CompletableFuture[] futures = batch.getFutures(); + Request[] requests = batch.getRequests(); + if (reader.supportsAsync()) + { + for (int i = 0; i < batch.size(); ++i) + { + CompletableFuture future = futures[i]; + Request request = requests[i]; + reader.seek(request.start); + reader.readAsync(request.length).whenComplete((resp, err) -> + { + if (resp != null) + { + future.complete(resp); + } + else + { + logger.error("Failed to read asynchronously from path '" + + reader.getPath() + "'.", err); + err.printStackTrace(); + } + }); + } + } + else + { + for (int i = 0; i < batch.size(); ++i) + { + Request request = requests[i]; + reader.seek(request.start); + futures[i].complete(reader.readFully(request.length)); + } + } + + return CompletableFuture.allOf(futures); + } +} diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java new file mode 100644 index 000000000..a34091e0a --- /dev/null +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java @@ -0,0 +1,208 @@ +/* + * Copyright 2021 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ +package io.pixelsdb.pixels.common.physical.impl.scheduler; + +import io.pixelsdb.pixels.common.physical.PhysicalReader; +import io.pixelsdb.pixels.common.physical.Scheduler; +import io.pixelsdb.pixels.common.utils.ConfigFactory; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** + * SortMerge scheduler firstly sorts the requests in the batch by the start offset, + * then it tries to merge the requests that can be read sequentially from the reader. + * Created at: 9/12/21 + * Author: hank + */ +public class SortMergeScheduler implements Scheduler +{ + private static Logger logger = LogManager.getLogger(SortMergeScheduler.class); + private static int MaxGap; + + static + { + ConfigFactory.Instance().registerUpdateCallback("read.request.merge.gap", value -> + MaxGap = Integer.parseInt(ConfigFactory.Instance().getProperty("read.request.merge.gap"))); + MaxGap = Integer.parseInt(ConfigFactory.Instance().getProperty("read.request.merge.gap")); + } + + @Override + public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch batch) throws IOException + { + if (batch.size() <= 0) + { + throw new IllegalArgumentException("Request batch is empty."); + } + CompletableFuture[] futures = batch.getFutures(); + Request[] requests = batch.getRequests(); + List requestFutures = new ArrayList<>(batch.size()); + for (int i = 0; i < batch.size(); ++i) + { + requestFutures.add(new RequestFuture(requests[i], futures[i])); + } + Collections.sort(requestFutures); + List mergedRequests = new ArrayList<>(); + MergedRequest mr1 = new MergedRequest(requestFutures.get(0)); + MergedRequest mr2 = mr1; + for (int i = 1; i < batch.size(); ++i) + { + mr2 = mr1.merge(requestFutures.get(i)); + if (mr1 == mr2) + { + continue; + } + mergedRequests.add(mr1); + mr1 = mr2; + } + mergedRequests.add(mr2); + + if (reader.supportsAsync()) + { + for (MergedRequest merged : mergedRequests) + { + reader.seek(merged.getStart()); + reader.readAsync(merged.getLength()).whenComplete((resp, err) -> + { + if (resp != null) + { + merged.complete(resp); + } + else + { + logger.error("Failed to read asynchronously from path '" + + reader.getPath() + "'.", err); + err.printStackTrace(); + } + }); + } + } + else + { + for (MergedRequest merged : mergedRequests) + { + reader.seek(merged.getStart()); + ByteBuffer buffer = reader.readFully(merged.getLength()); + merged.complete(buffer); + } + } + + return CompletableFuture.allOf(futures); + } + + class RequestFuture implements Comparable + { + public Request request; + public CompletableFuture future; + + public RequestFuture(Request request, CompletableFuture future) + { + this.request = request; + this.future = future; + } + + @Override + public int compareTo(RequestFuture o) + { + return this.request.compareTo(o.request); + } + } + + class MergedRequest + { + private long start; + private long end; + private int position; + private int size; + private List positions; + private List lengths; + private List> futures; + + public MergedRequest(RequestFuture first) + { + this.start = first.request.start; + this.end = first.request.start + first.request.length; + this.positions = new ArrayList<>(); + this.lengths = new ArrayList<>(); + this.futures = new ArrayList<>(); + this.positions.add(0); + this.lengths.add(first.request.length); + this.position = first.request.length; + this.futures.add(first.future); + this.size = 1; + } + + public MergedRequest merge(RequestFuture curr) + { + if (curr.request.start < this.end) + { + throw new IllegalArgumentException("Can not merge backward request."); + } + int gap = (int) (curr.request.start - this.end); + if (gap <= MaxGap) + { + this.positions.add(this.position + gap); + this.lengths.add(curr.request.length); + this.position += (gap + curr.request.length); + this.end = curr.request.start + curr.request.length; + this.futures.add(curr.future); + this.size ++; + return this; + } + return new MergedRequest(curr); + } + + public long getStart() + { + return start; + } + + public int getLength() + { + return (int) (end - start); + } + + public int getSize() + { + return size; + } + + /** + * When the data has been read, complete all the + * futures, + * @param buffer the data that has been read. + */ + public void complete(ByteBuffer buffer) + { + for (int i = 0; i < size; ++i) + { + buffer.position(positions.get(i)); + buffer.limit(positions.get(i) + lengths.get(i)); + futures.get(i).complete(buffer.slice()); + } + } + } +} diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/ConfigFactory.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/ConfigFactory.java index 6b1c8c1e1..40457a769 100755 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/ConfigFactory.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/utils/ConfigFactory.java @@ -23,6 +23,8 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; +import java.util.HashMap; +import java.util.Map; import java.util.Properties; /** @@ -49,9 +51,24 @@ public static ConfigFactory Instance() // Properties is thread safe, so we do not put synchronization on it. private Properties prop = null; + /** + * Issue #114: + * Add configuration update callbacks. + * By registering update callback, other class can get their + * members updated when the properties on ConfigurationFactory + * are reloaded or updated. + */ + public interface UpdateCallback + { + void update(String value); + } + + private Map callbacks; + private ConfigFactory() { prop = new Properties(); + callbacks = new HashMap<>(); String pixelsHome = System.getenv("PIXELS_HOME"); InputStream in = null; if (pixelsHome == null) @@ -87,13 +104,26 @@ private ConfigFactory() } } - public void loadProperties(String propFilePath) throws IOException + public synchronized void registerUpdateCallback(String key, UpdateCallback callback) + { + this.callbacks.put(key, callback); + } + + public synchronized void loadProperties(String propFilePath) throws IOException { FileInputStream in = null; try { in = new FileInputStream(propFilePath); this.prop.load(in); + for (Map.Entry entry : this.callbacks.entrySet()) + { + String value = this.prop.getProperty(entry.getKey()); + if (value != null) + { + entry.getValue().update(value); + } + } } catch (IOException e) { @@ -115,12 +145,16 @@ public void loadProperties(String propFilePath) throws IOException } } - public void addProperty(String key, String value) + public synchronized void addProperty(String key, String value) { this.prop.setProperty(key, value); + if (this.callbacks.containsKey(key)) + { + this.callbacks.get(key).update(value); + } } - public String getProperty(String key) + public synchronized String getProperty(String key) { return this.prop.getProperty(key); } diff --git a/pixels-common/src/main/resources/pixels.properties b/pixels-common/src/main/resources/pixels.properties index 9f29997aa..f5baa6c71 100755 --- a/pixels-common/src/main/resources/pixels.properties +++ b/pixels-common/src/main/resources/pixels.properties @@ -57,6 +57,9 @@ etcd.port=2379 fixed.split.size=-1 # true to enable just-in-time splitting in ordered path. multi.split.for.ordered=true +# which scheduler to use for read requests. +read.request.scheduler=noop +read.request.merge.gap=2097152 # the rate of free memory in jvm. pixels.gc.threshold=0.3 From 3a666c5035afe1e718af405d02be9402c4a14db4 Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Sun, 12 Sep 2021 22:49:58 +0200 Subject: [PATCH 05/19] fix and optimize read scheduler. --- .../pixels/common/physical/Scheduler.java | 74 ++++++++++++++----- .../common/physical/SchedulerFactory.java | 4 + .../impl/scheduler/NoopScheduler.java | 15 ++-- .../impl/scheduler/SortMergeScheduler.java | 12 +-- .../src/main/resources/pixels.properties | 3 +- .../io/pixelsdb/pixels/common/TestS3.java | 8 +- .../core/reader/PixelsRecordReaderImpl.java | 24 ++++++ 7 files changed, 101 insertions(+), 39 deletions(-) diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Scheduler.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Scheduler.java index 4159da546..20daa52ed 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Scheduler.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Scheduler.java @@ -21,6 +21,8 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.CompletableFuture; /** @@ -32,6 +34,14 @@ */ public interface Scheduler { + /** + * Execute a batch of read requests, and return the future of the completion of + * all the requests. + * @param reader + * @param batch + * @return should never return null. + * @throws IOException + */ CompletableFuture executeBatch(PhysicalReader reader, RequestBatch batch) throws IOException; class Request implements Comparable @@ -72,33 +82,39 @@ public int compareTo(Request o) class RequestBatch { - private int pos; private int size; - private Request[] requests; - private CompletableFuture[] futures; + private List requests; + private List> futures; - @SuppressWarnings("unchecked") - public RequestBatch(int size) + public RequestBatch() { - if (size <= 0) + this.requests = new ArrayList<>(); + this.futures = new ArrayList<>(); + this.size = 0; + } + + public RequestBatch(int capacity) + { + if (capacity <= 0) { - throw new IllegalArgumentException("Request batch size: " + size); + throw new IllegalArgumentException("Request batch capacity: " + capacity); } - this.requests = new Request[size]; - this.futures = new CompletableFuture[size]; - this.pos = 0; + this.requests = new ArrayList<>(capacity); + this.futures = new ArrayList<>(capacity); + this.size = 0; + } + + public CompletableFuture add(long start, int length) + { + return add(new Request(start, length)); } public CompletableFuture add(Request request) { - if (pos >= size) - { - throw new IndexOutOfBoundsException("pos: " + pos); - } CompletableFuture future = new CompletableFuture<>(); - requests[pos] = request; - futures[pos] = future; - pos++; + requests.add(request); + futures.add(future); + size++; return future; } @@ -107,14 +123,34 @@ public int size() return size; } - public Request[] getRequests() + public List getRequests() { return requests; } - public CompletableFuture[] getFutures() + public List> getFutures() { return futures; } + + /** + * If batch is empty, this method returns and completed future. + * @return + */ + public CompletableFuture completeAll() + { + if (size <= 0) + { + CompletableFuture future = new CompletableFuture<>(); + future.complete(null); + return future; + } + CompletableFuture[] fs = new CompletableFuture[size]; + for (int i = 0; i < size; ++i) + { + fs[i] = this.futures.get(i); + } + return CompletableFuture.allOf(fs); + } } } diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/SchedulerFactory.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/SchedulerFactory.java index ff2b5d4ce..f9b2970da 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/SchedulerFactory.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/SchedulerFactory.java @@ -20,6 +20,7 @@ package io.pixelsdb.pixels.common.physical; import io.pixelsdb.pixels.common.physical.impl.scheduler.NoopScheduler; +import io.pixelsdb.pixels.common.physical.impl.scheduler.SortMergeScheduler; import io.pixelsdb.pixels.common.utils.ConfigFactory; /** @@ -50,6 +51,9 @@ private SchedulerFactory() case "noop": scheduler = new NoopScheduler(); break; + case "sortmerge": + scheduler = new SortMergeScheduler(); + break; default: throw new UnsupportedOperationException("The read request scheduler '" + name + "' is unsupported."); diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java index 572c36243..f62dd9a01 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java @@ -26,6 +26,7 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.List; import java.util.concurrent.CompletableFuture; /** @@ -44,14 +45,14 @@ public class NoopScheduler implements Scheduler @Override public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch batch) throws IOException { - CompletableFuture[] futures = batch.getFutures(); - Request[] requests = batch.getRequests(); + List> futures = batch.getFutures(); + List requests = batch.getRequests(); if (reader.supportsAsync()) { for (int i = 0; i < batch.size(); ++i) { - CompletableFuture future = futures[i]; - Request request = requests[i]; + CompletableFuture future = futures.get(i); + Request request = requests.get(i); reader.seek(request.start); reader.readAsync(request.length).whenComplete((resp, err) -> { @@ -72,12 +73,12 @@ public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch { for (int i = 0; i < batch.size(); ++i) { - Request request = requests[i]; + Request request = requests.get(i); reader.seek(request.start); - futures[i].complete(reader.readFully(request.length)); + futures.get(i).complete(reader.readFully(request.length)); } } - return CompletableFuture.allOf(futures); + return batch.completeAll(); } } diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java index a34091e0a..b83416952 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java @@ -53,16 +53,12 @@ public class SortMergeScheduler implements Scheduler @Override public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch batch) throws IOException { - if (batch.size() <= 0) - { - throw new IllegalArgumentException("Request batch is empty."); - } - CompletableFuture[] futures = batch.getFutures(); - Request[] requests = batch.getRequests(); + List> futures = batch.getFutures(); + List requests = batch.getRequests(); List requestFutures = new ArrayList<>(batch.size()); for (int i = 0; i < batch.size(); ++i) { - requestFutures.add(new RequestFuture(requests[i], futures[i])); + requestFutures.add(new RequestFuture(requests.get(i), futures.get(i))); } Collections.sort(requestFutures); List mergedRequests = new ArrayList<>(); @@ -110,7 +106,7 @@ public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch } } - return CompletableFuture.allOf(futures); + return batch.completeAll(); } class RequestFuture implements Comparable diff --git a/pixels-common/src/main/resources/pixels.properties b/pixels-common/src/main/resources/pixels.properties index f5baa6c71..d45fd6a59 100755 --- a/pixels-common/src/main/resources/pixels.properties +++ b/pixels-common/src/main/resources/pixels.properties @@ -58,7 +58,8 @@ fixed.split.size=-1 # true to enable just-in-time splitting in ordered path. multi.split.for.ordered=true # which scheduler to use for read requests. -read.request.scheduler=noop +# valid values: noop, sortmerge +read.request.scheduler=sortmerge read.request.merge.gap=2097152 # the rate of free memory in jvm. diff --git a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java index 4977ada1e..539e16e95 100644 --- a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java +++ b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java @@ -16,9 +16,9 @@ public class TestS3 @Test public void testS3Writer() throws IOException { - PhysicalWriter writer = PhysicalWriterUtil.newPhysicalWriter(Storage.Scheme.s3, "pixels-01/object-3", + PhysicalWriter writer = PhysicalWriterUtil.newPhysicalWriter(Storage.Scheme.s3, "pixels-01/object-4", 0, (short) 1, false); - ByteBuffer buffer = ByteBuffer.allocate(1024); + ByteBuffer buffer = ByteBuffer.allocate(10240); buffer.putLong(1); writer.append(buffer); writer.flush(); @@ -28,8 +28,8 @@ public void testS3Writer() throws IOException @Test public void testS3Reader() throws IOException { - PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(Storage.Scheme.s3, "pixels-01/object-0"); - CompletableFuture future = reader.readAsync(1024); + PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(Storage.Scheme.s3, "pixels-01/object-4"); + CompletableFuture future = reader.readAsync(8); future.whenComplete((resp, err) -> { if (resp != null) diff --git a/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java b/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java index cc9ffb2c8..ad3f6db92 100644 --- a/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java +++ b/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java @@ -19,10 +19,13 @@ */ package io.pixelsdb.pixels.core.reader; +import com.google.protobuf.InvalidProtocolBufferException; import io.pixelsdb.pixels.cache.ColumnletId; import io.pixelsdb.pixels.cache.PixelsCacheReader; import io.pixelsdb.pixels.common.metrics.ReadPerfMetrics; import io.pixelsdb.pixels.common.physical.PhysicalReader; +import io.pixelsdb.pixels.common.physical.Scheduler; +import io.pixelsdb.pixels.common.physical.SchedulerFactory; import io.pixelsdb.pixels.common.utils.ConfigFactory; import io.pixelsdb.pixels.core.*; import io.pixelsdb.pixels.core.predicate.PixelsPredicate; @@ -396,6 +399,11 @@ else if (predicate.matchesNone()) // read row group footers rowGroupFooters = new PixelsProto.RowGroupFooter[targetRGNum]; + /** + * Issue #114: + * Use request batch and read scheduler to execute the read requests. + */ + Scheduler.RequestBatch requestBatch = new Scheduler.RequestBatch(targetRGNum); for (int i = 0; i < targetRGNum; i++) { int rgId = targetRGs[i]; @@ -408,6 +416,7 @@ else if (predicate.matchesNone()) footer.getRowGroupInfos(rgId); long footerOffset = rowGroupInformation.getFooterOffset(); long footerLength = rowGroupInformation.getFooterLength(); + /* byte[] footerBuffer = new byte[(int) footerLength]; try { @@ -422,6 +431,19 @@ else if (predicate.matchesNone()) logger.error("failed to read file footer.", e); throw new IOException("failed to read file footer.", e); } + */ + int fi = i; + requestBatch.add(footerOffset, (int) footerLength).whenComplete((resp, err) -> + { + try + { + rowGroupFooters[fi] = PixelsProto.RowGroupFooter.parseFrom(resp); + pixelsFooterCache.putRGFooter(rgCacheId, rowGroupFooters[fi]); + } catch (InvalidProtocolBufferException e) + { + logger.error("failed to read file footer.", e); + } + }); } // cache hit else @@ -429,6 +451,8 @@ else if (predicate.matchesNone()) rowGroupFooters[i] = rowGroupFooter; } } + Scheduler scheduler = SchedulerFactory.Instance().getScheduler(); + scheduler.executeBatch(physicalReader, requestBatch).join(); return true; } From 7358f144d355830786e1c4814fc395a9f8ae9996 Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Mon, 13 Sep 2021 00:00:33 +0200 Subject: [PATCH 06/19] use read scheduler in record reader. --- .../core/reader/PixelsRecordReaderImpl.java | 172 ++++++++---------- 1 file changed, 73 insertions(+), 99 deletions(-) diff --git a/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java b/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java index ad3f6db92..06deb5a63 100644 --- a/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java +++ b/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java @@ -39,6 +39,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.*; +import java.util.concurrent.atomic.AtomicReference; /** * @author guodong @@ -404,6 +405,7 @@ else if (predicate.matchesNone()) * Use request batch and read scheduler to execute the read requests. */ Scheduler.RequestBatch requestBatch = new Scheduler.RequestBatch(targetRGNum); + AtomicReference lastErr = new AtomicReference<>(null); for (int i = 0; i < targetRGNum; i++) { int rgId = targetRGs[i]; @@ -416,32 +418,25 @@ else if (predicate.matchesNone()) footer.getRowGroupInfos(rgId); long footerOffset = rowGroupInformation.getFooterOffset(); long footerLength = rowGroupInformation.getFooterLength(); - /* - byte[] footerBuffer = new byte[(int) footerLength]; - try - { - physicalReader.seek(footerOffset); - physicalReader.readFully(footerBuffer); - rowGroupFooters[i] = - PixelsProto.RowGroupFooter.parseFrom(footerBuffer); - pixelsFooterCache.putRGFooter(rgCacheId, rowGroupFooters[i]); - } - catch (IOException e) - { - logger.error("failed to read file footer.", e); - throw new IOException("failed to read file footer.", e); - } - */ int fi = i; requestBatch.add(footerOffset, (int) footerLength).whenComplete((resp, err) -> { - try + if (resp != null) { - rowGroupFooters[fi] = PixelsProto.RowGroupFooter.parseFrom(resp); - pixelsFooterCache.putRGFooter(rgCacheId, rowGroupFooters[fi]); - } catch (InvalidProtocolBufferException e) + try + { + rowGroupFooters[fi] = PixelsProto.RowGroupFooter.parseFrom(resp); + pixelsFooterCache.putRGFooter(rgCacheId, rowGroupFooters[fi]); + } catch (InvalidProtocolBufferException e) + { + logger.error("Failed to parse row group footer from byte buffer.", e); + lastErr.set(e); + } + } + else { - logger.error("failed to read file footer.", e); + logger.error("Failed to read row group footer.", err); + lastErr.set(err); } }); } @@ -451,6 +446,11 @@ else if (predicate.matchesNone()) rowGroupFooters[i] = rowGroupFooter; } } + if (lastErr.get() != null) + { + throw new IOException("Failed to read row group footers, " + + "only the last error is thrown, check the logs for more information.", lastErr.get()); + } Scheduler scheduler = SchedulerFactory.Instance().getScheduler(); scheduler.executeBatch(physicalReader, requestBatch).join(); return true; @@ -611,7 +611,7 @@ private boolean read() throws IOException long cacheReadCost = cacheReadEndNano - cacheReadStartNano; /* // We used deal with null or empty cache chunk here to get more accurate cacheReadCost. - // In Issue #67 (patch), we move the logic into the above loop for better performance. + // In Issue #67 (patch), we moved this logic into the above loop for better performance. for (ColumnletId chunkId : cacheChunks) { short rgId = chunkId.rowGroupId; @@ -654,7 +654,7 @@ private boolean read() throws IOException } } - if (diskChunks.isEmpty() == false) + if (!diskChunks.isEmpty()) { /** * Comments added in Issue #67 (path): @@ -665,91 +665,65 @@ private boolean read() throws IOException * similar strategy is called disk range merging. * * diskChunks.sort and ChunkSeq are also optimized in Issue #67 (path). + * + * Issue #114: + * disk chunks ordering and request merging are moved into the sortmerge + * scheduler, which can be enabled by setting read.request.scheduler=sortmerge. */ - - // sort chunks by starting offset - diskChunks.sort((o1, o2) -> ( - o1.offset < o2.offset ? -1 : - (o1.offset > o2.offset ? 1 : 0))); - - // get chunk blocks. In best case, diskChunkSeqs.size() == targetRGNum. - List diskChunkSeqs = new ArrayList<>(this.targetRGNum); - ChunkSeq diskChunkSeq = new ChunkSeq(); + Scheduler.RequestBatch batch = new Scheduler.RequestBatch(diskChunks.size()); + AtomicReference lastErr = new AtomicReference<>(null); for (ChunkId chunk : diskChunks) { - if (!diskChunkSeq.addChunk(chunk)) - { - diskChunkSeqs.add(diskChunkSeq); - diskChunkSeq = new ChunkSeq(); - diskChunkSeq.addChunk(chunk); - } - } - diskChunkSeqs.add(diskChunkSeq); - - // read chunk blocks into buffers - try - { - for (ChunkSeq seq : diskChunkSeqs) + /** + * Comments added in Issue #103: + * chunk.rowGroupId does not mean the row group id in the Pixels file, + * it is the index of group that is to be read (some row groups in the file + * may be filtered out by the predicate and will not be read) and it is + * used to calculate the index of chunkBuffers. + */ + int rgIdx = chunk.rowGroupId; + int colId = chunk.columnId; + /** + * Issue #114: + * The old code segment of chunk reading is remove in this issue. + * Now, if enableMetrics == true, we can add the read performance metrics here. + * + * Examples of how to add performance metrics: + * + * BytesMsCost seekCost = new BytesMsCost(); + * seekCost.setBytes(seekDistanceInBytes); + * seekCost.setMs(seekTimeMs); + * readPerfMetrics.addSeek(seekCost); + * + * BytesMsCost readCost = new BytesMsCost(); + * readCost.setBytes(bytesRead); + * readCost.setMs(readTimeMs); + * readPerfMetrics.addSeqRead(readCost); + */ + batch.add(new Scheduler.Request(chunk.offset, (int)chunk.length)) + .whenComplete((resp, err) -> { - if (seq.getLength() == 0) + if (resp != null) { - continue; + chunkBuffers[rgIdx * includedColumns.length + colId] = resp; } - int offset = (int) seq.getOffset(); - int length = (int) seq.getLength(); - diskReadBytes += length; - ByteBuffer chunkBlockBuffer = ByteBuffer.allocate(length); - memoryUsage += length; -// if (enableMetrics) -// { -// long seekStart = System.currentTimeMillis(); -// physicalFSReader.seek(offset); -// long seekEnd = System.currentTimeMillis(); -// BytesMsCost seekCost = new BytesMsCost(); -// seekCost.setBytes(Math.abs(offsetBeforeSeek - offset)); -// seekCost.setMs(seekEnd - seekStart); -// readPerfMetrics.addSeek(seekCost); -// offsetBeforeSeek = offset; -// -// long readStart = System.currentTimeMillis(); -// physicalFSReader.readFully(chunkBlockBuffer); -// long readEnd = System.currentTimeMillis(); -// BytesMsCost readCost = new BytesMsCost(); -// readCost.setBytes(length); -// readCost.setMs(readEnd - readStart); -// readPerfMetrics.addSeqRead(readCost); -// } -// else -// { - physicalReader.seek(offset); - physicalReader.readFully(chunkBlockBuffer.array()); -// } - List chunkIds = seq.getChunks(); - int chunkSliceOffset = 0; - for (ChunkId chunkId : chunkIds) + else { - int chunkLength = (int) chunkId.length; - /** - * Comments added in Issue #103: - * chunkId.rowGroupId does not mean the row group id in the Pixels file, - * it is the index of group that is to be read (some row groups in the file - * may be filtered out by the predicate and will not be read) and it is - * used to calculate the index of chunkBuffers. - */ - int rgIdx = chunkId.rowGroupId; - int colId = chunkId.columnId; - chunkBlockBuffer.position(chunkSliceOffset); - chunkBlockBuffer.limit(chunkSliceOffset + chunkLength); - ByteBuffer chunkBuffer = chunkBlockBuffer.slice(); - chunkBuffers[rgIdx * includedColumns.length + colId] = chunkBuffer; - chunkSliceOffset += chunkLength; + logger.error("Failed to read chunks block into buffers.", err); + lastErr.set(err); } - } - } catch (IOException e) + }); + // don't update statistics in whenComplete as it may be executed in other threads. + diskReadBytes += chunk.length; + memoryUsage += chunk.length; + } + + Scheduler scheduler = SchedulerFactory.Instance().getScheduler(); + scheduler.executeBatch(physicalReader, batch).join(); + if (lastErr.get() != null) { - logger.error("failed to read chunks block into buffers.", e); - throw new IOException("failed to read chunks block into buffers.", e); - // return false; + throw new IOException("Failed to read chunks block into buffers, " + + "only the last error is thrown, check the logs for more information.", lastErr.get()); } } From dd3671e5228d61078e5116fd20fd322b0da66323 Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Mon, 13 Sep 2021 00:59:35 +0200 Subject: [PATCH 07/19] clean code, add dependencies, fix bugs. --- .../impl/scheduler/NoopScheduler.java | 4 + .../impl/scheduler/SortMergeScheduler.java | 4 + .../io/pixelsdb/pixels/common/TestS3.java | 19 +++ .../java/io/pixelsdb/pixels/core/ChunkId.java | 41 ------- .../io/pixelsdb/pixels/core/ChunkSeq.java | 112 ------------------ .../core/reader/PixelsRecordReaderImpl.java | 21 +++- pixels-daemon/pom.xml | 7 ++ pixels-load/pom.xml | 7 ++ pixels-presto/pom.xml | 7 ++ 9 files changed, 67 insertions(+), 155 deletions(-) delete mode 100644 pixels-core/src/main/java/io/pixelsdb/pixels/core/ChunkId.java delete mode 100644 pixels-core/src/main/java/io/pixelsdb/pixels/core/ChunkSeq.java diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java index f62dd9a01..33a369131 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java @@ -45,6 +45,10 @@ public class NoopScheduler implements Scheduler @Override public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch batch) throws IOException { + if (batch.size() <= 0) + { + return batch.completeAll(); + } List> futures = batch.getFutures(); List requests = batch.getRequests(); if (reader.supportsAsync()) diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java index b83416952..e9a83f07e 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java @@ -53,6 +53,10 @@ public class SortMergeScheduler implements Scheduler @Override public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch batch) throws IOException { + if (batch.size() <= 0) + { + return batch.completeAll(); + } List> futures = batch.getFutures(); List requests = batch.getRequests(); List requestFutures = new ArrayList<>(batch.size()); diff --git a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java index 539e16e95..9f1e92bb5 100644 --- a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java +++ b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java @@ -1,3 +1,22 @@ +/* + * Copyright 2021 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ package io.pixelsdb.pixels.common; import io.pixelsdb.pixels.common.physical.*; diff --git a/pixels-core/src/main/java/io/pixelsdb/pixels/core/ChunkId.java b/pixels-core/src/main/java/io/pixelsdb/pixels/core/ChunkId.java deleted file mode 100644 index 002fc59f6..000000000 --- a/pixels-core/src/main/java/io/pixelsdb/pixels/core/ChunkId.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Copyright 2017-2019 PixelsDB. - * - * This file is part of Pixels. - * - * Pixels is free software: you can redistribute it and/or modify - * it under the terms of the Affero GNU General Public License as - * published by the Free Software Foundation, either version 3 of - * the License, or (at your option) any later version. - * - * Pixels is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * Affero GNU General Public License for more details. - * - * You should have received a copy of the Affero GNU General Public - * License along with Pixels. If not, see - * . - */ -package io.pixelsdb.pixels.core; - -/** - * pixels - * - * @author guodong - */ -public class ChunkId -{ - public final int rowGroupId; - public final int columnId; - public final long offset; - public final long length; - - public ChunkId(int rowGroupId, int columnId, long offset, long length) - { - this.rowGroupId = rowGroupId; - this.columnId = columnId; - this.offset = offset; - this.length = length; - } -} diff --git a/pixels-core/src/main/java/io/pixelsdb/pixels/core/ChunkSeq.java b/pixels-core/src/main/java/io/pixelsdb/pixels/core/ChunkSeq.java deleted file mode 100644 index 9bca9ee86..000000000 --- a/pixels-core/src/main/java/io/pixelsdb/pixels/core/ChunkSeq.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Copyright 2017-2019 PixelsDB. - * - * This file is part of Pixels. - * - * Pixels is free software: you can redistribute it and/or modify - * it under the terms of the Affero GNU General Public License as - * published by the Free Software Foundation, either version 3 of - * the License, or (at your option) any later version. - * - * Pixels is distributed in the hope that it will be useful, - * but WITHOUT ANY WARRANTY; without even the implied warranty of - * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the - * Affero GNU General Public License for more details. - * - * You should have received a copy of the Affero GNU General Public - * License along with Pixels. If not, see - * . - */ -package io.pixelsdb.pixels.core; - -import java.util.ArrayList; -import java.util.List; - -/** - * pixels chunk block. - * Each chunk block is sequential in file, and supposed to be read sequentially on disk. - * - * @author guodong - */ -public class ChunkSeq -{ - private final List chunks; - private long offset = 0; - private long length = 0; - - public ChunkSeq() - { - chunks = new ArrayList<>(); - } - - public ChunkSeq(List sortedChunks, - long offset, long length) - { - this.chunks = sortedChunks; - this.offset = offset; - this.length = length; - } - - /** - * Add chunk by the order of chunks' offset - */ - public boolean addChunk(ChunkId chunk) - { - if (length == 0) - { - chunks.add(chunk); - offset = chunk.offset; - length += chunk.length; - return true; - } - else - { - if (chunk.offset - offset - length == 0) - { - chunks.add(chunk); - length += chunk.length; - return true; - } - } - return false; - } - - public void setOffset(long offset) - { - this.offset = offset; - } - - public void setLength(long length) - { - this.length = length; - } - - public List getChunks() - { - return chunks; - } - - /** - *

Legacy code.

- * This method should be the same as getChunks(), - * because chunks are ordered before being added into ChunkSeq. - * @return - */ - public List getSortedChunks() - { - chunks.sort((o1, o2) -> ( - o1.offset < o2.offset ? -1 : - (o1.offset > o2.offset ? 1 : 0))); - return chunks; - } - - public long getOffset() - { - return offset; - } - - public long getLength() - { - return length; - } -} diff --git a/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java b/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java index 06deb5a63..20c34dd0c 100644 --- a/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java +++ b/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java @@ -667,8 +667,9 @@ private boolean read() throws IOException * diskChunks.sort and ChunkSeq are also optimized in Issue #67 (path). * * Issue #114: - * disk chunks ordering and request merging are moved into the sortmerge - * scheduler, which can be enabled by setting read.request.scheduler=sortmerge. + * Disk chunks ordering and request merging (was implemented in the removed + * ChunkSeq) are moved into the sortmerge scheduler, which can be enabled + * by setting read.request.scheduler=sortmerge. */ Scheduler.RequestBatch batch = new Scheduler.RequestBatch(diskChunks.size()); AtomicReference lastErr = new AtomicReference<>(null); @@ -1099,4 +1100,20 @@ public void close() throws IOException // reset read performance metrics // readPerfMetrics.clear(); } + + public class ChunkId + { + public final int rowGroupId; + public final int columnId; + public final long offset; + public final long length; + + public ChunkId(int rowGroupId, int columnId, long offset, long length) + { + this.rowGroupId = rowGroupId; + this.columnId = columnId; + this.offset = offset; + this.length = length; + } + } } diff --git a/pixels-daemon/pom.xml b/pixels-daemon/pom.xml index 5106c1867..e057b9ce4 100644 --- a/pixels-daemon/pom.xml +++ b/pixels-daemon/pom.xml @@ -79,6 +79,13 @@ -->
+ + + software.amazon.awssdk + s3 + true + + org.apache.hadoop diff --git a/pixels-load/pom.xml b/pixels-load/pom.xml index 5f33f9da1..294a74fb4 100644 --- a/pixels-load/pom.xml +++ b/pixels-load/pom.xml @@ -39,6 +39,13 @@ true + + + software.amazon.awssdk + s3 + true + + org.apache.hadoop diff --git a/pixels-presto/pom.xml b/pixels-presto/pom.xml index ac63581b5..a729d1bdd 100644 --- a/pixels-presto/pom.xml +++ b/pixels-presto/pom.xml @@ -32,6 +32,13 @@ true + + + software.amazon.awssdk + s3 + true + + org.apache.hadoop From 683ba552ae6f8a3f2d0b53460bb2a35659abaeaf Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Mon, 13 Sep 2021 03:11:13 +0200 Subject: [PATCH 08/19] fix MergedRequest and PixelsPageSource. --- .../common/physical/impl/scheduler/SortMergeScheduler.java | 6 +++++- .../java/io/pixelsdb/pixels/presto/PixelsPageSource.java | 7 +++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java index e9a83f07e..79cd68bb3 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java @@ -199,8 +199,12 @@ public void complete(ByteBuffer buffer) { for (int i = 0; i < size; ++i) { - buffer.position(positions.get(i)); + /** + * Issue #114: + * Limit should be set before position. + */ buffer.limit(positions.get(i) + lengths.get(i)); + buffer.position(positions.get(i)); futures.get(i).complete(buffer.slice()); } } diff --git a/pixels-presto/src/main/java/io/pixelsdb/pixels/presto/PixelsPageSource.java b/pixels-presto/src/main/java/io/pixelsdb/pixels/presto/PixelsPageSource.java index ab3596467..9c5505bb6 100644 --- a/pixels-presto/src/main/java/io/pixelsdb/pixels/presto/PixelsPageSource.java +++ b/pixels-presto/src/main/java/io/pixelsdb/pixels/presto/PixelsPageSource.java @@ -316,6 +316,13 @@ public void close() this.memoryUsage += recordReader.getMemoryUsage(); } pixelsReader.close(); + /** + * Issue #114: + * Must set pixelsReader and recordReader to null, + * close() may be called multiple times by Presto. + */ + recordReader = null; + pixelsReader = null; } rowBatch = null; } catch (Exception e) From 9c5f5ca651615494d1eeff1cac6ef994b373cdd8 Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Mon, 13 Sep 2021 10:20:20 +0200 Subject: [PATCH 09/19] refine. --- .../io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java b/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java index 20c34dd0c..82eb63f6d 100644 --- a/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java +++ b/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java @@ -968,6 +968,7 @@ public VectorizedRowBatch readBatch() * Before that, it will always return false. * @return true if reach EOF. */ + @Override public boolean isEndOfFile () { if (this.resultRowBatch != null) From a2aa995161c8431052fe938211013107a5317d1b Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Fri, 24 Sep 2021 03:26:11 +0200 Subject: [PATCH 10/19] implement S3OutputStream, finish S3, add S3 copy to pixels.load. --- .../pixels/common/physical/Storage.java | 24 ++ .../common/physical/StorageFactory.java | 19 +- .../pixels/common/physical/impl/HDFS.java | 1 + .../pixels/common/physical/impl/LocalFS.java | 6 + .../pixels/common/physical/impl/S3.java | 67 ++++- .../common/physical/impl/S3OutputStream.java | 258 ++++++++++++++++++ .../io/pixelsdb/pixels/common/TestS3.java | 6 + .../core/reader/PixelsRecordReaderImpl.java | 2 +- .../io/pixelsdb/pixels/load/multi/Main.java | 12 +- 9 files changed, 378 insertions(+), 17 deletions(-) create mode 100644 pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3OutputStream.java diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Storage.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Storage.java index 962a31d51..59c1bcdbd 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Storage.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Storage.java @@ -49,6 +49,16 @@ public static Scheme from(String value) return valueOf(value.toLowerCase()); } + /** + * Parse the scheme from the path which is prefixed with the storage scheme. + * @param schemedPath + */ + public static Scheme fromPath(String schemedPath) + { + String scheme = schemedPath.substring(0, schemedPath.indexOf("://")); + return Scheme.from(scheme); + } + /** * Whether the value is a valid storage scheme. * @param value @@ -141,6 +151,20 @@ public boolean equals(Scheme other) DataOutputStream create(String path, boolean overwrite, int bufferSize, short replication) throws IOException; + /** + * This method is for the compatability of block-based storage like HDFS. + * For local fs, path is considered as local. + * @param path + * @param overwrite + * @param bufferSize + * @param replication + * @param blockSize + * @return + * @throws IOException if path is a directory. + */ + DataOutputStream create(String path, boolean overwrite, + int bufferSize, short replication, long blockSize) throws IOException; + /** * For local fs, path is considered as local. * @param path diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/StorageFactory.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/StorageFactory.java index 3544f09b1..5c880fc15 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/StorageFactory.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/StorageFactory.java @@ -59,16 +59,29 @@ public synchronized void reload() throws IOException storageImpls.put(Storage.Scheme.s3, new S3()); } - public synchronized Storage getStorage(String scheme) throws IOException + /** + * Get the storage instance from a scheme name or a scheme prefixed path. + * @param schemeOrPath + * @return + * @throws IOException + */ + public synchronized Storage getStorage(String schemeOrPath) throws IOException { try { // 'synchronized' in Java is reentrant, it is fine the call the other getStorage(). - return getStorage(Storage.Scheme.from(scheme)); + if (schemeOrPath.contains("://")) + { + return getStorage(Storage.Scheme.fromPath(schemeOrPath)); + } + else + { + return getStorage(Storage.Scheme.from(schemeOrPath)); + } } catch (RuntimeException re) { - throw new IOException("Invalid storage scheme: " + scheme, re); + throw new IOException("Invalid storage scheme or path: " + schemeOrPath, re); } } diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/HDFS.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/HDFS.java index 13c7f25fc..2405fb53b 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/HDFS.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/HDFS.java @@ -265,6 +265,7 @@ public DataOutputStream create(String path, boolean overwrite, int bufferSize, s return fs.create(fsPath, overwrite, bufferSize, replication, fs.getDefaultBlockSize(fsPath)); } + @Override public DataOutputStream create(String path, boolean overwrite, int bufferSize, short replication, long blockSize) throws IOException { return fs.create(new Path(path), overwrite, bufferSize, replication, blockSize); diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/LocalFS.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/LocalFS.java index e07930eba..3559d2b99 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/LocalFS.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/LocalFS.java @@ -224,6 +224,12 @@ public DataOutputStream create(String path, boolean overwrite, int bufferSize, s return new DataOutputStream(new BufferedOutputStream(new FileOutputStream(file), bufferSize)); } + @Override + public DataOutputStream create(String path, boolean overwrite, int bufferSize, short replication, long blockSize) throws IOException + { + return this.create(path, overwrite, bufferSize, replication); + } + @Override public boolean delete(String path, boolean recursive) throws IOException { diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java index fa562fc59..210ad4399 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java @@ -37,7 +37,11 @@ import java.io.DataOutputStream; import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; import static io.pixelsdb.pixels.common.lock.EtcdAutoIncrement.GenerateId; import static io.pixelsdb.pixels.common.lock.EtcdAutoIncrement.InitId; @@ -93,7 +97,7 @@ private String getPathFrom(String key) { if (key.startsWith(S3_META_PREFIX)) { - key.substring(S3_META_PREFIX.length()); + return key.substring(S3_META_PREFIX.length()); } return null; } @@ -108,6 +112,10 @@ public static class Path public Path(String path) { requireNonNull(path); + if (path.contains("://")) + { + path = path.substring(path.indexOf("://") + 3); + } int slash = path.indexOf("/"); if (slash > 0) { @@ -123,6 +131,20 @@ public Path(String path) this.valid = true; } } + + @Override + public String toString() + { + if (!this.valid) + { + return null; + } + if (this.isBucket) + { + return this.bucket; + } + return this.bucket + "/" + this.key; + } } @Override @@ -132,15 +154,42 @@ public Scheme getScheme() } @Override - public List listStatus(String path) + public List listStatus(String path) throws IOException { - return null; + Path p = new Path(path); + if (!p.valid) + { + throw new IOException("Path '" + path + "' is not valid."); + } + if (!this.exists(path)) + { + throw new IOException("Path '" + path + "' does not exist."); + } + ListObjectsV2Request request = ListObjectsV2Request.builder() + .bucket(p.bucket).build(); + CompletableFuture response = s3.listObjectsV2(request); + try + { + List objects = response.get().contents(); + List statuses = new ArrayList<>(); + Path op = new Path(path); + for (S3Object object : objects) + { + op.key = object.key(); + statuses.add(new Status(op.toString(), object.size(), false, 1)); + } + return statuses; + } catch (InterruptedException | ExecutionException e) + { + throw new IOException("Failed to list objects.", e); + } } @Override public List listPaths(String path) throws IOException { - return null; + return this.listStatus(path).stream().map(Status::getPath) + .collect(Collectors.toList()); } /** @@ -246,7 +295,7 @@ public DataInputStream open(String path) throws IOException * @param overwrite * @param bufferSize * @param replication - * @return always return null. + * @return * @throws IOException */ @Override @@ -263,7 +312,13 @@ public DataOutputStream create(String path, boolean overwrite, int bufferSize, s } long id = GenerateId(S3_ID_KEY); EtcdUtil.Instance().putKeyValue(getPathKey(path), Long.toString(id)); - return null; + return new DataOutputStream(new S3OutputStream(s3, p.bucket, p.key)); + } + + @Override + public DataOutputStream create(String path, boolean overwrite, int bufferSize, short replication, long blockSize) throws IOException + { + return this.create(path, overwrite, bufferSize, replication); } @Override diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3OutputStream.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3OutputStream.java new file mode 100644 index 000000000..da181ba88 --- /dev/null +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3OutputStream.java @@ -0,0 +1,258 @@ +/* + * Copyright 2021 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ +package io.pixelsdb.pixels.common.physical.impl; + +import software.amazon.awssdk.core.async.AsyncRequestBody; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.model.*; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +/** + *

+ * Referenced the implementation from + * Barry Lagerweij. + *

+ * Created at: 9/24/21 + * Author: hank + */ +public class S3OutputStream extends OutputStream +{ + + /** + * Default chunk size is 10MB + */ + protected static final int BUFFER_SIZE = 10 * 1024 * 1024; + + /** + * The bucket-name on Amazon S3 + */ + private final String bucket; + + /** + * The path (key) name within the bucket + */ + private final String key; + + /** + * The temporary buffer used for storing the chunks + */ + private final byte[] buf; + + /** + * The position in the buffer + */ + private int position; + + /** + * Amazon S3 client. + */ + private final S3AsyncClient s3Client; + + /** + * The unique id for this upload + */ + private String uploadId; + + /** + * Collection of the etags for the parts that have been uploaded + */ + private final List parts; + + /** + * indicates whether the stream is still open / valid + */ + private boolean open; + + /** + * Creates a new S3 OutputStream + * + * @param s3Client the AmazonS3 client + * @param bucket name of the bucket + * @param key path (key) within the bucket + */ + public S3OutputStream(S3AsyncClient s3Client, String bucket, String key) + { + this.s3Client = s3Client; + this.bucket = bucket; + this.key = key; + this.buf = new byte[BUFFER_SIZE]; + this.position = 0; + this.parts = new ArrayList<>(); + this.open = true; + } + + /** + * Write an array to the S3 output stream. + * + * @param b the byte-array to append + */ + @Override + public void write(byte[] b) throws IOException + { + write(b, 0, b.length); + } + + /** + * Writes an array to the S3 Output Stream + * + * @param byteArray the array to write + * @param o the offset into the array + * @param l the number of bytes to write + */ + @Override + public void write(final byte[] byteArray, final int o, final int l) throws IOException + { + this.assertOpen(); + int ofs = o, len = l; + int size; + while (len > (size = this.buf.length - position)) + { + System.arraycopy(byteArray, ofs, this.buf, this.position, size); + this.position += size; + flushBufferAndRewind(); + ofs += size; + len -= size; + } + System.arraycopy(byteArray, ofs, this.buf, this.position, len); + this.position += len; + } + + /** + * Flushes the buffer by uploading a part to S3. + */ + @Override + public synchronized void flush() + { + this.assertOpen(); + } + + protected void flushBufferAndRewind() throws IOException + { + try + { + if (uploadId == null) + { + final CreateMultipartUploadRequest request = CreateMultipartUploadRequest.builder().bucket(this.bucket) + .key(this.key).acl(ObjectCannedACL.BUCKET_OWNER_FULL_CONTROL).build(); + CompletableFuture response = s3Client.createMultipartUpload(request); + this.uploadId = response.get().uploadId(); + } + } catch (ExecutionException | InterruptedException e) + { + throw new IOException("Failed to initiate multipart upload.", e); + } + uploadPart(); + this.position = 0; + } + + protected void uploadPart() throws IOException + { + UploadPartRequest request = UploadPartRequest.builder() + .bucket(this.bucket) + .key(this.key) + .uploadId(this.uploadId) + .partNumber(this.parts.size() + 1).build(); + CompletableFuture response = + this.s3Client.uploadPart(request, AsyncRequestBody.fromByteBuffer( + ByteBuffer.wrap(buf, 0, position))); + try + { + String etag = response.get().eTag(); + CompletedPart part = CompletedPart.builder().partNumber(this.parts.size() + 1).eTag(etag).build(); + this.parts.add(part); + } catch (InterruptedException | ExecutionException e) + { + throw new IOException("Failed to upload part.", e); + } + } + + @Override + public void close() throws IOException + { + if (this.open) + { + this.open = false; + if (this.uploadId != null) + { + if (this.position > 0) + { + uploadPart(); + } + CompletedMultipartUpload completedMultipartUpload = CompletedMultipartUpload.builder() + .parts(this.parts) + .build(); + CompleteMultipartUploadRequest completeMultipartUploadRequest = + CompleteMultipartUploadRequest.builder() + .bucket(this.bucket) + .key(this.key) + .uploadId(uploadId) + .multipartUpload(completedMultipartUpload) + .build(); + this.s3Client.completeMultipartUpload(completeMultipartUploadRequest).join(); + } else + { + final PutObjectRequest request = PutObjectRequest.builder().bucket(this.bucket).key(this.key) + .acl(ObjectCannedACL.BUCKET_OWNER_FULL_CONTROL).build(); + this.s3Client.putObject(request, AsyncRequestBody.fromByteBuffer( + ByteBuffer.wrap(buf, 0, position))).join(); + } + } + } + + /** + * Should be called to cancel multipart upload when there is any exception. + */ + public void cancel() + { + this.open = false; + if (this.uploadId != null) + { + AbortMultipartUploadRequest request = AbortMultipartUploadRequest.builder() + .bucket(this.bucket).key(this.key).uploadId(this.uploadId).build(); + this.s3Client.abortMultipartUpload(request).join(); + } + } + + @Override + public void write(int b) throws IOException + { + this.assertOpen(); + if (position >= this.buf.length) + { + flushBufferAndRewind(); + } + this.buf[position++] = (byte) b; + } + + private void assertOpen() + { + if (!this.open) + { + throw new IllegalStateException("Closed"); + } + } +} diff --git a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java index 9f1e92bb5..16828efc5 100644 --- a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java +++ b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java @@ -32,6 +32,12 @@ */ public class TestS3 { + @Test + public void testStorageScheme() + { + System.out.println(Storage.Scheme.fromPath("s3://container/object")); + } + @Test public void testS3Writer() throws IOException { diff --git a/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java b/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java index 82eb63f6d..01ce8deb6 100644 --- a/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java +++ b/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java @@ -734,7 +734,7 @@ private boolean read() throws IOException /** * Issue #105: * We use preRowInRG instead of curRowInRG to deal with queries like: - * select ... from t where f = null + * select ... from t where f = null. * Such query is invalid but Presto does not reject it. For such query, * Presto will call PageSource.getNextPage() but will not call load() on * the lazy blocks inside the returned page. diff --git a/pixels-load/src/main/java/io/pixelsdb/pixels/load/multi/Main.java b/pixels-load/src/main/java/io/pixelsdb/pixels/load/multi/Main.java index ffc4c8747..95791e251 100644 --- a/pixels-load/src/main/java/io/pixelsdb/pixels/load/multi/Main.java +++ b/pixels-load/src/main/java/io/pixelsdb/pixels/load/multi/Main.java @@ -26,7 +26,6 @@ import io.pixelsdb.pixels.common.physical.Status; import io.pixelsdb.pixels.common.physical.Storage; import io.pixelsdb.pixels.common.physical.StorageFactory; -import io.pixelsdb.pixels.common.physical.impl.HDFS; import io.pixelsdb.pixels.common.utils.ConfigFactory; import io.pixelsdb.pixels.common.utils.Constants; import io.pixelsdb.pixels.common.utils.DateUtil; @@ -359,9 +358,10 @@ public static void main(String args[]) ConfigFactory configFactory = ConfigFactory.Instance(); - HDFS storage = (HDFS) StorageFactory.Instance().getStorage("hdfs"); + Storage sourceStorage = StorageFactory.Instance().getStorage(source); + Storage destStorage = StorageFactory.Instance().getStorage(destination); - List files = storage.listStatus(source); + List files = sourceStorage.listStatus(source); long blockSize = Long.parseLong(configFactory.getProperty("block.size")) * 1024l * 1024l; short replication = Short.parseShort(configFactory.getProperty("block.replication")); @@ -374,12 +374,10 @@ public static void main(String args[]) sourceName.substring(0, sourceName.indexOf(postfix)) + "_copy_" + DateUtil.getCurTime() + postfix; String dest = destName; - DataInputStream inputStream = storage.open(s.getPath()); - DataOutputStream outputStream = storage.create(dest, false, + DataInputStream inputStream = sourceStorage.open(s.getPath()); + DataOutputStream outputStream = destStorage.create(dest, false, Constants.HDFS_BUFFER_SIZE, replication, blockSize); IOUtils.copyBytes(inputStream, outputStream, Constants.HDFS_BUFFER_SIZE, true); - inputStream.close(); - outputStream.close(); } } } From a8a4808d78e6c17bbeea025498c9531993ff1f7c Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Wed, 29 Sep 2021 04:25:37 +0200 Subject: [PATCH 11/19] add tests and fix pom. --- .../io/pixelsdb/pixels/common/TestS3.java | 23 ++++++++++++++++++- pixels-load/pom.xml | 18 +++++++++++++++ 2 files changed, 40 insertions(+), 1 deletion(-) diff --git a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java index 16828efc5..cd1dea6d3 100644 --- a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java +++ b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java @@ -20,9 +20,12 @@ package io.pixelsdb.pixels.common; import io.pixelsdb.pixels.common.physical.*; +import io.pixelsdb.pixels.common.physical.impl.S3OutputStream; +import org.apache.hadoop.io.IOUtils; import org.junit.Test; +import software.amazon.awssdk.services.s3.S3AsyncClient; -import java.io.IOException; +import java.io.*; import java.nio.ByteBuffer; import java.util.concurrent.CompletableFuture; @@ -50,6 +53,24 @@ public void testS3Writer() throws IOException writer.close(); } + @Test + public void testS3OutputStream() throws IOException + { + S3AsyncClient s3 = S3AsyncClient.builder().build(); + InputStream input = new FileInputStream("/home/hank/Downloads/JData/JData_Action_201603.csv"); + OutputStream output = new S3OutputStream(s3, "pixels-01", "object-6"); + IOUtils.copyBytes(input, output, 1024*1024, true); + } + + @Test + public void testS3Download() throws IOException + { + Storage storage = StorageFactory.Instance().getStorage("s3://pixels-01/object-6"); + InputStream input = storage.open("s3://pixels-01/object-6"); + OutputStream output = new FileOutputStream("/home/hank/JData_Action_201603.csv"); + IOUtils.copyBytes(input, output, 1024*1024, true); + } + @Test public void testS3Reader() throws IOException { diff --git a/pixels-load/pom.xml b/pixels-load/pom.xml index 294a74fb4..8d137576e 100644 --- a/pixels-load/pom.xml +++ b/pixels-load/pom.xml @@ -39,6 +39,24 @@ true
+ + io.etcd + jetcd-core + true + + + software.amazon.awssdk From 4a1ddb4e4b2def69d93fb8b792cd92df75520294 Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Wed, 29 Sep 2021 04:40:41 +0200 Subject: [PATCH 12/19] fix s3 path. --- .../java/io/pixelsdb/pixels/common/physical/impl/S3.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java index 210ad4399..c3408a35d 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java @@ -130,6 +130,11 @@ public Path(String path) } this.valid = true; } + else if (path.length() > 0) + { + this.bucket = path; + this.isBucket = true; + } } @Override From 81da9cfc39d341b7729b8a982a30b029563e0df1 Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Thu, 30 Sep 2021 05:21:59 +0200 Subject: [PATCH 13/19] implement S3InputStream. --- .../pixels/common/physical/impl/S3.java | 21 +- .../common/physical/impl/S3InputStream.java | 243 ++++++++++++++++++ .../common/physical/impl/S3OutputStream.java | 40 +-- .../io/pixelsdb/pixels/common/TestS3.java | 16 +- 4 files changed, 283 insertions(+), 37 deletions(-) create mode 100644 pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3InputStream.java diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java index c3408a35d..794937d02 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java @@ -27,8 +27,6 @@ import io.pixelsdb.pixels.common.utils.EtcdUtil; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import software.amazon.awssdk.core.ResponseBytes; -import software.amazon.awssdk.core.async.AsyncResponseTransformer; import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.S3AsyncClientBuilder; import software.amazon.awssdk.services.s3.model.*; @@ -116,6 +114,10 @@ public Path(String path) { path = path.substring(path.indexOf("://") + 3); } + else if (path.startsWith("/")) + { + path = path.substring(1); + } int slash = path.indexOf("/"); if (slash > 0) { @@ -217,13 +219,13 @@ public Status getStatus(String path) throws IOException } if (p.isBucket) { - return new Status(path, 0, true, 1); + return new Status(p.toString(), 0, true, 1); } HeadObjectRequest request = HeadObjectRequest.builder().bucket(p.bucket).key(p.key).build(); try { HeadObjectResponse response = s3.headObject(request).get(); - return new Status(path, response.contentLength(), false, 1); + return new Status(p.toString(), response.contentLength(), false, 1); } catch (Exception e) { throw new IOException("Failed to get object head of '" + path + "'", e); @@ -281,16 +283,7 @@ public DataInputStream open(String path) throws IOException { throw new IOException("Path '" + path + "' does not exist."); } - GetObjectRequest request = GetObjectRequest.builder().bucket(p.bucket).key(p.key).build(); - try - { - ResponseBytes get = - s3.getObject(request, AsyncResponseTransformer.toBytes()).get(); - return new DataInputStream(get.asInputStream()); - } catch (Exception e) - { - throw new IOException("Failed to get object '" + path + "'.", e); - } + return new DataInputStream(new S3InputStream(s3, p.bucket, p.key)); } /** diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3InputStream.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3InputStream.java new file mode 100644 index 000000000..c69ddb155 --- /dev/null +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3InputStream.java @@ -0,0 +1,243 @@ +/* + * Copyright 2021 PixelsDB. + * + * This file is part of Pixels. + * + * Pixels is free software: you can redistribute it and/or modify + * it under the terms of the Affero GNU General Public License as + * published by the Free Software Foundation, either version 3 of + * the License, or (at your option) any later version. + * + * Pixels is distributed in the hope that it will be useful, + * but WITHOUT ANY WARRANTY; without even the implied warranty of + * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + * Affero GNU General Public License for more details. + * + * You should have received a copy of the Affero GNU General Public + * License along with Pixels. If not, see + * . + */ +package io.pixelsdb.pixels.common.physical.impl; + +import software.amazon.awssdk.core.ResponseBytes; +import software.amazon.awssdk.core.async.AsyncResponseTransformer; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; + +import java.io.IOException; +import java.io.InputStream; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +/** + * Created at: 9/29/21 + * Author: hank + */ +public class S3InputStream extends InputStream +{ + /** + * Default chunk size is 128MB + */ + protected static final int BUFFER_SIZE = 128 * 1024 * 1024; + + /** + * The bucket-name on Amazon S3 + */ + private final String bucket; + + /** + * The path (key) name within the bucket + */ + private final String key; + + /** + * The temporary buffer used for storing the chunks + */ + private byte[] buffer; + + /** + * The position in the buffer + */ + private int bufferPosition; + + /** + * The position in the object + */ + private long position; + + /** + * The size of the object + */ + private final long length; + + /** + * Amazon S3 client. + */ + private final S3AsyncClient s3Client; + + /** + * indicates whether the stream is still open / valid + */ + private boolean open; + + /** + * Creates a new S3 InputStream + * + * @param s3Client the AmazonS3 client + * @param bucket name of the bucket + * @param key path (key) within the bucket + */ + public S3InputStream(S3AsyncClient s3Client, String bucket, String key) throws IOException + { + this.s3Client = s3Client; + this.bucket = bucket; + this.key = key; + this.bufferPosition = 0; + this.position = 0L; + this.open = true; + + HeadObjectRequest request = HeadObjectRequest.builder().bucket(bucket).key(key).build(); + try + { + HeadObjectResponse response = s3Client.headObject(request).get(); + this.length = response.contentLength(); + } catch (Exception e) + { + throw new IOException("Failed to get object head of '" + bucket + "/" + key + "'", e); + } + } + + @Override + public int read() throws IOException + { + this.assertOpen(); + if (bufferPosition >= this.buffer.length) + { + populateBuffer(); + } + return this.buffer[bufferPosition++]; + } + + @Override + public int read(byte[] b) throws IOException + { + return read(b, 0, b.length); + } + + @Override + public int read(byte[] buf, int off, int len) throws IOException + { + this.assertOpen(); + if (this.buffer == null) + { + // try to populate the buffer for the first time or after exception or EOF. + if (populateBuffer() < 0) + { + return -1; + } + } + int offsetInBuf = off, remainToRead = len; + + while (remainToRead > 0) + { + int remainInBuffer = this.buffer.length - bufferPosition; + if (remainInBuffer >= remainToRead) + { + // The read can be served in buffer. + System.arraycopy(this.buffer, this.bufferPosition, buf, offsetInBuf, remainToRead); + this.bufferPosition += remainToRead; + offsetInBuf += remainToRead; + remainToRead = 0; + break; + } + // Read the remaining bytes in buffer. + System.arraycopy(this.buffer, this.bufferPosition, buf, offsetInBuf, remainInBuffer); + this.bufferPosition += remainInBuffer; + offsetInBuf += remainInBuffer; + remainToRead -= remainInBuffer; + // try to populate the buffer. + if (populateBuffer() < 0) + { + // reach EOF, this.buffer is set to null, the next read will return -1. + break; + } + } + + return offsetInBuf - off; + } + + /** + * Populate the read buffer. + * @return the bytes been populated into the read buffer, -1 if reaches EOF. + * @throws IOException + */ + protected int populateBuffer() throws IOException + { + int bytesToRead = BUFFER_SIZE; + if (this.length - this.position < bytesToRead) + { + // Do not exceed EOF. + bytesToRead = (int)(this.length - this.position); + if (bytesToRead <= 0) + { + this.buffer = null; + this.bufferPosition = 0; + return -1; + } + } + GetObjectRequest request = GetObjectRequest.builder().bucket(this.bucket) + .key(this.key).range(toRange(this.position, bytesToRead)).build(); + CompletableFuture> future = + this.s3Client.getObject(request, AsyncResponseTransformer.toBytes()); + try + { + this.buffer = future.get().asByteArray(); + this.bufferPosition = 0; + this.position += this.buffer.length; + return this.buffer.length; + } catch (InterruptedException | ExecutionException e) + { + this.buffer = null; + this.bufferPosition = 0; + throw new IOException("Failed to read part.", e); + } + } + + private String toRange(long start, int length) + { + StringBuilder builder = new StringBuilder("bytes="); + builder.append(start).append('-').append(start+length); + return builder.toString(); + } + + @Override + public int available() throws IOException + { + if (this.length - this.position > Integer.MAX_VALUE) + { + return Integer.MAX_VALUE; + } + return (int) (this.length - this.position); + } + + @Override + public void close() throws IOException + { + if (this.open) + { + this.open = false; + // Don't close s3Client as it is external. + } + } + + private void assertOpen() + { + if (!this.open) + { + throw new IllegalStateException("Closed"); + } + } +} diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3OutputStream.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3OutputStream.java index da181ba88..6b956d466 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3OutputStream.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3OutputStream.java @@ -60,7 +60,7 @@ public class S3OutputStream extends OutputStream /** * The temporary buffer used for storing the chunks */ - private final byte[] buf; + private final byte[] buffer; /** * The position in the buffer @@ -75,7 +75,7 @@ public class S3OutputStream extends OutputStream /** * The unique id for this upload */ - private String uploadId; + private String uploadId = null; /** * Collection of the etags for the parts that have been uploaded @@ -99,7 +99,7 @@ public S3OutputStream(S3AsyncClient s3Client, String bucket, String key) this.s3Client = s3Client; this.bucket = bucket; this.key = key; - this.buf = new byte[BUFFER_SIZE]; + this.buffer = new byte[BUFFER_SIZE]; this.position = 0; this.parts = new ArrayList<>(); this.open = true; @@ -119,26 +119,26 @@ public void write(byte[] b) throws IOException /** * Writes an array to the S3 Output Stream * - * @param byteArray the array to write - * @param o the offset into the array - * @param l the number of bytes to write + * @param buf the array to write + * @param off the offset into the array + * @param len the number of bytes to write */ @Override - public void write(final byte[] byteArray, final int o, final int l) throws IOException + public void write(final byte[] buf, final int off, final int len) throws IOException { this.assertOpen(); - int ofs = o, len = l; - int size; - while (len > (size = this.buf.length - position)) + int offsetInBuf = off, remainToRead = len; + int remainInBuffer; + while (remainToRead > (remainInBuffer = this.buffer.length - position)) { - System.arraycopy(byteArray, ofs, this.buf, this.position, size); - this.position += size; + System.arraycopy(buf, offsetInBuf, this.buffer, this.position, remainInBuffer); + this.position += remainInBuffer; flushBufferAndRewind(); - ofs += size; - len -= size; + offsetInBuf += remainInBuffer; + remainToRead -= remainInBuffer; } - System.arraycopy(byteArray, ofs, this.buf, this.position, len); - this.position += len; + System.arraycopy(buf, offsetInBuf, this.buffer, this.position, remainToRead); + this.position += remainToRead; } /** @@ -178,7 +178,7 @@ protected void uploadPart() throws IOException .partNumber(this.parts.size() + 1).build(); CompletableFuture response = this.s3Client.uploadPart(request, AsyncRequestBody.fromByteBuffer( - ByteBuffer.wrap(buf, 0, position))); + ByteBuffer.wrap(buffer, 0, position))); try { String etag = response.get().eTag(); @@ -218,7 +218,7 @@ public void close() throws IOException final PutObjectRequest request = PutObjectRequest.builder().bucket(this.bucket).key(this.key) .acl(ObjectCannedACL.BUCKET_OWNER_FULL_CONTROL).build(); this.s3Client.putObject(request, AsyncRequestBody.fromByteBuffer( - ByteBuffer.wrap(buf, 0, position))).join(); + ByteBuffer.wrap(buffer, 0, position))).join(); } } } @@ -241,11 +241,11 @@ public void cancel() public void write(int b) throws IOException { this.assertOpen(); - if (position >= this.buf.length) + if (position >= this.buffer.length) { flushBufferAndRewind(); } - this.buf[position++] = (byte) b; + this.buffer[position++] = (byte) b; } private void assertOpen() diff --git a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java index cd1dea6d3..51f0cc7d6 100644 --- a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java +++ b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java @@ -65,12 +65,22 @@ public void testS3OutputStream() throws IOException @Test public void testS3Download() throws IOException { - Storage storage = StorageFactory.Instance().getStorage("s3://pixels-01/object-6"); - InputStream input = storage.open("s3://pixels-01/object-6"); - OutputStream output = new FileOutputStream("/home/hank/JData_Action_201603.csv"); + Storage storage = StorageFactory.Instance().getStorage("s3://pixels-00/20200828093836_0.compact_copy_20210929102009_0.pxl"); + InputStream input = storage.open("s3://pixels-00/20200828093836_0.compact_copy_20210929102009_0.pxl"); + OutputStream output = new FileOutputStream("20200828093836_0.compact_copy_20210929102009_0.pxl"); IOUtils.copyBytes(input, output, 1024*1024, true); } + @Test + public void testGetStatus() throws IOException + { + Storage storage = StorageFactory.Instance().getStorage("s3://pixels-00/20200828093836_0.compact_copy_20210929102009_0.pxl"); + Status status = storage.getStatus("/pixels-00/20200828093836_0.compact_copy_20210929102009_0.pxl"); + System.out.println(status.getLength()); + System.out.println(status.getName()); + System.out.println(status.getPath()); + } + @Test public void testS3Reader() throws IOException { From 5a627a7a59d1c028e992bc7560a4518d4c45cbcc Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Thu, 30 Sep 2021 07:41:10 +0200 Subject: [PATCH 14/19] support direct copy in s3 and fix bugs. --- .../pixels/common/physical/Storage.java | 21 ++++++++++ .../common/physical/StorageFactory.java | 8 ++++ .../pixels/common/physical/impl/HDFS.java | 18 +++++++++ .../pixels/common/physical/impl/LocalFS.java | 15 +++++++ .../pixels/common/physical/impl/S3.java | 39 ++++++++++++++++++- .../io/pixelsdb/pixels/common/TestS3.java | 12 +++--- .../io/pixelsdb/pixels/load/multi/Main.java | 28 ++++++++++--- 7 files changed, 127 insertions(+), 14 deletions(-) diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Storage.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Storage.java index 59c1bcdbd..54c71b723 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Storage.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Storage.java @@ -174,6 +174,27 @@ DataOutputStream create(String path, boolean overwrite, */ boolean delete(String path, boolean recursive) throws IOException; + /** + * Whether this storage supports direct (short circuit) copying. + * @return true if copy is supported. + */ + boolean supportDirectCopy(); + + /** + * Copy from the source to the destination without going through this client. + * @param src + * @param dest + * @return + * @throws IOException + */ + boolean directCopy(String src, String dest) throws IOException; + + /** + * Close the storage. + * @throws IOException + */ + void close() throws IOException; + /** * For local fs, path is considered as local. * @param path diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/StorageFactory.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/StorageFactory.java index 5c880fc15..d3bb61993 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/StorageFactory.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/StorageFactory.java @@ -112,4 +112,12 @@ else if (scheme == Storage.Scheme.file) return storage; } + + public void closeAll() throws IOException + { + for (Storage.Scheme scheme : storageImpls.keySet()) + { + storageImpls.get(scheme).close(); + } + } } diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/HDFS.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/HDFS.java index 2405fb53b..18b3a1098 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/HDFS.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/HDFS.java @@ -277,6 +277,24 @@ public boolean delete(String path, boolean recursive) throws IOException return fs.delete(new Path(path), recursive); } + @Override + public boolean supportDirectCopy() + { + return false; + } + + @Override + public boolean directCopy(String src, String dest) + { + throw new UnsupportedOperationException("direct copy is unsupported on HDFS storage."); + } + + @Override + public void close() throws IOException + { + this.fs.close(); + } + @Override public boolean exists(String path) throws IOException { diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/LocalFS.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/LocalFS.java index 3559d2b99..51b927905 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/LocalFS.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/LocalFS.java @@ -261,6 +261,21 @@ public boolean delete(String path, boolean recursive) throws IOException return subDeleted && new File(path).delete(); } + @Override + public boolean supportDirectCopy() + { + return false; + } + + @Override + public boolean directCopy(String src, String dest) + { + throw new UnsupportedOperationException("Direct copy is unsupported on LocalFS storage."); + } + + @Override + public void close() throws IOException { } + @Override public boolean exists(String path) { diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java index 794937d02..4ac8ee235 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java @@ -136,6 +136,7 @@ else if (path.length() > 0) { this.bucket = path; this.isBucket = true; + this.valid = true; } } @@ -168,9 +169,9 @@ public List listStatus(String path) throws IOException { throw new IOException("Path '" + path + "' is not valid."); } - if (!this.exists(path)) + if (!p.isBucket) { - throw new IOException("Path '" + path + "' does not exist."); + throw new IOException("Path '" + path + "' is not a directory (bucket)."); } ListObjectsV2Request request = ListObjectsV2Request.builder() .bucket(p.bucket).build(); @@ -180,6 +181,7 @@ public List listStatus(String path) throws IOException List objects = response.get().contents(); List statuses = new ArrayList<>(); Path op = new Path(path); + op.isBucket = false; for (S3Object object : objects) { op.key = object.key(); @@ -361,6 +363,39 @@ public boolean delete(String path, boolean recursive) throws IOException return true; } + @Override + public boolean supportDirectCopy() + { + return true; + } + + @Override + public boolean directCopy(String src, String dest) throws IOException + { + Path srcPath = new Path(src); + Path destPath = new Path(dest); + CopyObjectRequest copyReq = CopyObjectRequest.builder() + .copySource(srcPath.toString()) + .destinationBucket(destPath.bucket) + .destinationKey(destPath.key) + .build(); + try + { + s3.copyObject(copyReq).join(); + return true; + } + catch (RuntimeException e) + { + throw new IOException("Failed to copy object from '" + src + "' to '" + dest + "'", e); + } + } + + @Override + public void close() throws IOException + { + s3.close(); + } + @Override public boolean exists(String path) { diff --git a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java index 51f0cc7d6..a24886ba2 100644 --- a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java +++ b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java @@ -57,7 +57,7 @@ public void testS3Writer() throws IOException public void testS3OutputStream() throws IOException { S3AsyncClient s3 = S3AsyncClient.builder().build(); - InputStream input = new FileInputStream("/home/hank/Downloads/JData/JData_Action_201603.csv"); + InputStream input = new FileInputStream("/home/hank/test.csv"); OutputStream output = new S3OutputStream(s3, "pixels-01", "object-6"); IOUtils.copyBytes(input, output, 1024*1024, true); } @@ -65,17 +65,17 @@ public void testS3OutputStream() throws IOException @Test public void testS3Download() throws IOException { - Storage storage = StorageFactory.Instance().getStorage("s3://pixels-00/20200828093836_0.compact_copy_20210929102009_0.pxl"); - InputStream input = storage.open("s3://pixels-00/20200828093836_0.compact_copy_20210929102009_0.pxl"); - OutputStream output = new FileOutputStream("20200828093836_0.compact_copy_20210929102009_0.pxl"); + Storage storage = StorageFactory.Instance().getStorage("s3://pixels-01/test.pxl"); + InputStream input = storage.open("s3://pixels-01/test.pxl"); + OutputStream output = new FileOutputStream("test.pxl"); IOUtils.copyBytes(input, output, 1024*1024, true); } @Test public void testGetStatus() throws IOException { - Storage storage = StorageFactory.Instance().getStorage("s3://pixels-00/20200828093836_0.compact_copy_20210929102009_0.pxl"); - Status status = storage.getStatus("/pixels-00/20200828093836_0.compact_copy_20210929102009_0.pxl"); + Storage storage = StorageFactory.Instance().getStorage("s3://pixels-01/test.pxl"); + Status status = storage.getStatus("s3://pixels-01/test.pxl"); System.out.println(status.getLength()); System.out.println(status.getName()); System.out.println(status.getPath()); diff --git a/pixels-load/src/main/java/io/pixelsdb/pixels/load/multi/Main.java b/pixels-load/src/main/java/io/pixelsdb/pixels/load/multi/Main.java index 95791e251..4f815d3dc 100644 --- a/pixels-load/src/main/java/io/pixelsdb/pixels/load/multi/Main.java +++ b/pixels-load/src/main/java/io/pixelsdb/pixels/load/multi/Main.java @@ -29,6 +29,7 @@ import io.pixelsdb.pixels.common.utils.ConfigFactory; import io.pixelsdb.pixels.common.utils.Constants; import io.pixelsdb.pixels.common.utils.DateUtil; +import io.pixelsdb.pixels.common.utils.EtcdUtil; import io.pixelsdb.pixels.core.compactor.CompactLayout; import io.pixelsdb.pixels.core.compactor.PixelsCompactor; import net.sourceforge.argparse4j.ArgumentParsers; @@ -104,6 +105,14 @@ public static void main(String args[]) if (inputStr.equalsIgnoreCase("exit") || inputStr.equalsIgnoreCase("quit") || inputStr.equalsIgnoreCase("-q")) { + try + { + StorageFactory.Instance().closeAll(); + EtcdUtil.Instance().getClient().close(); + } catch (IOException e) + { + e.printStackTrace(); + } System.out.println("Bye."); break; } @@ -370,14 +379,21 @@ public static void main(String args[]) for (Status s : files) { String sourceName = s.getName(); - String destName = destination + + String destPath = destination + sourceName.substring(0, sourceName.indexOf(postfix)) + "_copy_" + DateUtil.getCurTime() + postfix; - String dest = destName; - DataInputStream inputStream = sourceStorage.open(s.getPath()); - DataOutputStream outputStream = destStorage.create(dest, false, - Constants.HDFS_BUFFER_SIZE, replication, blockSize); - IOUtils.copyBytes(inputStream, outputStream, Constants.HDFS_BUFFER_SIZE, true); + if (sourceStorage.getScheme() == destStorage.getScheme() && + destStorage.supportDirectCopy()) + { + destStorage.directCopy(s.getPath(), destPath); + } + else + { + DataInputStream inputStream = sourceStorage.open(s.getPath()); + DataOutputStream outputStream = destStorage.create(destPath, false, + Constants.HDFS_BUFFER_SIZE, replication, blockSize); + IOUtils.copyBytes(inputStream, outputStream, Constants.HDFS_BUFFER_SIZE, true); + } } } } From 6e7a68db0f98188402dfaa2cbf190f09f5735173 Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Thu, 30 Sep 2021 09:45:44 +0200 Subject: [PATCH 15/19] fix etcd lock release and data copying. --- .../pixels/common/lock/EtcdReadWriteLock.java | 24 +++++++++++++++---- .../pixels/common/lock/LockInternals.java | 7 ++++-- .../io/pixelsdb/pixels/load/multi/Main.java | 4 ++++ 3 files changed, 29 insertions(+), 6 deletions(-) diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/lock/EtcdReadWriteLock.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/lock/EtcdReadWriteLock.java index ad984481e..4373f6be5 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/lock/EtcdReadWriteLock.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/lock/EtcdReadWriteLock.java @@ -23,12 +23,17 @@ /** * @author: tao + * @author hank * @date: Create in 2018-10-27 14:29 **/ public class EtcdReadWriteLock { - private final EtcdMutex readMutex; - private final EtcdMutex writeMutex; + private EtcdMutex readMutex; + private EtcdMutex writeMutex; + + private final Client client; + private final String basePath; + private final byte[] lockData; private static final String READ_LOCK_NAME = "_READ_"; private static final String WRITE_LOCK_NAME = "_WRIT_"; @@ -49,8 +54,11 @@ public EtcdReadWriteLock(Client client, String basePath) */ public EtcdReadWriteLock(Client client, String basePath, byte[] lockData) { - this.writeMutex = new EtcdReadWriteLock.InternalInterProcessMutex(client, basePath, WRITE_LOCK_NAME, lockData); - this.readMutex = new EtcdReadWriteLock.InternalInterProcessMutex(client, basePath, READ_LOCK_NAME, lockData); + this.client = client; + this.basePath = basePath; + this.lockData = lockData; + this.readMutex = null; + this.writeMutex = null; } /** @@ -60,6 +68,10 @@ public EtcdReadWriteLock(Client client, String basePath, byte[] lockData) */ public EtcdMutex readLock() { + if (this.readMutex == null) + { + this.readMutex = new EtcdReadWriteLock.InternalInterProcessMutex(client, basePath, READ_LOCK_NAME, lockData); + } return this.readMutex; } @@ -70,6 +82,10 @@ public EtcdMutex readLock() */ public EtcdMutex writeLock() { + if (this.writeMutex == null) + { + this.writeMutex = new EtcdReadWriteLock.InternalInterProcessMutex(client, basePath, WRITE_LOCK_NAME, lockData); + } return this.writeMutex; } diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/lock/LockInternals.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/lock/LockInternals.java index 7d3665a2c..2c886d794 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/lock/LockInternals.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/lock/LockInternals.java @@ -55,6 +55,7 @@ public class LockInternals private Long leaseId = 0L; private static AtomicInteger count = new AtomicInteger(0); private volatile Map pathToVersion = new HashMap<>(); + private ScheduledExecutorService keepAliveService; public LockInternals(Client client, String path, String lockName) { @@ -72,8 +73,9 @@ public LockInternals(Client client, String path, String lockName) logger.error("[create-lease-error]: " + e1); return; } - ScheduledExecutorService service = Executors.newSingleThreadScheduledExecutor(); - service.scheduleAtFixedRate(new KeepAliveTask(leaseClient, leaseId), 1, 12, TimeUnit.SECONDS); + keepAliveService = Executors.newSingleThreadScheduledExecutor(); + keepAliveService.scheduleAtFixedRate(new KeepAliveTask(leaseClient, leaseId), 1, 12, TimeUnit.SECONDS); + keepAliveService.shutdown(); } LockInternals verbose(boolean verbose) @@ -348,6 +350,7 @@ private void deleteOurPath(String ourPath) throws Exception public void releaseLock(String lockPath) throws Exception { deleteOurPath(lockPath); + this.keepAliveService.shutdownNow(); } public static class KeepAliveTask implements Runnable diff --git a/pixels-load/src/main/java/io/pixelsdb/pixels/load/multi/Main.java b/pixels-load/src/main/java/io/pixelsdb/pixels/load/multi/Main.java index 4f815d3dc..8ac3dfed1 100644 --- a/pixels-load/src/main/java/io/pixelsdb/pixels/load/multi/Main.java +++ b/pixels-load/src/main/java/io/pixelsdb/pixels/load/multi/Main.java @@ -379,6 +379,10 @@ public static void main(String args[]) for (Status s : files) { String sourceName = s.getName(); + if (!sourceName.contains(postfix)) + { + continue; + } String destPath = destination + sourceName.substring(0, sourceName.indexOf(postfix)) + "_copy_" + DateUtil.getCurTime() + postfix; From 50cebb45f858ad51c1f002b6e8578e1a138df1cf Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Thu, 30 Sep 2021 10:16:10 +0200 Subject: [PATCH 16/19] fix s3 input stream. --- .../pixels/common/physical/impl/S3InputStream.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3InputStream.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3InputStream.java index c69ddb155..d832cc2ac 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3InputStream.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3InputStream.java @@ -116,7 +116,10 @@ public int read() throws IOException this.assertOpen(); if (bufferPosition >= this.buffer.length) { - populateBuffer(); + if (populateBuffer() < 0) + { + return -1; + } } return this.buffer[bufferPosition++]; } @@ -140,7 +143,6 @@ public int read(byte[] buf, int off, int len) throws IOException } } int offsetInBuf = off, remainToRead = len; - while (remainToRead > 0) { int remainInBuffer = this.buffer.length - bufferPosition; @@ -216,11 +218,12 @@ private String toRange(long start, int length) @Override public int available() throws IOException { - if (this.length - this.position > Integer.MAX_VALUE) + long available = this.length - this.position; + if (this.buffer != null) { - return Integer.MAX_VALUE; + available += this.buffer.length - this.bufferPosition; } - return (int) (this.length - this.position); + return available > Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) available; } @Override From 13ef35548a1db198635c3d94d3ec64407a34e334 Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Fri, 1 Oct 2021 02:41:46 +0200 Subject: [PATCH 17/19] fix s3 storage. --- pixels-common/pom.xml | 6 ++ .../physical/impl/PhysicalS3Reader.java | 8 ++- .../pixels/common/physical/impl/S3.java | 65 ++++++++++--------- .../io/pixelsdb/pixels/common/TestS3.java | 30 +++++++++ .../core/reader/PixelsRecordReaderImpl.java | 4 +- pixels-daemon/pom.xml | 6 ++ pixels-load/pom.xml | 6 ++ pixels-presto/pom.xml | 6 ++ 8 files changed, 97 insertions(+), 34 deletions(-) diff --git a/pixels-common/pom.xml b/pixels-common/pom.xml index 2563e91fc..eede71dc7 100644 --- a/pixels-common/pom.xml +++ b/pixels-common/pom.xml @@ -55,6 +55,12 @@ true + + software.amazon.awssdk + netty-nio-client + true + + com.google.protobuf diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java index 98c395b79..aba0fdbde 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java @@ -70,7 +70,7 @@ public PhysicalS3Reader(Storage storage, String path) throws IOException this.id = this.s3.getFileId(path); this.length = this.s3.getStatus(path).getLength(); this.position = new AtomicLong(0); - this.client = s3.getClient(); + this.client = this.s3.getClient(); } private String toRange(long start, int length) @@ -139,7 +139,8 @@ public void readFully(byte[] buffer, int off, int len) throws IOException @Override public boolean supportsAsync() { - return true; + // TODO: async read does not work properly. + return false; } @Override @@ -194,7 +195,8 @@ public int readInt() throws IOException @Override public void close() throws IOException { - this.client.close(); + // Should not close the client because it is shared by all threads. + // this.client.close(); // Closing s3 client may take several seconds. } @Override diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java index 4ac8ee235..a1cc8e23a 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java @@ -27,8 +27,11 @@ import io.pixelsdb.pixels.common.utils.EtcdUtil; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient; +import software.amazon.awssdk.http.nio.netty.SdkEventLoopGroup; import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.S3AsyncClientBuilder; +import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.*; import java.io.DataInputStream; @@ -37,8 +40,6 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import static io.pixelsdb.pixels.common.lock.EtcdAutoIncrement.GenerateId; @@ -61,15 +62,20 @@ public class S3 implements Storage { private static Logger logger = LogManager.getLogger(S3.class); - private static S3AsyncClient s3; + private static S3Client s3; + private static S3AsyncClient s3Async; static { InitId(S3_ID_KEY); - S3AsyncClientBuilder builder = S3AsyncClient.builder(); - // TODO: config builder. - s3 = builder.build(); + S3AsyncClientBuilder builder = S3AsyncClient.builder() + .httpClientBuilder(NettyNioAsyncHttpClient.builder() + .eventLoopGroup(SdkEventLoopGroup.builder().numberOfThreads(10).build()) + .maxConcurrency(100).maxPendingConnectionAcquires(10_000)); + s3Async = builder.build(); + + s3 = S3Client.builder().build(); } private String[] allHosts; @@ -175,23 +181,17 @@ public List listStatus(String path) throws IOException } ListObjectsV2Request request = ListObjectsV2Request.builder() .bucket(p.bucket).build(); - CompletableFuture response = s3.listObjectsV2(request); - try + ListObjectsV2Response response = s3.listObjectsV2(request); + List objects = response.contents(); + List statuses = new ArrayList<>(); + Path op = new Path(path); + op.isBucket = false; + for (S3Object object : objects) { - List objects = response.get().contents(); - List statuses = new ArrayList<>(); - Path op = new Path(path); - op.isBucket = false; - for (S3Object object : objects) - { - op.key = object.key(); - statuses.add(new Status(op.toString(), object.size(), false, 1)); - } - return statuses; - } catch (InterruptedException | ExecutionException e) - { - throw new IOException("Failed to list objects.", e); + op.key = object.key(); + statuses.add(new Status(op.toString(), object.size(), false, 1)); } + return statuses; } @Override @@ -226,7 +226,7 @@ public Status getStatus(String path) throws IOException HeadObjectRequest request = HeadObjectRequest.builder().bucket(p.bucket).key(p.key).build(); try { - HeadObjectResponse response = s3.headObject(request).get(); + HeadObjectResponse response = s3.headObject(request); return new Status(p.toString(), response.contentLength(), false, 1); } catch (Exception e) { @@ -238,6 +238,13 @@ public Status getStatus(String path) throws IOException public long getFileId(String path) throws IOException { KeyValue kv = EtcdUtil.Instance().getKeyValue(getPathKey(path)); + if (kv == null) + { + // the file id does not exist, register a new id for this file. + long id = GenerateId(S3_ID_KEY); + EtcdUtil.Instance().putKeyValue(getPathKey(path), Long.toString(id)); + return id; + } return Long.parseLong(kv.getValue().toString(StandardCharsets.UTF_8)); } @@ -285,7 +292,7 @@ public DataInputStream open(String path) throws IOException { throw new IOException("Path '" + path + "' does not exist."); } - return new DataInputStream(new S3InputStream(s3, p.bucket, p.key)); + return new DataInputStream(new S3InputStream(s3Async, p.bucket, p.key)); } /** @@ -312,7 +319,7 @@ public DataOutputStream create(String path, boolean overwrite, int bufferSize, s } long id = GenerateId(S3_ID_KEY); EtcdUtil.Instance().putKeyValue(getPathKey(path), Long.toString(id)); - return new DataOutputStream(new S3OutputStream(s3, p.bucket, p.key)); + return new DataOutputStream(new S3OutputStream(s3Async, p.bucket, p.key)); } @Override @@ -342,7 +349,7 @@ public boolean delete(String path, boolean recursive) throws IOException DeleteObjectRequest request = DeleteObjectRequest.builder().bucket(sub.bucket).key(sub.key).build(); try { - s3.deleteObject(request).get(); + s3Async.deleteObject(request).get(); } catch (Exception e) { throw new IOException("Failed to delete object '" + sub.bucket + "/" + sub.key + "' from S3.", e); @@ -354,7 +361,7 @@ public boolean delete(String path, boolean recursive) throws IOException DeleteObjectRequest request = DeleteObjectRequest.builder().bucket(p.bucket).key(p.key).build(); try { - s3.deleteObject(request).get(); + s3Async.deleteObject(request).get(); } catch (Exception e) { throw new IOException("Failed to delete object '" + p.bucket + "/" + p.key + "' from S3.", e); @@ -381,7 +388,7 @@ public boolean directCopy(String src, String dest) throws IOException .build(); try { - s3.copyObject(copyReq).join(); + s3Async.copyObject(copyReq).join(); return true; } catch (RuntimeException e) @@ -393,7 +400,7 @@ public boolean directCopy(String src, String dest) throws IOException @Override public void close() throws IOException { - s3.close(); + s3Async.close(); } @Override @@ -416,6 +423,6 @@ public boolean isDirectory(String path) public S3AsyncClient getClient() { - return s3; + return s3Async; } } diff --git a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java index a24886ba2..f876012e3 100644 --- a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java +++ b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java @@ -27,6 +27,7 @@ import java.io.*; import java.nio.ByteBuffer; +import java.util.List; import java.util.concurrent.CompletableFuture; /** @@ -81,6 +82,34 @@ public void testGetStatus() throws IOException System.out.println(status.getPath()); } + @Test + public void testlistStatus() throws IOException, InterruptedException + { + Storage storage = StorageFactory.Instance().getStorage("s3://pixels-00"); + List statuses = storage.listStatus("s3://pixels-00"); + System.out.println(statuses.size()); + for (Status status : statuses) + { + System.out.println(status.getPath()); + } + } + + @Test + public void testGetPaths() throws IOException + { + Storage storage = StorageFactory.Instance().getStorage("s3://pixels-00"); + List paths = storage.listPaths("s3://pixels-00"); + if (paths == null) + { + System.out.println("null"); + return; + } + for (String path : paths) + { + System.out.println(path); + } + } + @Test public void testS3Reader() throws IOException { @@ -98,5 +127,6 @@ public void testS3Reader() throws IOException } }); future.join(); + reader.close(); } } diff --git a/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java b/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java index 01ce8deb6..4ac6c4274 100644 --- a/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java +++ b/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java @@ -80,8 +80,8 @@ public class PixelsRecordReaderImpl private int curRGIdx = 0; // index of current reading row group in targetRGs private int curRowInRG = 0; // starting index of values to read by reader in current row group - private PixelsProto.RowGroupFooter[] rowGroupFooters; - private ByteBuffer[] chunkBuffers; // buffers of each chunk in this file, arranged by chunk's row group id and column id + private volatile PixelsProto.RowGroupFooter[] rowGroupFooters; + private volatile ByteBuffer[] chunkBuffers; // buffers of each chunk in this file, arranged by chunk's row group id and column id private ColumnReader[] readers; // column readers for each target columns private long diskReadBytes = 0L; diff --git a/pixels-daemon/pom.xml b/pixels-daemon/pom.xml index e057b9ce4..56f304cbe 100644 --- a/pixels-daemon/pom.xml +++ b/pixels-daemon/pom.xml @@ -86,6 +86,12 @@ true + + software.amazon.awssdk + netty-nio-client + true + + org.apache.hadoop diff --git a/pixels-load/pom.xml b/pixels-load/pom.xml index 8d137576e..558c65b11 100644 --- a/pixels-load/pom.xml +++ b/pixels-load/pom.xml @@ -64,6 +64,12 @@ true + + software.amazon.awssdk + netty-nio-client + true + + org.apache.hadoop diff --git a/pixels-presto/pom.xml b/pixels-presto/pom.xml index a729d1bdd..e6cc12165 100644 --- a/pixels-presto/pom.xml +++ b/pixels-presto/pom.xml @@ -39,6 +39,12 @@ true + + software.amazon.awssdk + netty-nio-client + true + + org.apache.hadoop From 4b6a34ff13c13368bb20bcdbcf657e641ee6c4db Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Fri, 1 Oct 2021 03:16:14 +0200 Subject: [PATCH 18/19] use S3OutputStream in PhysicalS3Writer. --- .../physical/impl/PhysicalS3Writer.java | 20 +++---------------- 1 file changed, 3 insertions(+), 17 deletions(-) diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java index 1a460fa17..f027dfbc2 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Writer.java @@ -24,10 +24,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import software.amazon.awssdk.services.s3.S3AsyncClient; -import software.amazon.awssdk.services.s3.model.PutObjectRequest; -import java.io.File; -import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; @@ -47,7 +44,6 @@ public class PhysicalS3Writer implements PhysicalWriter private String pathStr; private long position; private S3AsyncClient client; - private File tempFile; private OutputStream out; public PhysicalS3Writer(Storage storage, String path) throws IOException @@ -70,8 +66,7 @@ public PhysicalS3Writer(Storage storage, String path) throws IOException this.position = 0L; this.client = s3.getClient(); this.s3.create(path, false, S3_BUFFER_SIZE, (short)1); - this.tempFile = File.createTempFile("pixels-s3-", ".tmp"); - this.out = new FileOutputStream(tempFile); + this.out = new S3OutputStream(this.client, this.path.bucket, this.path.key); } /** @@ -125,17 +120,8 @@ public long append(byte[] buffer, int offset, int length) throws IOException public void close() throws IOException { this.out.close(); - PutObjectRequest request = PutObjectRequest.builder() - .bucket(path.bucket).key(path.key).build(); - try - { - this.client.putObject(request, this.tempFile.toPath()).get(); - } catch (Exception e) - { - throw new IOException("Failed to put local temp file to S3.", e); - } - this.tempFile.deleteOnExit(); - this.client.close(); + // Don't close the client as it is external. + // this.client.close(); } /** From 182139655c40359529a0aecd5c184de7d4f41620 Mon Sep 17 00:00:00 2001 From: Haoqiong Bian Date: Fri, 1 Oct 2021 08:48:19 +0200 Subject: [PATCH 19/19] fix s3 async read. --- .../common/physical/PhysicalReader.java | 9 +- .../pixels/common/physical/Scheduler.java | 10 +- .../physical/impl/PhysicalHDFSReader.java | 2 +- .../physical/impl/PhysicalLocalReader.java | 2 +- .../physical/impl/PhysicalS3Reader.java | 14 +-- .../pixels/common/physical/impl/S3.java | 10 +- .../common/physical/impl/S3InputStream.java | 2 +- .../impl/scheduler/NoopScheduler.java | 10 +- .../impl/scheduler/SortMergeScheduler.java | 13 ++- .../io/pixelsdb/pixels/common/TestS3.java | 37 ++++++- .../core/reader/PixelsRecordReaderImpl.java | 100 +++++++++--------- 11 files changed, 126 insertions(+), 83 deletions(-) diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalReader.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalReader.java index db47c3832..964e41df8 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalReader.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/PhysicalReader.java @@ -46,7 +46,14 @@ public interface PhysicalReader */ boolean supportsAsync(); - CompletableFuture readAsync(int length) throws IOException; + /** + * readAsync does not affect the position of this reader, and is not affected by seek(). + * @param offset + * @param length + * @return + * @throws IOException + */ + CompletableFuture readAsync(long offset, int length) throws IOException; long readLong() throws IOException; diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Scheduler.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Scheduler.java index 20daa52ed..7672d1dde 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Scheduler.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/Scheduler.java @@ -42,7 +42,9 @@ public interface Scheduler * @return should never return null. * @throws IOException */ - CompletableFuture executeBatch(PhysicalReader reader, RequestBatch batch) throws IOException; + CompletableFuture executeBatch(PhysicalReader reader, RequestBatch batch, + List actionFutures) + throws IOException; class Request implements Comparable { @@ -137,8 +139,10 @@ public List> getFutures() * If batch is empty, this method returns and completed future. * @return */ - public CompletableFuture completeAll() + public CompletableFuture completeAll(List actionFutures) { + assert actionFutures != null; + assert actionFutures.size() == size; if (size <= 0) { CompletableFuture future = new CompletableFuture<>(); @@ -148,7 +152,7 @@ public CompletableFuture completeAll() CompletableFuture[] fs = new CompletableFuture[size]; for (int i = 0; i < size; ++i) { - fs[i] = this.futures.get(i); + fs[i] = actionFutures.get(i); } return CompletableFuture.allOf(fs); } diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalHDFSReader.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalHDFSReader.java index 96ea5a604..816ed9721 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalHDFSReader.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalHDFSReader.java @@ -203,7 +203,7 @@ public boolean supportsAsync() } @Override - public CompletableFuture readAsync(int length) throws IOException + public CompletableFuture readAsync(long offset, int length) throws IOException { throw new IOException("Asynchronous read is not supported for HDFS."); } diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalReader.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalReader.java index d7c03f996..5d8041fa6 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalReader.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalLocalReader.java @@ -100,7 +100,7 @@ public boolean supportsAsync() } @Override - public CompletableFuture readAsync(int length) throws IOException + public CompletableFuture readAsync(long offset, int length) throws IOException { throw new IOException("Asynchronous read is not supported for local fs."); } diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java index aba0fdbde..544e4a27c 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/PhysicalS3Reader.java @@ -75,8 +75,9 @@ public PhysicalS3Reader(Storage storage, String path) throws IOException private String toRange(long start, int length) { + assert start >= 0 && length > 0; StringBuilder builder = new StringBuilder("bytes="); - builder.append(start).append('-').append(start+length); + builder.append(start).append('-').append(start+length-1); return builder.toString(); } @@ -140,19 +141,19 @@ public void readFully(byte[] buffer, int off, int len) throws IOException public boolean supportsAsync() { // TODO: async read does not work properly. - return false; + return true; } @Override - public CompletableFuture readAsync(int len) throws IOException + public CompletableFuture readAsync(long offset, int len) throws IOException { - if (this.position.get() + len > this.length) + if (offset + len > this.length) { - throw new IOException("Current position " + this.position.get() + " plus " + + throw new IOException("Offset " + offset + " plus " + len + " exceeds object length " + this.length + "."); } GetObjectRequest request = GetObjectRequest.builder().bucket(path.bucket) - .key(path.key).range(toRange(position.get(), len)).build(); + .key(path.key).range(toRange(offset, len)).build(); CompletableFuture> future = client.getObject(request, AsyncResponseTransformer.toBytes()); try @@ -163,7 +164,6 @@ public CompletableFuture readAsync(int len) throws IOException if (resp != null) { futureBuffer.complete(ByteBuffer.wrap(resp.asByteArray())); - this.position.addAndGet(len); } else { diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java index a1cc8e23a..63d0e177a 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3.java @@ -27,8 +27,6 @@ import io.pixelsdb.pixels.common.utils.EtcdUtil; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient; -import software.amazon.awssdk.http.nio.netty.SdkEventLoopGroup; import software.amazon.awssdk.services.s3.S3AsyncClient; import software.amazon.awssdk.services.s3.S3AsyncClientBuilder; import software.amazon.awssdk.services.s3.S3Client; @@ -69,10 +67,10 @@ public class S3 implements Storage { InitId(S3_ID_KEY); - S3AsyncClientBuilder builder = S3AsyncClient.builder() - .httpClientBuilder(NettyNioAsyncHttpClient.builder() - .eventLoopGroup(SdkEventLoopGroup.builder().numberOfThreads(10).build()) - .maxConcurrency(100).maxPendingConnectionAcquires(10_000)); + S3AsyncClientBuilder builder = S3AsyncClient.builder(); + //.httpClientBuilder(NettyNioAsyncHttpClient.builder() + // .eventLoopGroup(SdkEventLoopGroup.builder().numberOfThreads(20).build()) + // .maxConcurrency(100).maxPendingConnectionAcquires(10_000)); s3Async = builder.build(); s3 = S3Client.builder().build(); diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3InputStream.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3InputStream.java index d832cc2ac..3f8536b08 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3InputStream.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/S3InputStream.java @@ -211,7 +211,7 @@ protected int populateBuffer() throws IOException private String toRange(long start, int length) { StringBuilder builder = new StringBuilder("bytes="); - builder.append(start).append('-').append(start+length); + builder.append(start).append('-').append(start+length-1); return builder.toString(); } diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java index 33a369131..6e1ea527f 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/NoopScheduler.java @@ -43,11 +43,12 @@ public class NoopScheduler implements Scheduler private static Logger logger = LogManager.getLogger(NoopScheduler.class); @Override - public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch batch) throws IOException + public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch batch, + List actionFutures) throws IOException { if (batch.size() <= 0) { - return batch.completeAll(); + return batch.completeAll(actionFutures); } List> futures = batch.getFutures(); List requests = batch.getRequests(); @@ -57,8 +58,7 @@ public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch { CompletableFuture future = futures.get(i); Request request = requests.get(i); - reader.seek(request.start); - reader.readAsync(request.length).whenComplete((resp, err) -> + reader.readAsync(request.start, request.length).whenComplete((resp, err) -> { if (resp != null) { @@ -83,6 +83,6 @@ public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch } } - return batch.completeAll(); + return batch.completeAll(actionFutures); } } diff --git a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java index 79cd68bb3..34f8ac6e6 100644 --- a/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java +++ b/pixels-common/src/main/java/io/pixelsdb/pixels/common/physical/impl/scheduler/SortMergeScheduler.java @@ -51,11 +51,12 @@ public class SortMergeScheduler implements Scheduler } @Override - public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch batch) throws IOException + public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch batch, + List actionFutures) throws IOException { if (batch.size() <= 0) { - return batch.completeAll(); + return batch.completeAll(actionFutures); } List> futures = batch.getFutures(); List requests = batch.getRequests(); @@ -84,8 +85,7 @@ public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch { for (MergedRequest merged : mergedRequests) { - reader.seek(merged.getStart()); - reader.readAsync(merged.getLength()).whenComplete((resp, err) -> + reader.readAsync(merged.getStart(), merged.getLength()).whenComplete((resp, err) -> { if (resp != null) { @@ -110,7 +110,7 @@ public CompletableFuture executeBatch(PhysicalReader reader, RequestBatch } } - return batch.completeAll(); + return batch.completeAll(actionFutures); } class RequestFuture implements Comparable @@ -191,8 +191,7 @@ public int getSize() } /** - * When the data has been read, complete all the - * futures, + * When the data has been read, complete all the futures. * @param buffer the data that has been read. */ public void complete(ByteBuffer buffer) diff --git a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java index f876012e3..f8e86bb45 100644 --- a/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java +++ b/pixels-common/src/test/java/io/pixelsdb/pixels/common/TestS3.java @@ -29,6 +29,7 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; /** * Created at: 9/8/21 @@ -114,7 +115,7 @@ public void testGetPaths() throws IOException public void testS3Reader() throws IOException { PhysicalReader reader = PhysicalReaderUtil.newPhysicalReader(Storage.Scheme.s3, "pixels-01/object-4"); - CompletableFuture future = reader.readAsync(8); + CompletableFuture future = reader.readAsync(0, 8); future.whenComplete((resp, err) -> { if (resp != null) @@ -129,4 +130,38 @@ public void testS3Reader() throws IOException future.join(); reader.close(); } + + @Test + public void testEnclosure() throws InterruptedException + { + //byte[] bytes = new byte[100]; + //PixelsProto.RowGroupFooter footer = PixelsProto.RowGroupFooter.parseFrom(ByteBuffer.wrap(null)); + //System.out.println(footer); + AtomicInteger integer = new AtomicInteger(0); + for (int i = 0; i < 3; ++i) + { + String a = "" + i; + int fi = i; + Thread thread = new Thread(() -> + { + try + { + Thread.sleep(500); + System.out.println("in: " + integer.get()); + System.out.println("in: " + a + ", " + fi); + integer.addAndGet(10); + System.out.println("in: " + integer.get()); + System.out.println("in: " + a + ", " + fi); + } catch (InterruptedException e) + { + e.printStackTrace(); + } + }); + thread.start(); + } + integer.set(5); + System.out.println(integer.get()); + Thread.sleep(1000); + System.out.println(integer.get()); + } } diff --git a/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java b/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java index 4ac6c4274..1e3d4c746 100644 --- a/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java +++ b/pixels-core/src/main/java/io/pixelsdb/pixels/core/reader/PixelsRecordReaderImpl.java @@ -27,7 +27,9 @@ import io.pixelsdb.pixels.common.physical.Scheduler; import io.pixelsdb.pixels.common.physical.SchedulerFactory; import io.pixelsdb.pixels.common.utils.ConfigFactory; -import io.pixelsdb.pixels.core.*; +import io.pixelsdb.pixels.core.PixelsFooterCache; +import io.pixelsdb.pixels.core.PixelsProto; +import io.pixelsdb.pixels.core.TypeDescription; import io.pixelsdb.pixels.core.predicate.PixelsPredicate; import io.pixelsdb.pixels.core.stats.ColumnStats; import io.pixelsdb.pixels.core.stats.StatsRecorder; @@ -39,7 +41,8 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.*; -import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReferenceArray; /** * @author guodong @@ -62,7 +65,7 @@ public class PixelsRecordReaderImpl private final boolean enableCache; private final List cacheOrder; private final PixelsCacheReader cacheReader; - private final PixelsFooterCache pixelsFooterCache; + private volatile PixelsFooterCache pixelsFooterCache; private final String fileName; private TypeDescription fileSchema; @@ -80,8 +83,9 @@ public class PixelsRecordReaderImpl private int curRGIdx = 0; // index of current reading row group in targetRGs private int curRowInRG = 0; // starting index of values to read by reader in current row group - private volatile PixelsProto.RowGroupFooter[] rowGroupFooters; - private volatile ByteBuffer[] chunkBuffers; // buffers of each chunk in this file, arranged by chunk's row group id and column id + private volatile AtomicReferenceArray rowGroupFooters; + // buffers of each chunk in this file, arranged by chunk's row group id and column id + private volatile AtomicReferenceArray chunkBuffers; private ColumnReader[] readers; // column readers for each target columns private long diskReadBytes = 0L; @@ -398,14 +402,16 @@ else if (predicate.matchesNone()) targetRGNum = targetRGIdx; // read row group footers - rowGroupFooters = - new PixelsProto.RowGroupFooter[targetRGNum]; + rowGroupFooters = new AtomicReferenceArray<>(targetRGNum); /** * Issue #114: * Use request batch and read scheduler to execute the read requests. + * + * Here, we create an empty batch as footer cache is very likely to be hit in + * the subsequent queries on the same table. */ - Scheduler.RequestBatch requestBatch = new Scheduler.RequestBatch(targetRGNum); - AtomicReference lastErr = new AtomicReference<>(null); + Scheduler.RequestBatch requestBatch = new Scheduler.RequestBatch(); + List actionFutures = new ArrayList<>(); for (int i = 0; i < targetRGNum; i++) { int rgId = targetRGs[i]; @@ -419,40 +425,37 @@ else if (predicate.matchesNone()) long footerOffset = rowGroupInformation.getFooterOffset(); long footerLength = rowGroupInformation.getFooterLength(); int fi = i; - requestBatch.add(footerOffset, (int) footerLength).whenComplete((resp, err) -> + actionFutures.add(requestBatch.add(footerOffset, (int) footerLength).whenComplete((resp, err) -> { if (resp != null) { try { - rowGroupFooters[fi] = PixelsProto.RowGroupFooter.parseFrom(resp); - pixelsFooterCache.putRGFooter(rgCacheId, rowGroupFooters[fi]); + PixelsProto.RowGroupFooter parsed = PixelsProto.RowGroupFooter.parseFrom(resp); + rowGroupFooters.set(fi, parsed); + pixelsFooterCache.putRGFooter(rgCacheId, parsed); } catch (InvalidProtocolBufferException e) { - logger.error("Failed to parse row group footer from byte buffer.", e); - lastErr.set(e); + throw new RuntimeException("Failed to parse row group footer from byte buffer.", e); } } - else - { - logger.error("Failed to read row group footer.", err); - lastErr.set(err); - } - }); + })); } // cache hit else { - rowGroupFooters[i] = rowGroupFooter; + rowGroupFooters.set(i, rowGroupFooter); } } - if (lastErr.get() != null) + Scheduler scheduler = SchedulerFactory.Instance().getScheduler(); + try + { + scheduler.executeBatch(physicalReader, requestBatch, actionFutures).get(); + } catch (Exception e) { throw new IOException("Failed to read row group footers, " + - "only the last error is thrown, check the logs for more information.", lastErr.get()); + "only the last error is thrown, check the logs for more information.", e); } - Scheduler scheduler = SchedulerFactory.Instance().getScheduler(); - scheduler.executeBatch(physicalReader, requestBatch).join(); return true; } @@ -512,7 +515,7 @@ private boolean read() throws IOException } // read chunk offset and length of each target column chunks - this.chunkBuffers = new ByteBuffer[targetRGNum * includedColumns.length]; + this.chunkBuffers = new AtomicReferenceArray<>(targetRGNum * includedColumns.length); List diskChunks = new ArrayList<>(targetRGNum * targetColumns.length); // read cached data which are in need if (enableCache) @@ -556,7 +559,7 @@ private boolean read() throws IOException else { PixelsProto.RowGroupIndex rowGroupIndex = - rowGroupFooters[rgIdx].getRowGroupIndexEntry(); + rowGroupFooters.get(rgIdx).getRowGroupIndexEntry(); PixelsProto.ColumnChunkIndex chunkIndex = rowGroupIndex.getColumnChunkIndexEntries(colId); /** @@ -584,7 +587,7 @@ private boolean read() throws IOException memoryUsage += columnletId.direct ? 0 : columnlet.capacity(); // long getEnd = System.nanoTime(); // logger.debug("[cache get]: " + columnlet.length + "," + (getEnd - getBegin)); - chunkBuffers[(rgId - RGStart) * includedColumns.length + colId] = columnlet; + chunkBuffers.set((rgId - RGStart) * includedColumns.length + colId, columnlet); if (columnlet == null || columnlet.capacity() == 0) { /** @@ -595,7 +598,7 @@ private boolean read() throws IOException */ int rgIdx = rgId - RGStart; PixelsProto.RowGroupIndex rowGroupIndex = - rowGroupFooters[rgIdx].getRowGroupIndexEntry(); + rowGroupFooters.get(rgIdx).getRowGroupIndexEntry(); PixelsProto.ColumnChunkIndex chunkIndex = rowGroupIndex.getColumnChunkIndexEntries(colId); ChunkId diskChunk = new ChunkId(rgIdx, colId, chunkIndex.getChunkOffset(), @@ -641,7 +644,7 @@ private boolean read() throws IOException for (int rgIdx = 0; rgIdx < targetRGNum; rgIdx++) { PixelsProto.RowGroupIndex rowGroupIndex = - rowGroupFooters[rgIdx].getRowGroupIndexEntry(); + rowGroupFooters.get(rgIdx).getRowGroupIndexEntry(); for (int colId : targetColumns) { PixelsProto.ColumnChunkIndex chunkIndex = @@ -671,8 +674,8 @@ private boolean read() throws IOException * ChunkSeq) are moved into the sortmerge scheduler, which can be enabled * by setting read.request.scheduler=sortmerge. */ - Scheduler.RequestBatch batch = new Scheduler.RequestBatch(diskChunks.size()); - AtomicReference lastErr = new AtomicReference<>(null); + Scheduler.RequestBatch requestBatch = new Scheduler.RequestBatch(diskChunks.size()); + List actionFutures = new ArrayList<>(diskChunks.size()); for (ChunkId chunk : diskChunks) { /** @@ -683,6 +686,7 @@ private boolean read() throws IOException * used to calculate the index of chunkBuffers. */ int rgIdx = chunk.rowGroupId; + int numCols = includedColumns.length; int colId = chunk.columnId; /** * Issue #114: @@ -701,30 +705,27 @@ private boolean read() throws IOException * readCost.setMs(readTimeMs); * readPerfMetrics.addSeqRead(readCost); */ - batch.add(new Scheduler.Request(chunk.offset, (int)chunk.length)) + actionFutures.add(requestBatch.add(new Scheduler.Request(chunk.offset, (int)chunk.length)) .whenComplete((resp, err) -> { if (resp != null) { - chunkBuffers[rgIdx * includedColumns.length + colId] = resp; - } - else - { - logger.error("Failed to read chunks block into buffers.", err); - lastErr.set(err); + chunkBuffers.set(rgIdx * numCols + colId, resp); } - }); + })); // don't update statistics in whenComplete as it may be executed in other threads. diskReadBytes += chunk.length; memoryUsage += chunk.length; } Scheduler scheduler = SchedulerFactory.Instance().getScheduler(); - scheduler.executeBatch(physicalReader, batch).join(); - if (lastErr.get() != null) + try + { + scheduler.executeBatch(physicalReader, requestBatch, actionFutures).get(); + } catch (Exception e) { throw new IOException("Failed to read chunks block into buffers, " + - "only the last error is thrown, check the logs for more information.", lastErr.get()); + "only the last error is thrown, check the logs for more information.", e); } } @@ -903,8 +904,7 @@ public VectorizedRowBatch readBatch(int batchSize) { if (!columnVectors[i].duplicated) { - PixelsProto.RowGroupFooter rowGroupFooter = - rowGroupFooters[curRGIdx]; + PixelsProto.RowGroupFooter rowGroupFooter = rowGroupFooters.get(curRGIdx); PixelsProto.ColumnEncoding encoding = rowGroupFooter.getRowGroupEncoding() .getColumnChunkEncodings(resultColumns[i]); int index = curRGIdx * includedColumns.length + resultColumns[i]; @@ -912,7 +912,7 @@ public VectorizedRowBatch readBatch(int batchSize) .getColumnChunkIndexEntries( resultColumns[i]); // TODO: read chunk buffer lazily when a column block is read by PixelsPageSource. - readers[i].read(chunkBuffers[index], encoding, curRowInRG, curBatchSize, + readers[i].read(chunkBuffers.get(index), encoding, curRowInRG, curBatchSize, postScript.getPixelStride(), resultRowBatch.size, columnVectors[i], chunkIndex); } } @@ -1043,9 +1043,9 @@ public void close() throws IOException // release chunk buffer if (chunkBuffers != null) { - for (int i = 0; i < chunkBuffers.length; i++) + for (int i = 0; i < chunkBuffers.length(); i++) { - chunkBuffers[i] = null; + chunkBuffers.set(i, null); } } if (readers != null) @@ -1075,9 +1075,9 @@ public void close() throws IOException } if (rowGroupFooters != null) { - for (int i = 0; i < rowGroupFooters.length; ++i) + for (int i = 0; i < rowGroupFooters.length(); ++i) { - rowGroupFooters[i] = null; + rowGroupFooters.set(i, null); } } // write out read performance metrics