Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -982,7 +982,10 @@ public CompletableFuture<LedgerEntries> batchReadAsync(long startEntry, int maxC
public CompletableFuture<LedgerEntries> batchReadUnconfirmedAsync(long startEntry, int maxCount, long maxSize) {
// Little sanity check
if (startEntry < 0 || maxCount < 0 || maxSize < 0) {
LOG.error("IncorrectParameterException on ledgerId:{} firstEntry:{} when batch read", ledgerId, startEntry);
log.error()
.attr("ledgerId", ledgerId)
.attr("startEntry", startEntry)
.log("IncorrectParameterException when batch read");
return FutureUtils.exception(new BKIncorrectParameterException());
}

Expand Down
1 change: 1 addition & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -220,6 +220,7 @@
<nar-maven-plugin.version>3.10.1</nar-maven-plugin.version>
<os-maven-plugin.version>1.7.1</os-maven-plugin.version>
<protobuf-maven-plugin.version>0.6.1</protobuf-maven-plugin.version>
<lightproto-maven-plugin.version>0.7.0</lightproto-maven-plugin.version>
<puppycrawl.checkstyle.version>9.3</puppycrawl.checkstyle.version>
<spotbugs-maven-plugin.version>4.7.3.2</spotbugs-maven-plugin.version>
<build-helper-maven-plugin.version>3.6.0</build-helper-maven-plugin.version>
Expand Down
27 changes: 27 additions & 0 deletions stream/proto/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,15 @@
</extension>
</extensions>
<plugins>
<plugin>
<groupId>org.apache.rat</groupId>
<artifactId>apache-rat-plugin</artifactId>
<configuration>
<excludes combine.children="append">
<exclude>target/generated-sources/lightproto/**</exclude>
</excludes>
</configuration>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
Expand Down Expand Up @@ -96,6 +105,24 @@
</execution>
</executions>
</plugin>
<plugin>
<groupId>io.streamnative.lightproto</groupId>
<artifactId>lightproto-maven-plugin</artifactId>
<version>${lightproto-maven-plugin.version}</version>
<configuration>
<sources>
<source>${project.basedir}/src/main/proto-lightproto/cluster.proto</source>
</sources>
<targetSourcesSubDir>generated-sources/lightproto/java</targetSourcesSubDir>
</configuration>
<executions>
<execution>
<goals>
<goal>generate</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
Expand Down
30 changes: 16 additions & 14 deletions stream/statelib/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -61,8 +61,8 @@
<artifactId>rocksdbjni</artifactId>
</dependency>
<dependency>
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
<groupId>io.netty</groupId>
<artifactId>netty-buffer</artifactId>
</dependency>
<dependency>
<groupId>org.apache.distributedlog</groupId>
Expand Down Expand Up @@ -104,25 +104,27 @@
</dependency>
</dependencies>
<build>
<extensions>
<extension>
<groupId>kr.motd.maven</groupId>
<artifactId>os-maven-plugin</artifactId>
<version>${os-maven-plugin.version}</version>
</extension>
</extensions>
<plugins>
<plugin>
<groupId>org.xolstice.maven.plugins</groupId>
<artifactId>protobuf-maven-plugin</artifactId>
<version>${protobuf-maven-plugin.version}</version>
<groupId>org.apache.rat</groupId>
<artifactId>apache-rat-plugin</artifactId>
<configuration>
<protocArtifact>com.google.protobuf:protoc:${protoc.version}:exe:${os.detected.classifier}</protocArtifact>
<excludes combine.children="append">
<exclude>target/generated-sources/lightproto/**</exclude>
</excludes>
</configuration>
</plugin>
<plugin>
<groupId>io.streamnative.lightproto</groupId>
<artifactId>lightproto-maven-plugin</artifactId>
<version>${lightproto-maven-plugin.version}</version>
<configuration>
<targetSourcesSubDir>generated-sources/lightproto/java</targetSourcesSubDir>
</configuration>
<executions>
<execution>
<goals>
<goal>compile</goal>
<goal>generate</goal>
</goals>
</execution>
</executions>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@

import static org.apache.bookkeeper.statelib.impl.kv.KVUtils.newCommand;

import com.google.protobuf.InvalidProtocolBufferException;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufUtil;
import lombok.AccessLevel;
Expand Down Expand Up @@ -48,8 +47,8 @@ public void applyCommand(long revision, ByteBuf cmdBuf, RocksdbKVStore<byte[], b
Command command;
try {
command = newCommand(cmdBuf);
} catch (InvalidProtocolBufferException e) {
log.error().attr("buffer", cmdBuf).attr("txid", revision)
} catch (RuntimeException e) {
log.error().exception(e).attr("buffer", cmdBuf).attr("txid", revision)
.log("Invalid kv command found");
// TODO: better to handle this
return;
Expand All @@ -74,25 +73,25 @@ public void applyCommand(long revision, ByteBuf cmdBuf, RocksdbKVStore<byte[], b

private void applyPutCommand(long revision, Command command, RocksdbKVStore<byte[], byte[]> store) {
PutRequest putReq = command.getPutReq();
byte[] keyBytes = putReq.getKey().toByteArray();
byte[] valBytes = putReq.getValue().toByteArray();
byte[] keyBytes = putReq.getKey();
byte[] valBytes = putReq.getValue();
@Cleanup("release") ByteBuf serializedValBuf = KVUtils.serialize(valBytes, revision);
byte[] serializedValBytes = ByteBufUtil.getBytes(serializedValBuf);
store.put(keyBytes, serializedValBytes, revision);
}

private void applyPutIfAbsentCommand(long revision, Command command, RocksdbKVStore<byte[], byte[]> store) {
PutRequest putReq = command.getPutReq();
byte[] keyBytes = putReq.getKey().toByteArray();
byte[] valBytes = putReq.getValue().toByteArray();
byte[] keyBytes = putReq.getKey();
byte[] valBytes = putReq.getValue();
@Cleanup("release") ByteBuf serializedValBuf = KVUtils.serialize(valBytes, revision);
byte[] serializedValBytes = ByteBufUtil.getBytes(serializedValBuf);
store.putIfAbsent(keyBytes, serializedValBytes, revision);
}

private void applyDeleteCommand(long revision, Command command, RocksdbKVStore<byte[], byte[]> store) {
DeleteRequest delReq = command.getDelReq();
byte[] keyBytes = delReq.getKey().toByteArray();
byte[] keyBytes = delReq.getKey();
store.delete(keyBytes, revision);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,22 +18,14 @@
*/
package org.apache.bookkeeper.statelib.impl.kv;

import com.google.protobuf.InvalidProtocolBufferException;
import com.google.protobuf.UnsafeByteOperations;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.ByteBufOutputStream;
import io.netty.buffer.PooledByteBufAllocator;
import io.netty.util.ReferenceCountUtil;
import java.io.IOException;
import lombok.AccessLevel;
import lombok.CustomLog;
import lombok.NoArgsConstructor;
import org.apache.bookkeeper.common.coder.Coder;
import org.apache.bookkeeper.proto.statestore.kv.Command;
import org.apache.bookkeeper.proto.statestore.kv.DeleteRequest;
import org.apache.bookkeeper.proto.statestore.kv.NopRequest;
import org.apache.bookkeeper.proto.statestore.kv.PutIfAbsentRequest;
import org.apache.bookkeeper.proto.statestore.kv.PutRequest;

/**
* Utils for kv stores.
Expand All @@ -42,9 +34,13 @@
@NoArgsConstructor(access = AccessLevel.PRIVATE)
final class KVUtils {

static final Command NOP_CMD = Command.newBuilder()
.setNopReq(NopRequest.newBuilder().build())
.build();
static final Command NOP_CMD = newNopCommand();

private static Command newNopCommand() {
Command cmd = new Command();
cmd.setNopReq();
return cmd;
}

static ByteBuf serialize(ByteBuf valBuf, long revision) {
int serializedSize = valBuf.readableBytes() + Long.BYTES;
Expand All @@ -68,41 +64,21 @@ static <V> V deserialize(Coder<V> valCoder,
return valCoder.decode(valBuf);
}

static Command newCommand(ByteBuf cmdBuf) throws InvalidProtocolBufferException {
return Command.parseFrom(cmdBuf.nioBuffer());
static Command newCommand(ByteBuf cmdBuf) {
Command cmd = new Command();
cmd.parseFrom(cmdBuf, cmdBuf.readableBytes());
return cmd;
}

static ByteBuf newCommandBuf(Command cmd) throws IOException {
static ByteBuf newCommandBuf(Command cmd) {
ByteBuf buf = PooledByteBufAllocator.DEFAULT.buffer(cmd.getSerializedSize());
try {
cmd.writeTo(new ByteBufOutputStream(buf));
} catch (IOException e) {
cmd.writeTo(buf);
} catch (RuntimeException e) {
ReferenceCountUtil.release(buf);
throw e;
}
return buf;
}

static PutRequest newPutRequest(byte[] keyBytes,
byte[] valBytes) {
return PutRequest.newBuilder()
.setKey(UnsafeByteOperations.unsafeWrap(keyBytes))
.setValue(UnsafeByteOperations.unsafeWrap(valBytes))
.build();
}

static PutIfAbsentRequest newPutIfAbsentRequest(byte[] keyBytes,
byte[] valBytes) {
return PutIfAbsentRequest.newBuilder()
.setKey(UnsafeByteOperations.unsafeWrap(keyBytes))
.setValue(UnsafeByteOperations.unsafeWrap(valBytes))
.build();
}

static DeleteRequest newDeleteRequest(byte[] keyBytes) {
return DeleteRequest.newBuilder()
.setKey(UnsafeByteOperations.unsafeWrap(keyBytes))
.build();
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import io.netty.buffer.ByteBufUtil;
import io.netty.buffer.Unpooled;
import io.netty.util.ReferenceCountUtil;
import java.io.IOException;
import java.util.concurrent.CompletableFuture;
import java.util.function.Supplier;
import org.apache.bookkeeper.common.coder.Coder;
Expand Down Expand Up @@ -91,9 +90,8 @@ public CompletableFuture<Void> put(K key, V value) {
byte[] keyBytes = keyCoder.encode(key);
byte[] valBytes = valCoder.encode(value);

Command command = Command.newBuilder()
.setPutReq(KVUtils.newPutRequest(keyBytes, valBytes))
.build();
Command command = new Command();
command.setPutReq().setKey(keyBytes).setValue(valBytes);
return writeCommandReturnTxId(command).thenApplyAsync((revision) -> {
ByteBuf serializedBuf = KVUtils.serialize(valBytes, revision);
try {
Expand All @@ -112,9 +110,8 @@ public CompletableFuture<V> putIfAbsent(K key, V value) {
byte[] keyBytes = keyCoder.encode(key);
byte[] valBytes = valCoder.encode(value);

Command command = Command.newBuilder()
.setPutIfAbsentReq(KVUtils.newPutIfAbsentRequest(keyBytes, valBytes))
.build();
Command command = new Command();
command.setPutIfAbsentReq().setKey(keyBytes).setValue(valBytes);
return writeCommandReturnTxId(command).thenApplyAsync((revision) -> {
ByteBuf serializedBuf = KVUtils.serialize(valBytes, revision);
try {
Expand All @@ -135,9 +132,8 @@ public CompletableFuture<V> putIfAbsent(K key, V value) {
public CompletableFuture<V> delete(K key) {
byte[] keyBytes = keyCoder.encode(key);

Command command = Command.newBuilder()
.setDelReq(KVUtils.newDeleteRequest(keyBytes))
.build();
Command command = new Command();
command.setDelReq().setKey(keyBytes);
return writeCommandReturnTxId(command).thenApplyAsync((revision) -> {
byte[] prevValue = localStore.delete(keyBytes, revision);
if (null == prevValue) {
Expand Down Expand Up @@ -171,7 +167,7 @@ private CompletableFuture<Long> writeCommandReturnTxId(Command command) {
try {
ByteBuf cmdBuf = KVUtils.newCommandBuf(command);
return writeCommandBufReturnTxId(cmdBuf);
} catch (IOException e) {
} catch (RuntimeException e) {
return FutureUtils.exception(e);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,10 +56,8 @@ synchronized int getNumWatchers() {
@Override
public synchronized boolean initializeCluster(int numStorageContainers,
Optional<String> segmentStorePath) {
this.metadata = ClusterMetadata.newBuilder()
.setNumStorageContainers(numStorageContainers)
.build();
this.assignmentData = ClusterAssignmentData.newBuilder().build();
this.metadata = new ClusterMetadata().setNumStorageContainers(numStorageContainers);
this.assignmentData = new ClusterAssignmentData();
return true;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import static org.apache.bookkeeper.stream.storage.StorageConstants.getStoragePath;
import static org.apache.bookkeeper.stream.storage.StorageConstants.getWritableServersPath;

import com.google.protobuf.InvalidProtocolBufferException;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
Expand Down Expand Up @@ -95,11 +94,8 @@ public void close() {

@Override
public boolean initializeCluster(int numStorageContainers, Optional<String> segmentStorePath) {
ClusterMetadata metadata = ClusterMetadata.newBuilder()
.setNumStorageContainers(numStorageContainers)
.build();
ClusterAssignmentData assignmentData = ClusterAssignmentData.newBuilder()
.build();
ClusterMetadata metadata = new ClusterMetadata().setNumStorageContainers(numStorageContainers);
ClusterAssignmentData assignmentData = new ClusterAssignmentData();
try {
// we are using dlog for the storage backend, so we need to initialize the dlog namespace
BKDLConfig dlogConfig = new BKDLConfig(
Expand Down Expand Up @@ -130,10 +126,16 @@ public boolean initializeCluster(int numStorageContainers, Optional<String> segm
public ClusterAssignmentData getClusterAssignmentData() {
try {
byte[] data = client.getData().forPath(zkClusterAssignmentPath);
return ClusterAssignmentData.parseFrom(data);
} catch (InvalidProtocolBufferException ie) {
throw new StorageRuntimeException("The cluster assignment data from zookeeper @"
+ zkClusterAssignmentPath + " is corrupted", ie);
ClusterAssignmentData assignmentData = new ClusterAssignmentData();
try {
assignmentData.parseFrom(data);
} catch (RuntimeException ie) {
throw new StorageRuntimeException("The cluster assignment data from zookeeper @"
+ zkClusterAssignmentPath + " is corrupted", ie);
}
return assignmentData;
} catch (StorageRuntimeException e) {
throw e;
} catch (Exception e) {
throw new StorageRuntimeException("Failed to fetch cluster assignment data from zookeeper @"
+ zkClusterAssignmentPath, e);
Expand Down Expand Up @@ -193,10 +195,16 @@ public void unwatchClusterAssignmentData(Consumer<Void> watcher) {
public ClusterMetadata getClusterMetadata() {
try {
byte[] data = client.getData().forPath(zkClusterMetadataPath);
return ClusterMetadata.parseFrom(data);
} catch (InvalidProtocolBufferException ie) {
throw new StorageRuntimeException("The cluster metadata from zookeeper @"
+ zkClusterMetadataPath + " is corrupted", ie);
ClusterMetadata metadata = new ClusterMetadata();
try {
metadata.parseFrom(data);
} catch (RuntimeException ie) {
throw new StorageRuntimeException("The cluster metadata from zookeeper @"
+ zkClusterMetadataPath + " is corrupted", ie);
}
return metadata;
} catch (StorageRuntimeException e) {
throw e;
} catch (Exception e) {
throw new StorageRuntimeException("Failed to fetch cluster metadata from zookeeper @"
+ zkClusterMetadataPath, e);
Expand Down
Loading
Loading