diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java index 644deb7be8c..3019ef04187 100644 --- a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/SimpleStorageClientImpl.java @@ -34,6 +34,7 @@ import org.apache.bookkeeper.api.kv.PTable; import org.apache.bookkeeper.api.kv.Table; import org.apache.bookkeeper.clients.config.StorageClientSettings; +import org.apache.bookkeeper.clients.grpc.RootRangeServiceFutureStub; import org.apache.bookkeeper.clients.impl.kv.ByteBufTableImpl; import org.apache.bookkeeper.clients.impl.kv.PByteBufSimpleTableImpl; import org.apache.bookkeeper.clients.utils.GrpcUtils; @@ -43,8 +44,6 @@ import org.apache.bookkeeper.common.util.SharedResourceManager.Resource; import org.apache.bookkeeper.stream.proto.StorageType; import org.apache.bookkeeper.stream.proto.StreamProperties; -import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc; -import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc.RootRangeServiceFutureStub; import org.apache.bookkeeper.stream.proto.storage.StatusCode; /** @@ -63,7 +62,7 @@ public SimpleStorageClientImpl(String namespaceName, super(settings); this.defaultNamespace = namespaceName; this.rootRangeService = GrpcUtils.configureGrpcStub( - RootRangeServiceGrpc.newFutureStub(channel), + RootRangeServiceFutureStub.newFutureStub(channel), Optional.empty()); } @@ -74,7 +73,7 @@ public SimpleStorageClientImpl(String namespaceName, super(settings, schedulerResource, channel, false); this.defaultNamespace = namespaceName; this.rootRangeService = GrpcUtils.configureGrpcStub( - RootRangeServiceGrpc.newFutureStub(channel), + RootRangeServiceFutureStub.newFutureStub(channel), Optional.empty()); } diff --git a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/SimpleStorageAdminClientImpl.java b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/SimpleStorageAdminClientImpl.java index 04072e1952f..9c2344f8c3d 100644 --- a/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/SimpleStorageAdminClientImpl.java +++ b/stream/clients/java/all/src/main/java/org/apache/bookkeeper/clients/admin/SimpleStorageAdminClientImpl.java @@ -34,6 +34,7 @@ import org.apache.bookkeeper.clients.SimpleClientBase; import org.apache.bookkeeper.clients.SimpleStorageClientImpl; import org.apache.bookkeeper.clients.config.StorageClientSettings; +import org.apache.bookkeeper.clients.grpc.RootRangeServiceFutureStub; import org.apache.bookkeeper.clients.utils.ClientResources; import org.apache.bookkeeper.clients.utils.GrpcUtils; import org.apache.bookkeeper.common.concurrent.FutureUtils; @@ -43,8 +44,6 @@ import org.apache.bookkeeper.stream.proto.NamespaceProperties; import org.apache.bookkeeper.stream.proto.StreamConfiguration; import org.apache.bookkeeper.stream.proto.StreamProperties; -import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc; -import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc.RootRangeServiceFutureStub; import org.apache.bookkeeper.stream.proto.storage.StatusCode; /** @@ -66,7 +65,7 @@ public SimpleStorageAdminClientImpl(StorageClientSettings settings, Resource schedulerResource) { super(settings, schedulerResource); this.rootRangeService = GrpcUtils.configureGrpcStub( - RootRangeServiceGrpc.newFutureStub(channel), + RootRangeServiceFutureStub.newFutureStub(channel), Optional.empty()); } diff --git a/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/StorageClientImplTest.java b/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/StorageClientImplTest.java index 72eaa870dbc..5e736f5a248 100644 --- a/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/StorageClientImplTest.java +++ b/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/StorageClientImplTest.java @@ -56,12 +56,31 @@ public class StorageClientImplTest extends GrpcClientTestBase { private static final String NAMESPACE = "test-namespace"; private static final String STREAM_NAME = "test-stream-name"; - private static final StreamProperties STREAM_PROPERTIES = StreamProperties.newBuilder() - .setStreamId(1234L) - .setStreamConf(DEFAULT_STREAM_CONF) - .setStreamName(STREAM_NAME) - .setStorageContainerId(16) - .build(); + private static final StreamProperties STREAM_PROPERTIES = newStreamProperties(); + + private static StreamProperties newStreamProperties() { + StreamProperties props = new StreamProperties() + .setStreamId(1234L) + .setStreamName(STREAM_NAME) + .setStorageContainerId(16); + props.setStreamConf().copyFrom(DEFAULT_STREAM_CONF); + return props; + } + + private static StreamProperties newStreamPropsForTable(String name) { + StreamProperties props = new StreamProperties().copyFrom(STREAM_PROPERTIES); + props.setStreamName(name); + StreamConfiguration conf = props.setStreamConf(); + conf.copyFrom(DEFAULT_STREAM_CONF).setStorageType(StorageType.TABLE); + return props; + } + + private static StreamProperties newStreamPropsAsTable() { + StreamProperties props = new StreamProperties().copyFrom(STREAM_PROPERTIES); + StreamConfiguration conf = props.setStreamConf(); + conf.copyFrom(DEFAULT_STREAM_CONF).setStorageType(StorageType.TABLE); + return props; + } private StorageClientImpl client; @@ -82,11 +101,7 @@ protected void doTeardown() { @SuppressWarnings("unchecked") @Test public void testOpenPTable() throws Exception { - StreamProperties streamProps = StreamProperties.newBuilder(STREAM_PROPERTIES) - .setStreamConf(StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) - .setStorageType(StorageType.TABLE) - .build()) - .build(); + StreamProperties streamProps = newStreamPropsAsTable(); when(client.getStreamProperties(anyString(), anyString())) .thenReturn(FutureUtils.value(streamProps)); @@ -105,21 +120,11 @@ public void testOpenPTable() throws Exception { @SuppressWarnings("unchecked") @Test public void testOpenPTableDifferentNamespace() throws Exception { - StreamProperties tableProps1 = StreamProperties.newBuilder(STREAM_PROPERTIES) - .setStreamName("table1") - .setStreamConf(StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) - .setStorageType(StorageType.TABLE) - .build()) - .build(); + StreamProperties tableProps1 = newStreamPropsForTable("table1"); when(client.getStreamProperties(eq(NAMESPACE), eq("table1"))) .thenReturn(FutureUtils.value(tableProps1)); - StreamProperties tableProps2 = StreamProperties.newBuilder(STREAM_PROPERTIES) - .setStreamName("table2") - .setStreamConf(StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) - .setStorageType(StorageType.TABLE) - .build()) - .build(); + StreamProperties tableProps2 = newStreamPropsForTable("table2"); when(client.getStreamProperties(eq(NAMESPACE), eq("table2"))) .thenReturn(FutureUtils.value(tableProps2)); @@ -146,11 +151,7 @@ public void testOpenPTableDifferentNamespace() throws Exception { @SuppressWarnings("unchecked") @Test public void testOpenTable() throws Exception { - StreamProperties streamProps = StreamProperties.newBuilder(STREAM_PROPERTIES) - .setStreamConf(StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) - .setStorageType(StorageType.TABLE) - .build()) - .build(); + StreamProperties streamProps = newStreamPropsAsTable(); when(client.getStreamProperties(anyString(), anyString())) .thenReturn(FutureUtils.value(streamProps)); @@ -171,21 +172,11 @@ public void testOpenTable() throws Exception { @SuppressWarnings("unchecked") @Test public void testOpenTableWithDifferentNamespace() throws Exception { - StreamProperties tableProps1 = StreamProperties.newBuilder(STREAM_PROPERTIES) - .setStreamName("table1") - .setStreamConf(StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) - .setStorageType(StorageType.TABLE) - .build()) - .build(); + StreamProperties tableProps1 = newStreamPropsForTable("table1"); when(client.getStreamProperties(eq(NAMESPACE), eq("table1"))) .thenReturn(FutureUtils.value(tableProps1)); - StreamProperties tableProps2 = StreamProperties.newBuilder(STREAM_PROPERTIES) - .setStreamName("table2") - .setStreamConf(StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) - .setStorageType(StorageType.TABLE) - .build()) - .build(); + StreamProperties tableProps2 = newStreamPropsForTable("table2"); when(client.getStreamProperties(eq(NAMESPACE), eq("table2"))) .thenReturn(FutureUtils.value(tableProps2)); @@ -216,11 +207,8 @@ public void testOpenTableWithDifferentNamespace() throws Exception { @SuppressWarnings("unchecked") @Test public void testOpenPTableIllegalOp() throws Exception { - StreamProperties streamProps = StreamProperties.newBuilder(STREAM_PROPERTIES) - .setStreamConf(StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) - .setStorageType(StorageType.STREAM) - .build()) - .build(); + StreamProperties streamProps = new StreamProperties().copyFrom(STREAM_PROPERTIES); + streamProps.setStreamConf().copyFrom(DEFAULT_STREAM_CONF).setStorageType(StorageType.STREAM); when(client.getStreamProperties(anyString(), anyString())) .thenReturn(FutureUtils.value(streamProps)); diff --git a/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/admin/TestStorageAdminClientImpl.java b/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/admin/TestStorageAdminClientImpl.java index d9a791a5efb..0f724dedbc1 100644 --- a/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/admin/TestStorageAdminClientImpl.java +++ b/stream/clients/java/all/src/test/java/org/apache/bookkeeper/clients/admin/TestStorageAdminClientImpl.java @@ -43,20 +43,32 @@ */ public class TestStorageAdminClientImpl { - private static final NamespaceConfiguration colConf = NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build(); - private static final NamespaceProperties colProps = NamespaceProperties.newBuilder() - .setNamespaceId(System.currentTimeMillis()) - .setNamespaceName("namespace") - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build(); - private static final StreamProperties streamProps = StreamProperties.newBuilder() - .setStreamId(System.currentTimeMillis()) - .setStorageContainerId(System.currentTimeMillis()) - .setStreamName("stream_" + System.currentTimeMillis()) - .setStreamConf(DEFAULT_STREAM_CONF) - .build(); + private static final NamespaceConfiguration colConf = newColConf(); + private static final NamespaceProperties colProps = newColProps(); + private static final StreamProperties streamProps = newStreamProps(); + + private static NamespaceConfiguration newColConf() { + NamespaceConfiguration c = new NamespaceConfiguration(); + c.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); + return c; + } + + private static NamespaceProperties newColProps() { + NamespaceProperties p = new NamespaceProperties() + .setNamespaceId(System.currentTimeMillis()) + .setNamespaceName("namespace"); + p.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); + return p; + } + + private static StreamProperties newStreamProps() { + StreamProperties p = new StreamProperties() + .setStreamId(System.currentTimeMillis()) + .setStorageContainerId(System.currentTimeMillis()) + .setStreamName("stream_" + System.currentTimeMillis()); + p.setStreamConf().copyFrom(DEFAULT_STREAM_CONF); + return p; + } @Rule public TestName testName = new TestName(); diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/MetaRangeServiceFutureStub.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/MetaRangeServiceFutureStub.java new file mode 100644 index 00000000000..c4477deff8b --- /dev/null +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/MetaRangeServiceFutureStub.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.bookkeeper.clients.grpc; + +import com.google.common.util.concurrent.ListenableFuture; +import io.grpc.CallOptions; +import io.grpc.Channel; +import io.grpc.stub.AbstractStub; +import io.grpc.stub.ClientCalls; +import org.apache.bookkeeper.stream.proto.storage.GetActiveRangesRequest; +import org.apache.bookkeeper.stream.proto.storage.GetActiveRangesResponse; +import org.apache.bookkeeper.stream.proto.storage.MetaRangeServiceGrpc; + +/** + * ListenableFuture-returning stub for MetaRangeService. + * + *

The lightproto gRPC plugin does not generate FutureStub classes, so this adapter + * provides a drop-in replacement that uses {@link ClientCalls#futureUnaryCall} directly. + */ +public final class MetaRangeServiceFutureStub extends AbstractStub { + + public static MetaRangeServiceFutureStub newFutureStub(Channel channel) { + return new MetaRangeServiceFutureStub(channel, CallOptions.DEFAULT); + } + + private MetaRangeServiceFutureStub(Channel channel, CallOptions callOptions) { + super(channel, callOptions); + } + + @Override + protected MetaRangeServiceFutureStub build(Channel channel, CallOptions callOptions) { + return new MetaRangeServiceFutureStub(channel, callOptions); + } + + public ListenableFuture getActiveRanges(GetActiveRangesRequest request) { + return ClientCalls.futureUnaryCall( + getChannel().newCall(MetaRangeServiceGrpc.getGetActiveRangesMethod(), getCallOptions()), + request); + } +} diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/RootRangeServiceFutureStub.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/RootRangeServiceFutureStub.java new file mode 100644 index 00000000000..9eda78f140c --- /dev/null +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/RootRangeServiceFutureStub.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.bookkeeper.clients.grpc; + +import com.google.common.util.concurrent.ListenableFuture; +import io.grpc.CallOptions; +import io.grpc.Channel; +import io.grpc.stub.AbstractStub; +import io.grpc.stub.ClientCalls; +import org.apache.bookkeeper.stream.proto.storage.CreateNamespaceRequest; +import org.apache.bookkeeper.stream.proto.storage.CreateNamespaceResponse; +import org.apache.bookkeeper.stream.proto.storage.CreateStreamRequest; +import org.apache.bookkeeper.stream.proto.storage.CreateStreamResponse; +import org.apache.bookkeeper.stream.proto.storage.DeleteNamespaceRequest; +import org.apache.bookkeeper.stream.proto.storage.DeleteNamespaceResponse; +import org.apache.bookkeeper.stream.proto.storage.DeleteStreamRequest; +import org.apache.bookkeeper.stream.proto.storage.DeleteStreamResponse; +import org.apache.bookkeeper.stream.proto.storage.GetNamespaceRequest; +import org.apache.bookkeeper.stream.proto.storage.GetNamespaceResponse; +import org.apache.bookkeeper.stream.proto.storage.GetStreamRequest; +import org.apache.bookkeeper.stream.proto.storage.GetStreamResponse; +import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc; + +/** + * ListenableFuture-returning stub for RootRangeService. + * + *

The lightproto gRPC plugin does not generate FutureStub classes, so this adapter + * provides a drop-in replacement that uses {@link ClientCalls#futureUnaryCall} directly. + */ +public final class RootRangeServiceFutureStub extends AbstractStub { + + public static RootRangeServiceFutureStub newFutureStub(Channel channel) { + return new RootRangeServiceFutureStub(channel, CallOptions.DEFAULT); + } + + private RootRangeServiceFutureStub(Channel channel, CallOptions callOptions) { + super(channel, callOptions); + } + + @Override + protected RootRangeServiceFutureStub build(Channel channel, CallOptions callOptions) { + return new RootRangeServiceFutureStub(channel, callOptions); + } + + public ListenableFuture createNamespace(CreateNamespaceRequest request) { + return ClientCalls.futureUnaryCall( + getChannel().newCall(RootRangeServiceGrpc.getCreateNamespaceMethod(), getCallOptions()), + request); + } + + public ListenableFuture deleteNamespace(DeleteNamespaceRequest request) { + return ClientCalls.futureUnaryCall( + getChannel().newCall(RootRangeServiceGrpc.getDeleteNamespaceMethod(), getCallOptions()), + request); + } + + public ListenableFuture getNamespace(GetNamespaceRequest request) { + return ClientCalls.futureUnaryCall( + getChannel().newCall(RootRangeServiceGrpc.getGetNamespaceMethod(), getCallOptions()), + request); + } + + public ListenableFuture createStream(CreateStreamRequest request) { + return ClientCalls.futureUnaryCall( + getChannel().newCall(RootRangeServiceGrpc.getCreateStreamMethod(), getCallOptions()), + request); + } + + public ListenableFuture deleteStream(DeleteStreamRequest request) { + return ClientCalls.futureUnaryCall( + getChannel().newCall(RootRangeServiceGrpc.getDeleteStreamMethod(), getCallOptions()), + request); + } + + public ListenableFuture getStream(GetStreamRequest request) { + return ClientCalls.futureUnaryCall( + getChannel().newCall(RootRangeServiceGrpc.getGetStreamMethod(), getCallOptions()), + request); + } +} diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/StorageContainerServiceFutureStub.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/StorageContainerServiceFutureStub.java new file mode 100644 index 00000000000..4ccafc07fea --- /dev/null +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/StorageContainerServiceFutureStub.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.bookkeeper.clients.grpc; + +import com.google.common.util.concurrent.ListenableFuture; +import io.grpc.CallOptions; +import io.grpc.Channel; +import io.grpc.stub.AbstractStub; +import io.grpc.stub.ClientCalls; +import org.apache.bookkeeper.stream.proto.storage.GetStorageContainerEndpointRequest; +import org.apache.bookkeeper.stream.proto.storage.GetStorageContainerEndpointResponse; +import org.apache.bookkeeper.stream.proto.storage.StorageContainerServiceGrpc; + +/** + * ListenableFuture-returning stub for StorageContainerService. + * + *

The lightproto gRPC plugin does not generate FutureStub classes, so this adapter + * provides a drop-in replacement that uses {@link ClientCalls#futureUnaryCall} directly. + */ +public final class StorageContainerServiceFutureStub extends AbstractStub { + + public static StorageContainerServiceFutureStub newFutureStub(Channel channel) { + return new StorageContainerServiceFutureStub(channel, CallOptions.DEFAULT); + } + + private StorageContainerServiceFutureStub(Channel channel, CallOptions callOptions) { + super(channel, callOptions); + } + + @Override + protected StorageContainerServiceFutureStub build(Channel channel, CallOptions callOptions) { + return new StorageContainerServiceFutureStub(channel, callOptions); + } + + public ListenableFuture getStorageContainerEndpoint( + GetStorageContainerEndpointRequest request) { + return ClientCalls.futureUnaryCall( + getChannel().newCall(StorageContainerServiceGrpc.getGetStorageContainerEndpointMethod(), getCallOptions()), + request); + } +} diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/TableServiceFutureStub.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/TableServiceFutureStub.java new file mode 100644 index 00000000000..bf299d398a6 --- /dev/null +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/TableServiceFutureStub.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.bookkeeper.clients.grpc; + +import com.google.common.util.concurrent.ListenableFuture; +import io.grpc.CallOptions; +import io.grpc.Channel; +import io.grpc.stub.AbstractStub; +import io.grpc.stub.ClientCalls; +import org.apache.bookkeeper.stream.proto.kv.rpc.DeleteRangeRequest; +import org.apache.bookkeeper.stream.proto.kv.rpc.DeleteRangeResponse; +import org.apache.bookkeeper.stream.proto.kv.rpc.IncrementRequest; +import org.apache.bookkeeper.stream.proto.kv.rpc.IncrementResponse; +import org.apache.bookkeeper.stream.proto.kv.rpc.PutRequest; +import org.apache.bookkeeper.stream.proto.kv.rpc.PutResponse; +import org.apache.bookkeeper.stream.proto.kv.rpc.RangeRequest; +import org.apache.bookkeeper.stream.proto.kv.rpc.RangeResponse; +import org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc; +import org.apache.bookkeeper.stream.proto.kv.rpc.TxnRequest; +import org.apache.bookkeeper.stream.proto.kv.rpc.TxnResponse; + +/** + * ListenableFuture-returning stub for TableService. + * + *

The lightproto gRPC plugin does not generate FutureStub classes, so this adapter + * provides a drop-in replacement that uses {@link ClientCalls#futureUnaryCall} directly. + */ +public final class TableServiceFutureStub extends AbstractStub { + + public static TableServiceFutureStub newFutureStub(Channel channel) { + return new TableServiceFutureStub(channel, CallOptions.DEFAULT); + } + + private TableServiceFutureStub(Channel channel, CallOptions callOptions) { + super(channel, callOptions); + } + + @Override + protected TableServiceFutureStub build(Channel channel, CallOptions callOptions) { + return new TableServiceFutureStub(channel, callOptions); + } + + public ListenableFuture range(RangeRequest request) { + return ClientCalls.futureUnaryCall( + getChannel().newCall(TableServiceGrpc.getRangeMethod(), getCallOptions()), + request); + } + + public ListenableFuture put(PutRequest request) { + return ClientCalls.futureUnaryCall( + getChannel().newCall(TableServiceGrpc.getPutMethod(), getCallOptions()), + request); + } + + public ListenableFuture delete(DeleteRangeRequest request) { + return ClientCalls.futureUnaryCall( + getChannel().newCall(TableServiceGrpc.getDeleteMethod(), getCallOptions()), + request); + } + + public ListenableFuture txn(TxnRequest request) { + return ClientCalls.futureUnaryCall( + getChannel().newCall(TableServiceGrpc.getTxnMethod(), getCallOptions()), + request); + } + + public ListenableFuture increment(IncrementRequest request) { + return ClientCalls.futureUnaryCall( + getChannel().newCall(TableServiceGrpc.getIncrementMethod(), getCallOptions()), + request); + } +} diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/package-info.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/package-info.java new file mode 100644 index 00000000000..2454994abb7 --- /dev/null +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/grpc/package-info.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * gRPC stub adapters that wrap LightProto-generated services to expose + * {@code ListenableFuture}-based async APIs (lightproto's plugin only generates + * blocking and {@code StreamObserver} stubs). + */ +package org.apache.bookkeeper.clients.grpc; diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/channel/StorageServerChannel.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/channel/StorageServerChannel.java index d581c942eb5..4dc8d1e6fd5 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/channel/StorageServerChannel.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/channel/StorageServerChannel.java @@ -28,19 +28,15 @@ import java.util.function.Function; import javax.annotation.concurrent.GuardedBy; import org.apache.bookkeeper.clients.config.StorageClientSettings; +import org.apache.bookkeeper.clients.grpc.MetaRangeServiceFutureStub; +import org.apache.bookkeeper.clients.grpc.RootRangeServiceFutureStub; +import org.apache.bookkeeper.clients.grpc.StorageContainerServiceFutureStub; +import org.apache.bookkeeper.clients.grpc.TableServiceFutureStub; import org.apache.bookkeeper.clients.impl.container.StorageContainerClientInterceptor; import org.apache.bookkeeper.clients.resolver.EndpointResolver; import org.apache.bookkeeper.clients.utils.GrpcUtils; import org.apache.bookkeeper.common.grpc.stats.MonitoringClientInterceptor; import org.apache.bookkeeper.stream.proto.common.Endpoint; -import org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc; -import org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.TableServiceFutureStub; -import org.apache.bookkeeper.stream.proto.storage.MetaRangeServiceGrpc; -import org.apache.bookkeeper.stream.proto.storage.MetaRangeServiceGrpc.MetaRangeServiceFutureStub; -import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc; -import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc.RootRangeServiceFutureStub; -import org.apache.bookkeeper.stream.proto.storage.StorageContainerServiceGrpc; -import org.apache.bookkeeper.stream.proto.storage.StorageContainerServiceGrpc.StorageContainerServiceFutureStub; /** * A channel connected to a range server. @@ -135,7 +131,7 @@ private StorageServerChannel(Channel channel, public synchronized RootRangeServiceFutureStub getRootRangeService() { if (null == rootRangeService) { rootRangeService = GrpcUtils.configureGrpcStub( - RootRangeServiceGrpc.newFutureStub(channel), + RootRangeServiceFutureStub.newFutureStub(channel), token); } return rootRangeService; @@ -144,7 +140,7 @@ public synchronized RootRangeServiceFutureStub getRootRangeService() { public synchronized MetaRangeServiceFutureStub getMetaRangeService() { if (null == metaRangeService) { metaRangeService = GrpcUtils.configureGrpcStub( - MetaRangeServiceGrpc.newFutureStub(channel), + MetaRangeServiceFutureStub.newFutureStub(channel), token); } return metaRangeService; @@ -153,7 +149,7 @@ public synchronized MetaRangeServiceFutureStub getMetaRangeService() { public synchronized StorageContainerServiceFutureStub getStorageContainerService() { if (null == scService) { scService = GrpcUtils.configureGrpcStub( - StorageContainerServiceGrpc.newFutureStub(channel), + StorageContainerServiceFutureStub.newFutureStub(channel), token); } return scService; @@ -162,7 +158,7 @@ public synchronized StorageContainerServiceFutureStub getStorageContainerService public synchronized TableServiceFutureStub getTableService() { if (null == kvService) { kvService = GrpcUtils.configureGrpcStub( - TableServiceGrpc.newFutureStub(channel), + TableServiceFutureStub.newFutureStub(channel), token); } return kvService; diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerChannel.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerChannel.java index f5ff7e1e5a4..415ce89f10e 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerChannel.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/container/StorageContainerChannel.java @@ -174,9 +174,9 @@ private void handleFetchStorageContainerInfoSuccess( } // we got the updated location List readEndpoints = - Lists.newArrayListWithExpectedSize(1 + endpoint.getRoEndpointCount()); + Lists.newArrayListWithExpectedSize(1 + endpoint.getRoEndpointsCount()); readEndpoints.add(endpoint.getRwEndpoint()); - readEndpoints.addAll(endpoint.getRoEndpointList()); + readEndpoints.addAll(endpoint.getRoEndpointsList()); scInfo = StorageContainerInfo.of( scId, endpoint.getRevision(), diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/LocationClientImpl.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/LocationClientImpl.java index e92b5718f06..233494a1739 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/LocationClientImpl.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/LocationClientImpl.java @@ -34,6 +34,7 @@ import java.util.stream.Stream; import lombok.CustomLog; import org.apache.bookkeeper.clients.config.StorageClientSettings; +import org.apache.bookkeeper.clients.grpc.StorageContainerServiceFutureStub; import org.apache.bookkeeper.clients.impl.internal.api.LocationClient; import org.apache.bookkeeper.clients.utils.ClientConstants; import org.apache.bookkeeper.clients.utils.GrpcChannels; @@ -44,8 +45,6 @@ import org.apache.bookkeeper.common.util.Revisioned; import org.apache.bookkeeper.stream.proto.storage.GetStorageContainerEndpointRequest; import org.apache.bookkeeper.stream.proto.storage.OneStorageContainerEndpointResponse; -import org.apache.bookkeeper.stream.proto.storage.StorageContainerServiceGrpc; -import org.apache.bookkeeper.stream.proto.storage.StorageContainerServiceGrpc.StorageContainerServiceFutureStub; /** * Default Implementation of {@link LocationClient}. @@ -67,7 +66,7 @@ public LocationClientImpl(StorageClientSettings settings, settings.serviceUri(), settings ).build(); this.locationService = GrpcUtils.configureGrpcStub( - StorageContainerServiceGrpc.newFutureStub(channel), + StorageContainerServiceFutureStub.newFutureStub(channel), Optional.empty()); } diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImpl.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImpl.java index 1e65082971c..d393479098d 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImpl.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImpl.java @@ -33,6 +33,7 @@ import java.util.concurrent.ScheduledExecutorService; import lombok.CustomLog; import org.apache.bookkeeper.clients.exceptions.ClientException; +import org.apache.bookkeeper.clients.grpc.RootRangeServiceFutureStub; import org.apache.bookkeeper.clients.impl.container.StorageContainerChannel; import org.apache.bookkeeper.clients.impl.container.StorageContainerChannelManager; import org.apache.bookkeeper.clients.impl.internal.api.RootRangeClient; @@ -52,7 +53,6 @@ import org.apache.bookkeeper.stream.proto.storage.DeleteStreamResponse; import org.apache.bookkeeper.stream.proto.storage.GetNamespaceResponse; import org.apache.bookkeeper.stream.proto.storage.GetStreamResponse; -import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc.RootRangeServiceFutureStub; import org.apache.bookkeeper.stream.proto.storage.StatusCode; /** diff --git a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/NetUtils.java b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/NetUtils.java index 8a3caf9db92..11d9226f016 100644 --- a/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/NetUtils.java +++ b/stream/clients/java/base/src/main/java/org/apache/bookkeeper/clients/utils/NetUtils.java @@ -113,10 +113,9 @@ public static Endpoint parseEndpoint(String endpointStr) { * @return an endpoint created from {@code hostname} and {@code port}. */ public static Endpoint createEndpoint(String hostname, int port) { - return Endpoint.newBuilder() + return new Endpoint() .setHostname(hostname) - .setPort(port) - .build(); + .setPort(port); } /** diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/grpc/GrpcClientTestBase.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/grpc/GrpcClientTestBase.java index de5169444d9..ff63924fd35 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/grpc/GrpcClientTestBase.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/grpc/GrpcClientTestBase.java @@ -45,10 +45,9 @@ */ public abstract class GrpcClientTestBase { - protected static final Endpoint ENDPOINT = Endpoint.newBuilder() + protected static final Endpoint ENDPOINT = new Endpoint() .setHostname("127.0.0.1") - .setPort(4181) - .build(); + .setPort(4181); protected String serverName; protected final MutableHandlerRegistry serviceRegistry = new MutableHandlerRegistry(); @@ -86,19 +85,16 @@ public void setUp() throws Exception { public void getStorageContainerEndpoint( GetStorageContainerEndpointRequest request, StreamObserver responseObserver) { - GetStorageContainerEndpointResponse.Builder respBuilder = - GetStorageContainerEndpointResponse.newBuilder(); - respBuilder.setStatusCode(StatusCode.SUCCESS); + GetStorageContainerEndpointResponse resp = new GetStorageContainerEndpointResponse(); + resp.setStatusCode(StatusCode.SUCCESS); for (OneStorageContainerEndpointRequest oneReq : request.getRequestsList()) { - OneStorageContainerEndpointResponse oneResp = OneStorageContainerEndpointResponse.newBuilder() - .setEndpoint(StorageContainerEndpoint.newBuilder() - .setStorageContainerId(oneReq.getStorageContainer()) - .setRevision(oneReq.getRevision() + 1) - .setRwEndpoint(ENDPOINT)) - .build(); - respBuilder.addResponses(oneResp); + OneStorageContainerEndpointResponse oneResp = resp.addResponse(); + StorageContainerEndpoint endpoint = oneResp.setEndpoint(); + endpoint.setStorageContainerId(oneReq.getStorageContainer()); + endpoint.setRevision(oneReq.getRevision() + 1); + endpoint.setRwEndpoint().copyFrom(ENDPOINT); } - responseObserver.onNext(respBuilder.build()); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/channel/TestStorageServerChannelManager.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/channel/TestStorageServerChannelManager.java index 4b1a713b7fc..84caca67bb3 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/channel/TestStorageServerChannelManager.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/channel/TestStorageServerChannelManager.java @@ -35,20 +35,17 @@ */ public class TestStorageServerChannelManager { - private final Endpoint endpoint1 = Endpoint.newBuilder() + private final Endpoint endpoint1 = new Endpoint() .setHostname("127.0.0.1") - .setPort(80) - .build(); + .setPort(80); private final StorageServerChannel channel1 = mock(StorageServerChannel.class); - private final Endpoint endpoint2 = Endpoint.newBuilder() + private final Endpoint endpoint2 = new Endpoint() .setHostname("127.0.0.2") - .setPort(8080) - .build(); + .setPort(8080); private final StorageServerChannel channel2 = mock(StorageServerChannel.class); - private final Endpoint endpoint3 = Endpoint.newBuilder() + private final Endpoint endpoint3 = new Endpoint() .setHostname("127.0.0.3") - .setPort(8181) - .build(); + .setPort(8181); private final StorageServerChannelManager channelManager = new StorageServerChannelManager((endpoint) -> { diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/container/TestStorageContainerChannel.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/container/TestStorageContainerChannel.java index 56233354be2..209a7b0b7a3 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/container/TestStorageContainerChannel.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/container/TestStorageContainerChannel.java @@ -62,23 +62,20 @@ public class TestStorageContainerChannel extends GrpcClientTestBase { private StorageServerChannel mockChannel = newMockServerChannel(); private StorageServerChannel mockChannel2 = newMockServerChannel(); private StorageServerChannel mockChannel3 = newMockServerChannel(); - private final Endpoint endpoint = Endpoint.newBuilder() + private final Endpoint endpoint = new Endpoint() .setHostname("127.0.0.1") - .setPort(8181) - .build(); - private final Endpoint endpoint2 = Endpoint.newBuilder() + .setPort(8181); + private final Endpoint endpoint2 = new Endpoint() .setHostname("127.0.0.2") - .setPort(8282) - .build(); - private final Endpoint endpoint3 = Endpoint.newBuilder() + .setPort(8282); + private final Endpoint endpoint3 = new Endpoint() .setHostname("127.0.0.3") - .setPort(8383) - .build(); + .setPort(8383); private final StorageServerChannelManager channelManager = new StorageServerChannelManager( ep -> { - if (endpoint2 == ep) { + if (endpoint2.equals(ep)) { return mockChannel2; - } else if (endpoint3 == ep) { + } else if (endpoint3.equals(ep)) { return mockChannel3; } else { return mockChannel; @@ -113,6 +110,18 @@ private StorageServerChannel newMockServerChannel() { return channel; } + private static OneStorageContainerEndpointResponse buildOneEndpointResponse( + long scId, long revision, Endpoint rwEndpoint, Endpoint roEndpoint) { + OneStorageContainerEndpointResponse oneResp = new OneStorageContainerEndpointResponse(); + oneResp.setStatusCode(StatusCode.SUCCESS); + StorageContainerEndpoint scEndpoint = oneResp.setEndpoint(); + scEndpoint.setStorageContainerId(scId); + scEndpoint.setRevision(revision); + scEndpoint.setRwEndpoint().copyFrom(rwEndpoint); + scEndpoint.addRoEndpoint().copyFrom(roEndpoint); + return oneResp; + } + private void ensureCallbackExecuted() throws Exception { final CountDownLatch latch = new CountDownLatch(1); scheduler.submit(() -> latch.countDown()); @@ -139,16 +148,8 @@ public void testGetRootRangeServiceSuccess() throws Exception { assertNull(scClient.getStorageContainerInfo()); verify(locationClient, times(1)).locateStorageContainers(anyList()); // prepare the result and complete the request - OneStorageContainerEndpointResponse oneResp = OneStorageContainerEndpointResponse.newBuilder() - .setStatusCode(StatusCode.SUCCESS) - .setEndpoint( - StorageContainerEndpoint.newBuilder() - .setStorageContainerId(ROOT_STORAGE_CONTAINER_ID) - .setRevision(1000L) - .setRwEndpoint(endpoint) - .addRoEndpoint(endpoint) - .build()) - .build(); + OneStorageContainerEndpointResponse oneResp = + buildOneEndpointResponse(ROOT_STORAGE_CONTAINER_ID, 1000L, endpoint, endpoint); locateResponses.complete(Lists.newArrayList(oneResp)); // get the service StorageServerChannel rsChannel = rsChannelFuture.get(); @@ -187,16 +188,8 @@ public void testGetRootRangeServiceFailureWhenClosingChannelManager() throws Exc // closing the channel manager channelManager.close(); // prepare the result and complete the request - OneStorageContainerEndpointResponse oneResp = OneStorageContainerEndpointResponse.newBuilder() - .setStatusCode(StatusCode.SUCCESS) - .setEndpoint( - StorageContainerEndpoint.newBuilder() - .setStorageContainerId(ROOT_STORAGE_CONTAINER_ID) - .setRevision(1000L) - .setRwEndpoint(endpoint) - .addRoEndpoint(endpoint) - .build()) - .build(); + OneStorageContainerEndpointResponse oneResp = + buildOneEndpointResponse(ROOT_STORAGE_CONTAINER_ID, 1000L, endpoint, endpoint); locateResponses.complete(Lists.newArrayList(oneResp)); // verify the result try { @@ -247,16 +240,8 @@ public void testGetRootRangeServiceFailureOnStaleGroupInfo() throws Exception { // // prepare the result and complete the request - OneStorageContainerEndpointResponse oneResp1 = OneStorageContainerEndpointResponse.newBuilder() - .setStatusCode(StatusCode.SUCCESS) - .setEndpoint( - StorageContainerEndpoint.newBuilder() - .setStorageContainerId(ROOT_STORAGE_CONTAINER_ID) - .setRevision(1000L) - .setRwEndpoint(endpoint) - .addRoEndpoint(endpoint) - .build()) - .build(); + OneStorageContainerEndpointResponse oneResp1 = + buildOneEndpointResponse(ROOT_STORAGE_CONTAINER_ID, 1000L, endpoint, endpoint); locateResponses1.complete(Lists.newArrayList(oneResp1)); // get the service StorageServerChannel rsChannel = rsChannelFuture.get(); @@ -277,16 +262,8 @@ public void testGetRootRangeServiceFailureOnStaleGroupInfo() throws Exception { scClient.resetStorageServerChannelFuture(); rsChannelFuture = scClient.getStorageContainerChannelFuture(); - OneStorageContainerEndpointResponse oneResp2 = OneStorageContainerEndpointResponse.newBuilder() - .setStatusCode(StatusCode.SUCCESS) - .setEndpoint( - StorageContainerEndpoint.newBuilder() - .setStorageContainerId(ROOT_STORAGE_CONTAINER_ID) - .setRevision(999L) - .setRwEndpoint(endpoint2) - .addRoEndpoint(endpoint2) - .build()) - .build(); + OneStorageContainerEndpointResponse oneResp2 = + buildOneEndpointResponse(ROOT_STORAGE_CONTAINER_ID, 999L, endpoint2, endpoint2); locateResponses2.complete(Lists.newArrayList(oneResp2)); ensureCallbackExecuted(); @@ -306,16 +283,8 @@ public void testGetRootRangeServiceFailureOnStaleGroupInfo() throws Exception { scClient.resetStorageServerChannelFuture(); rsChannelFuture = scClient.getStorageContainerChannelFuture(); - OneStorageContainerEndpointResponse oneResp3 = OneStorageContainerEndpointResponse.newBuilder() - .setStatusCode(StatusCode.SUCCESS) - .setEndpoint( - StorageContainerEndpoint.newBuilder() - .setStorageContainerId(ROOT_STORAGE_CONTAINER_ID) - .setRevision(1001L) - .setRwEndpoint(endpoint3) - .addRoEndpoint(endpoint3) - .build()) - .build(); + OneStorageContainerEndpointResponse oneResp3 = + buildOneEndpointResponse(ROOT_STORAGE_CONTAINER_ID, 1001L, endpoint3, endpoint3); locateResponses3.complete(Lists.newArrayList(oneResp3)); ensureCallbackExecuted(); @@ -354,16 +323,8 @@ public void testGetRootRangeServiceUnexpectedException() throws Exception { assertNull(scClient.getStorageContainerInfo()); verify(locationClient, times(1)).locateStorageContainers(anyList()); // prepare the result and complete the request - OneStorageContainerEndpointResponse oneResp = OneStorageContainerEndpointResponse.newBuilder() - .setStatusCode(StatusCode.SUCCESS) - .setEndpoint( - StorageContainerEndpoint.newBuilder() - .setStorageContainerId(ROOT_STORAGE_CONTAINER_ID) - .setRevision(1000L) - .setRwEndpoint(endpoint) - .addRoEndpoint(endpoint) - .build()) - .build(); + OneStorageContainerEndpointResponse oneResp = + buildOneEndpointResponse(ROOT_STORAGE_CONTAINER_ID, 1000L, endpoint, endpoint); // complete with wrong responses locateResponses1.complete(Lists.newArrayList(oneResp, oneResp)); ensureCallbackExecuted(); @@ -413,16 +374,8 @@ public void testGetRootRangeServiceExceptionally() throws Exception { assertNull(scClient.getStorageContainerInfo()); verify(locationClient, times(1)).locateStorageContainers(anyList()); // prepare the result and complete the request - OneStorageContainerEndpointResponse oneResp = OneStorageContainerEndpointResponse.newBuilder() - .setStatusCode(StatusCode.SUCCESS) - .setEndpoint( - StorageContainerEndpoint.newBuilder() - .setStorageContainerId(ROOT_STORAGE_CONTAINER_ID) - .setRevision(1000L) - .setRwEndpoint(endpoint) - .addRoEndpoint(endpoint) - .build()) - .build(); + OneStorageContainerEndpointResponse oneResp = + buildOneEndpointResponse(ROOT_STORAGE_CONTAINER_ID, 1000L, endpoint, endpoint); // complete exceptionally locateResponses1.completeExceptionally(new ClientException("test-exception")); ensureCallbackExecuted(); diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/container/TestStorageContainerInfo.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/container/TestStorageContainerInfo.java index ce0dc3650b4..0b0b3c985bd 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/container/TestStorageContainerInfo.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/container/TestStorageContainerInfo.java @@ -33,10 +33,9 @@ public class TestStorageContainerInfo { public void testBasic() { long groupId = 1234L; long revision = 4468L; - Endpoint endpoint = Endpoint.newBuilder() + Endpoint endpoint = new Endpoint() .setHostname("123.46.78.96") - .setPort(3181) - .build(); + .setPort(3181); StorageContainerInfo sc = StorageContainerInfo.of( groupId, revision, diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/container/TestStorageContainerManager.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/container/TestStorageContainerManager.java index a11f4f2c931..d0dddb4988a 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/container/TestStorageContainerManager.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/container/TestStorageContainerManager.java @@ -31,14 +31,12 @@ public class TestStorageContainerManager { private final StorageContainerManager manager = new StorageContainerManager(); - private final Endpoint endpoint1 = Endpoint.newBuilder() + private final Endpoint endpoint1 = new Endpoint() .setHostname("128.0.0.1") - .setPort(3181) - .build(); - private final Endpoint endpoint2 = Endpoint.newBuilder() + .setPort(3181); + private final Endpoint endpoint2 = new Endpoint() .setHostname("128.0.0.1") - .setPort(3181) - .build(); + .setPort(3181); @Test public void testGetNullStorageContainer() { diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImplTestBase.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImplTestBase.java index c61052c48f8..a70318528e5 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImplTestBase.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImplTestBase.java @@ -64,18 +64,15 @@ public abstract class RootRangeClientImplTestBase extends GrpcClientTestBase { private StorageServerChannel mockChannel = mock(StorageServerChannel.class); private StorageServerChannel mockChannel2 = mock(StorageServerChannel.class); private StorageServerChannel mockChannel3 = mock(StorageServerChannel.class); - private final Endpoint endpoint = Endpoint.newBuilder() + private final Endpoint endpoint = new Endpoint() .setHostname("127.0.0.1") - .setPort(8181) - .build(); - private final Endpoint endpoint2 = Endpoint.newBuilder() + .setPort(8181); + private final Endpoint endpoint2 = new Endpoint() .setHostname("127.0.0.2") - .setPort(8282) - .build(); - private final Endpoint endpoint3 = Endpoint.newBuilder() + .setPort(8282); + private final Endpoint endpoint3 = new Endpoint() .setHostname("127.0.0.3") - .setPort(8383) - .build(); + .setPort(8383); private final StorageServerChannelManager channelManager = new StorageServerChannelManager( ep -> { if (endpoint2 == ep) { diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImplWithRetriesTest.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImplWithRetriesTest.java index fa160028490..49fb23bc243 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImplWithRetriesTest.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/RootRangeClientImplWithRetriesTest.java @@ -48,11 +48,11 @@ public class RootRangeClientImplWithRetriesTest { private static final int NUM_RETRIES = 3; private static final String NS_NAME = "test-namespace"; - private static final NamespaceConfiguration NS_CONF = NamespaceConfiguration.newBuilder().build(); - private static final NamespaceProperties NS_PROPS = NamespaceProperties.newBuilder().build(); + private static final NamespaceConfiguration NS_CONF = new NamespaceConfiguration(); + private static final NamespaceProperties NS_PROPS = new NamespaceProperties(); private static final String STREAM_NAME = "test-stream"; - private static final StreamConfiguration STREAM_CONF = StreamConfiguration.newBuilder().build(); - private static final StreamProperties STREAM_PROPS = StreamProperties.newBuilder().build(); + private static final StreamConfiguration STREAM_CONF = new StreamConfiguration(); + private static final StreamProperties STREAM_PROPS = new StreamProperties(); private AtomicInteger callCounter; private RootRangeClient client; diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestLocationClientImpl.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestLocationClientImpl.java index 11f0f25140b..b304a5f2a9e 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestLocationClientImpl.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestLocationClientImpl.java @@ -62,12 +62,12 @@ public class TestLocationClientImpl extends GrpcClientTestBase { private static StorageContainerEndpoint createEndpoint(int groupId) { - return StorageContainerEndpoint.newBuilder() - .setStorageContainerId(groupId) - .setRevision(1000L + groupId) - .setRwEndpoint(NetUtils.createEndpoint("127.0.0." + groupId, groupId)) - .addRoEndpoint(NetUtils.createEndpoint("128.0.0." + groupId, groupId)) - .build(); + StorageContainerEndpoint endpoint = new StorageContainerEndpoint(); + endpoint.setStorageContainerId(groupId); + endpoint.setRevision(1000L + groupId); + endpoint.setRwEndpoint().copyFrom(NetUtils.createEndpoint("127.0.0." + groupId, groupId)); + endpoint.addRoEndpoint().copyFrom(NetUtils.createEndpoint("128.0.0." + groupId, groupId)); + return endpoint; } private LocationClientImpl locationClient; @@ -83,20 +83,21 @@ private static StorageContainerEndpoint createEndpoint(int groupId) { @Override public void getStorageContainerEndpoint(GetStorageContainerEndpointRequest request, StreamObserver responseObserver) { - GetStorageContainerEndpointResponse.Builder respBuilder = GetStorageContainerEndpointResponse.newBuilder(); + GetStorageContainerEndpointResponse resp = new GetStorageContainerEndpointResponse(); if (0 == request.getRequestsCount()) { responseObserver.onError(new StatusRuntimeException(Status.INVALID_ARGUMENT)); } else { for (OneStorageContainerEndpointRequest oneRequest : request.getRequestsList()) { - respBuilder.addResponses(processOneStorageContainerEndpointRequest(oneRequest)); + populateOneStorageContainerEndpointResponse(resp.addResponse(), oneRequest); } - respBuilder.setStatusCode(StatusCode.SUCCESS); - responseObserver.onNext(respBuilder.build()); + resp.setStatusCode(StatusCode.SUCCESS); + responseObserver.onNext(resp); } responseObserver.onCompleted(); } - OneStorageContainerEndpointResponse.Builder processOneStorageContainerEndpointRequest( + void populateOneStorageContainerEndpointResponse( + OneStorageContainerEndpointResponse response, OneStorageContainerEndpointRequest request) { StatusCode code; StorageContainerEndpoint endpoint = null; @@ -114,12 +115,10 @@ OneStorageContainerEndpointResponse.Builder processOneStorageContainerEndpointRe endpoint = null; } } - OneStorageContainerEndpointResponse.Builder builder = OneStorageContainerEndpointResponse.newBuilder() - .setStatusCode(code); + response.setStatusCode(code); if (null != endpoint) { - builder = builder.setEndpoint(endpoint); + response.setEndpoint().copyFrom(endpoint); } - return builder; } }; private ServerServiceDefinition locationServiceDefinition; diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestMetaRangeClientImpl.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestMetaRangeClientImpl.java index 985b9551576..f17ee14fd84 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestMetaRangeClientImpl.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestMetaRangeClientImpl.java @@ -61,12 +61,16 @@ public class TestMetaRangeClientImpl extends GrpcClientTestBase { private static final long streamId = 1234L; private static final long groupId = 456L; - private static final StreamProperties streamProps = StreamProperties.newBuilder() - .setStreamId(streamId) - .setStorageContainerId(groupId) - .setStreamName("test-meta-range-client") - .setStreamConf(StreamConfiguration.newBuilder().build()) - .build(); + private static final StreamProperties streamProps = newStreamProperties(); + + private static StreamProperties newStreamProperties() { + StreamProperties props = new StreamProperties(); + props.setStreamId(streamId); + props.setStorageContainerId(groupId); + props.setStreamName("test-meta-range-client"); + props.setStreamConf().copyFrom(new StreamConfiguration()); + return props; + } private final LocationClient locationClient = mock(LocationClient.class); private MetaRangeClientImpl metaRangeClient; private final StorageServerChannel rsChannel = mock(StorageServerChannel.class); @@ -96,30 +100,22 @@ protected void doTeardown() throws Exception { } - private RelatedRanges buildRelatedRange(long startKey, - long endKey, - long rangeId, - long groupId, - List parentRanges) { - return RelatedRanges.newBuilder() - .setProps(buildRangeMeta( - startKey, endKey, rangeId, groupId)) - .setType(RelationType.PARENTS) - .addAllRelatedRanges(parentRanges) - .build(); - } - - - private RangeProperties buildRangeMeta(long startKey, - long endKey, - long rangeId, - long groupId) { - return RangeProperties.newBuilder() - .setStartHashKey(startKey) - .setEndHashKey(endKey) - .setRangeId(rangeId) - .setStorageContainerId(groupId) - .build(); + private void addRelatedRange(GetActiveRangesResponse response, + long startKey, + long endKey, + long rangeId, + long groupId, + List parentRanges) { + RelatedRanges related = response.addRange(); + RangeProperties props = related.setProps(); + props.setStartHashKey(startKey); + props.setEndHashKey(endKey); + props.setRangeId(rangeId); + props.setStorageContainerId(groupId); + related.setType(RelationType.PARENTS); + for (Long parent : parentRanges) { + related.addRelatedRange(parent); + } } @Test @@ -128,13 +124,10 @@ public void testGetActiveStreamRanges() throws Exception { metaRangeClient.getStorageContainerClient().setStorageServerChannelFuture(serviceFuture); // create response - GetActiveRangesResponse getActiveRangesResponse = GetActiveRangesResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .addRanges( - buildRelatedRange(Long.MIN_VALUE, 0L, 123L, 1L, Lists.newArrayList(113L)) - ).addRanges( - buildRelatedRange(0L, Long.MAX_VALUE, 124L, 2L, Lists.newArrayList(114L)) - ).build(); + GetActiveRangesResponse getActiveRangesResponse = new GetActiveRangesResponse(); + getActiveRangesResponse.setCode(StatusCode.SUCCESS); + addRelatedRange(getActiveRangesResponse, Long.MIN_VALUE, 0L, 123L, 1L, Lists.newArrayList(113L)); + addRelatedRange(getActiveRangesResponse, 0L, Long.MAX_VALUE, 124L, 2L, Lists.newArrayList(114L)); MetaRangeServiceImplBase metaRangeService = new MetaRangeServiceImplBase() { @Override diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestProtocolInternalUtils.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestProtocolInternalUtils.java index 07bec6b581e..861702807d8 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestProtocolInternalUtils.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestProtocolInternalUtils.java @@ -68,32 +68,30 @@ public class TestProtocolInternalUtils { // Test Meta KeyRange Server Requests // + private static void addRange(GetActiveRangesResponse response, + long startHashKey, + long endHashKey, + long rangeId, + long scId) { + RelatedRanges related = response.addRange(); + RangeProperties props = related.setProps(); + props.setStartHashKey(startHashKey); + props.setEndHashKey(endHashKey); + props.setRangeId(rangeId); + props.setStorageContainerId(scId); + related.setType(RelationType.PARENTS); + related.addRelatedRange(INVALID_RANGE_ID); + } + @Test public void testCreateActiveRanges() { - GetActiveRangesResponse.Builder responseBuilder = GetActiveRangesResponse.newBuilder(); - responseBuilder.addRanges( - RelatedRanges.newBuilder() - .setProps(RangeProperties.newBuilder() - .setStartHashKey(Long.MIN_VALUE) - .setEndHashKey(0L) - .setRangeId(1L) - .setStorageContainerId(1L)) - .setType(RelationType.PARENTS) - .addRelatedRanges(INVALID_RANGE_ID) - ).addRanges( - RelatedRanges.newBuilder() - .setProps(RangeProperties.newBuilder() - .setStartHashKey(0L) - .setEndHashKey(Long.MAX_VALUE) - .setRangeId(2L) - .setStorageContainerId(2L)) - .setType(RelationType.PARENTS) - .addRelatedRanges(INVALID_RANGE_ID)); - GetActiveRangesResponse response = responseBuilder.build(); + GetActiveRangesResponse response = new GetActiveRangesResponse(); + addRange(response, Long.MIN_VALUE, 0L, 1L, 1L); + addRange(response, 0L, Long.MAX_VALUE, 2L, 2L); HashStreamRanges hsr = createActiveRanges(response); TreeMap activeRanges = Maps.newTreeMap(); - activeRanges.put(Long.MIN_VALUE, response.getRanges(0).getProps()); - activeRanges.put(0L, response.getRanges(1).getProps()); + activeRanges.put(Long.MIN_VALUE, response.getRangeAt(0).getProps()); + activeRanges.put(0L, response.getRangeAt(1).getProps()); HashStreamRanges expectedHSR = HashStreamRanges.ofHash( RangeKeyType.HASH, activeRanges); @@ -103,27 +101,11 @@ public void testCreateActiveRanges() { @Test public void testCreateActiveRangesInvalidKeyRange() { - GetActiveRangesResponse.Builder responseBuilder = GetActiveRangesResponse.newBuilder(); - responseBuilder.addRanges( - RelatedRanges.newBuilder() - .setProps(RangeProperties.newBuilder() - .setStartHashKey(Long.MIN_VALUE) - .setEndHashKey(0L) - .setRangeId(1L) - .setStorageContainerId(1L)) - .setType(RelationType.PARENTS) - .addRelatedRanges(INVALID_RANGE_ID) - ).addRanges( - RelatedRanges.newBuilder() - .setProps(RangeProperties.newBuilder() - .setStartHashKey(1L) - .setEndHashKey(Long.MAX_VALUE) - .setRangeId(2L) - .setStorageContainerId(2L)) - .setType(RelationType.PARENTS) - .addRelatedRanges(INVALID_RANGE_ID)); + GetActiveRangesResponse response = new GetActiveRangesResponse(); + addRange(response, Long.MIN_VALUE, 0L, 1L, 1L); + addRange(response, 1L, Long.MAX_VALUE, 2L, 2L); try { - createActiveRanges(responseBuilder.build()); + createActiveRanges(response); fail("Should fail with invalid key range"); } catch (IllegalStateException ise) { assertEquals( @@ -134,27 +116,11 @@ public void testCreateActiveRangesInvalidKeyRange() { @Test public void testCreateActiveRangesMissingKeyRange() { - GetActiveRangesResponse.Builder responseBuilder = GetActiveRangesResponse.newBuilder(); - responseBuilder.addRanges( - RelatedRanges.newBuilder() - .setProps(RangeProperties.newBuilder() - .setStartHashKey(Long.MIN_VALUE) - .setEndHashKey(0L) - .setRangeId(1L) - .setStorageContainerId(1L)) - .setType(RelationType.PARENTS) - .addRelatedRanges(INVALID_RANGE_ID) - ).addRanges( - RelatedRanges.newBuilder() - .setProps(RangeProperties.newBuilder() - .setStartHashKey(0L) - .setEndHashKey(1234L) - .setRangeId(2L) - .setStorageContainerId(2L)) - .setType(RelationType.PARENTS) - .addRelatedRanges(INVALID_RANGE_ID)); + GetActiveRangesResponse response = new GetActiveRangesResponse(); + addRange(response, Long.MIN_VALUE, 0L, 1L, 1L); + addRange(response, 0L, 1234L, 2L, 2L); try { - createActiveRanges(responseBuilder.build()); + createActiveRanges(response); fail("Should fail with missing key range"); } catch (IllegalStateException ise) { assertEquals( @@ -186,15 +152,15 @@ public void testCreateGetStorageContainerEndpointRequest() { @Test public void testCreateStorageContainerEndpointResponse() { List endpoints = Lists.newArrayList( - StorageContainerEndpoint.newBuilder().setStorageContainerId(1L).build(), - StorageContainerEndpoint.newBuilder().setStorageContainerId(2L).build(), - StorageContainerEndpoint.newBuilder().setStorageContainerId(3L).build()); + new StorageContainerEndpoint().setStorageContainerId(1L), + new StorageContainerEndpoint().setStorageContainerId(2L), + new StorageContainerEndpoint().setStorageContainerId(3L)); GetStorageContainerEndpointResponse response = createGetStorageContainerEndpointResponse(endpoints); assertEquals(3, response.getResponsesCount()); int i = 0; for (OneStorageContainerEndpointResponse oneResp : response.getResponsesList()) { assertEquals(StatusCode.SUCCESS, oneResp.getStatusCode()); - assertTrue(endpoints.get(i) == oneResp.getEndpoint()); + assertEquals(endpoints.get(i), oneResp.getEndpoint()); ++i; } } diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientCreateNamespaceRpc.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientCreateNamespaceRpc.java index 111ac665119..7f659b3d324 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientCreateNamespaceRpc.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientCreateNamespaceRpc.java @@ -48,9 +48,13 @@ public class TestRootRangeClientCreateNamespaceRpc extends RootRangeClientImplTe private long colId; private String colName; private NamespaceProperties colProps; - private static final NamespaceConfiguration colConf = NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build(); + private static final NamespaceConfiguration colConf = newColConf(); + + private static NamespaceConfiguration newColConf() { + NamespaceConfiguration conf = new NamespaceConfiguration(); + conf.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); + return conf; + } @Override protected void doSetup() throws Exception { @@ -58,11 +62,10 @@ protected void doSetup() throws Exception { this.colId = System.currentTimeMillis(); this.colName = testName.getMethodName(); - this.colProps = NamespaceProperties.newBuilder() - .setNamespaceId(colId) - .setNamespaceName(colName) - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build(); + this.colProps = new NamespaceProperties(); + this.colProps.setNamespaceId(colId); + this.colProps.setNamespaceName(colName); + this.colProps.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); } // @@ -80,10 +83,10 @@ protected RootRangeServiceImplBase createRootRangeServiceForSuccess() { @Override public void createNamespace(CreateNamespaceRequest request, StreamObserver responseObserver) { - responseObserver.onNext(CreateNamespaceResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .setNsProps(colProps) - .build()); + CreateNamespaceResponse resp = new CreateNamespaceResponse(); + resp.setCode(StatusCode.SUCCESS); + resp.setNsProps().copyFrom(colProps); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; @@ -92,7 +95,7 @@ public void createNamespace(CreateNamespaceRequest request, @Override protected void verifySuccess(RootRangeClient rootRangeClient) throws Exception { CompletableFuture createFuture = rootRangeClient.createNamespace(colName, colConf); - assertTrue(colProps == createFuture.get()); + assertEquals(colProps, createFuture.get()); } @@ -102,9 +105,9 @@ protected RootRangeServiceImplBase createRootRangeServiceForRequestFailure() { @Override public void createNamespace(CreateNamespaceRequest request, StreamObserver responseObserver) { - responseObserver.onNext(CreateNamespaceResponse.newBuilder() - .setCode(StatusCode.NAMESPACE_NOT_FOUND) - .build()); + CreateNamespaceResponse resp = new CreateNamespaceResponse(); + resp.setCode(StatusCode.NAMESPACE_NOT_FOUND); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientCreateStreamRpc.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientCreateStreamRpc.java index 62c395af10b..1755c618146 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientCreateStreamRpc.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientCreateStreamRpc.java @@ -58,12 +58,11 @@ protected void doSetup() throws Exception { this.streamId = System.currentTimeMillis(); this.colName = testName.getMethodName() + "_col"; this.streamName = testName.getMethodName() + "_stream"; - this.streamProps = StreamProperties.newBuilder() - .setStorageContainerId(System.currentTimeMillis()) - .setStreamId(streamId) - .setStreamName(streamName) - .setStreamConf(DEFAULT_STREAM_CONF) - .build(); + this.streamProps = new StreamProperties(); + this.streamProps.setStorageContainerId(System.currentTimeMillis()); + this.streamProps.setStreamId(streamId); + this.streamProps.setStreamName(streamName); + this.streamProps.setStreamConf().copyFrom(DEFAULT_STREAM_CONF); } @Override @@ -72,10 +71,10 @@ protected RootRangeServiceImplBase createRootRangeServiceForSuccess() { @Override public void createStream(CreateStreamRequest request, StreamObserver responseObserver) { - responseObserver.onNext(CreateStreamResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .setStreamProps(streamProps) - .build()); + CreateStreamResponse resp = new CreateStreamResponse(); + resp.setCode(StatusCode.SUCCESS); + resp.setStreamProps().copyFrom(streamProps); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; @@ -85,7 +84,7 @@ public void createStream(CreateStreamRequest request, protected void verifySuccess(RootRangeClient rootRangeClient) throws Exception { CompletableFuture createFuture = rootRangeClient.createStream(colName, streamName, streamConf); - assertTrue(streamProps == createFuture.get()); + assertEquals(streamProps, createFuture.get()); } @Override @@ -94,9 +93,9 @@ protected RootRangeServiceImplBase createRootRangeServiceForRequestFailure() { @Override public void createStream(CreateStreamRequest request, StreamObserver responseObserver) { - responseObserver.onNext(CreateStreamResponse.newBuilder() - .setCode(StatusCode.STREAM_NOT_FOUND) - .build()); + CreateStreamResponse resp = new CreateStreamResponse(); + resp.setCode(StatusCode.STREAM_NOT_FOUND); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientDeleteNamespaceRpc.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientDeleteNamespaceRpc.java index f8ce93068d6..5a553a146f0 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientDeleteNamespaceRpc.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientDeleteNamespaceRpc.java @@ -57,9 +57,9 @@ protected RootRangeServiceImplBase createRootRangeServiceForSuccess() { @Override public void deleteNamespace(DeleteNamespaceRequest request, StreamObserver responseObserver) { - responseObserver.onNext(DeleteNamespaceResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .build()); + DeleteNamespaceResponse resp = new DeleteNamespaceResponse(); + resp.setCode(StatusCode.SUCCESS); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; @@ -76,9 +76,9 @@ protected RootRangeServiceImplBase createRootRangeServiceForRequestFailure() { @Override public void deleteNamespace(DeleteNamespaceRequest request, StreamObserver responseObserver) { - responseObserver.onNext(DeleteNamespaceResponse.newBuilder() - .setCode(StatusCode.NAMESPACE_NOT_FOUND) - .build()); + DeleteNamespaceResponse resp = new DeleteNamespaceResponse(); + resp.setCode(StatusCode.NAMESPACE_NOT_FOUND); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientDeleteStreamRpc.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientDeleteStreamRpc.java index bd8f5494fa2..04a6670d035 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientDeleteStreamRpc.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientDeleteStreamRpc.java @@ -59,9 +59,9 @@ protected RootRangeServiceImplBase createRootRangeServiceForSuccess() { @Override public void deleteStream(DeleteStreamRequest request, StreamObserver responseObserver) { - responseObserver.onNext(DeleteStreamResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .build()); + DeleteStreamResponse resp = new DeleteStreamResponse(); + resp.setCode(StatusCode.SUCCESS); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; @@ -78,9 +78,9 @@ protected RootRangeServiceImplBase createRootRangeServiceForRequestFailure() { @Override public void deleteStream(DeleteStreamRequest request, StreamObserver responseObserver) { - responseObserver.onNext(DeleteStreamResponse.newBuilder() - .setCode(StatusCode.STREAM_NOT_FOUND) - .build()); + DeleteStreamResponse resp = new DeleteStreamResponse(); + resp.setCode(StatusCode.STREAM_NOT_FOUND); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientGetNamespaceRpc.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientGetNamespaceRpc.java index edbfeecb86a..1b8c1228d7b 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientGetNamespaceRpc.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientGetNamespaceRpc.java @@ -48,9 +48,13 @@ public class TestRootRangeClientGetNamespaceRpc extends RootRangeClientImplTestB private long colId; private String colName; private NamespaceProperties colProps; - private static final NamespaceConfiguration colConf = NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build(); + private static final NamespaceConfiguration colConf = newColConf(); + + private static NamespaceConfiguration newColConf() { + NamespaceConfiguration conf = new NamespaceConfiguration(); + conf.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); + return conf; + } @Override protected void doSetup() throws Exception { @@ -58,11 +62,10 @@ protected void doSetup() throws Exception { this.colId = System.currentTimeMillis(); this.colName = testName.getMethodName(); - this.colProps = NamespaceProperties.newBuilder() - .setNamespaceId(colId) - .setNamespaceName(colName) - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build(); + this.colProps = new NamespaceProperties(); + this.colProps.setNamespaceId(colId); + this.colProps.setNamespaceName(colName); + this.colProps.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); } // @@ -80,10 +83,10 @@ protected RootRangeServiceImplBase createRootRangeServiceForSuccess() { @Override public void getNamespace(GetNamespaceRequest request, StreamObserver responseObserver) { - responseObserver.onNext(GetNamespaceResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .setNsProps(colProps) - .build()); + GetNamespaceResponse resp = new GetNamespaceResponse(); + resp.setCode(StatusCode.SUCCESS); + resp.setNsProps().copyFrom(colProps); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; @@ -92,7 +95,7 @@ public void getNamespace(GetNamespaceRequest request, @Override protected void verifySuccess(RootRangeClient rootRangeClient) throws Exception { CompletableFuture getFuture = rootRangeClient.getNamespace(colName); - assertTrue(colProps == getFuture.get()); + assertEquals(colProps, getFuture.get()); } @@ -102,9 +105,9 @@ protected RootRangeServiceImplBase createRootRangeServiceForRequestFailure() { @Override public void getNamespace(GetNamespaceRequest request, StreamObserver responseObserver) { - responseObserver.onNext(GetNamespaceResponse.newBuilder() - .setCode(StatusCode.NAMESPACE_NOT_FOUND) - .build()); + GetNamespaceResponse resp = new GetNamespaceResponse(); + resp.setCode(StatusCode.NAMESPACE_NOT_FOUND); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientGetStreamByIdRpc.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientGetStreamByIdRpc.java index 388dab12cdb..033903d83b7 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientGetStreamByIdRpc.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientGetStreamByIdRpc.java @@ -53,12 +53,11 @@ protected void doSetup() throws Exception { this.streamId = System.currentTimeMillis(); String streamName = testName.getMethodName() + "_stream"; - this.streamProps = StreamProperties.newBuilder() - .setStorageContainerId(System.currentTimeMillis()) - .setStreamId(streamId) - .setStreamName(streamName) - .setStreamConf(DEFAULT_STREAM_CONF) - .build(); + this.streamProps = new StreamProperties(); + this.streamProps.setStorageContainerId(System.currentTimeMillis()); + this.streamProps.setStreamId(streamId); + this.streamProps.setStreamName(streamName); + this.streamProps.setStreamConf().copyFrom(DEFAULT_STREAM_CONF); } @Override @@ -67,10 +66,10 @@ protected RootRangeServiceImplBase createRootRangeServiceForSuccess() { @Override public void getStream(GetStreamRequest request, StreamObserver responseObserver) { - responseObserver.onNext(GetStreamResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .setStreamProps(streamProps) - .build()); + GetStreamResponse resp = new GetStreamResponse(); + resp.setCode(StatusCode.SUCCESS); + resp.setStreamProps().copyFrom(streamProps); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; @@ -79,7 +78,7 @@ public void getStream(GetStreamRequest request, @Override protected void verifySuccess(RootRangeClient rootRangeClient) throws Exception { CompletableFuture getFuture = rootRangeClient.getStream(streamId); - assertTrue(streamProps == getFuture.get()); + assertEquals(streamProps, getFuture.get()); } @Override @@ -88,9 +87,9 @@ protected RootRangeServiceImplBase createRootRangeServiceForRequestFailure() { @Override public void getStream(GetStreamRequest request, StreamObserver responseObserver) { - responseObserver.onNext(GetStreamResponse.newBuilder() - .setCode(StatusCode.STREAM_NOT_FOUND) - .build()); + GetStreamResponse resp = new GetStreamResponse(); + resp.setCode(StatusCode.STREAM_NOT_FOUND); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientGetStreamRpc.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientGetStreamRpc.java index d9d66681a60..feaa8145991 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientGetStreamRpc.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestRootRangeClientGetStreamRpc.java @@ -56,12 +56,11 @@ protected void doSetup() throws Exception { this.streamId = System.currentTimeMillis(); this.colName = testName.getMethodName() + "_col"; this.streamName = testName.getMethodName() + "_stream"; - this.streamProps = StreamProperties.newBuilder() - .setStorageContainerId(System.currentTimeMillis()) - .setStreamId(streamId) - .setStreamName(streamName) - .setStreamConf(DEFAULT_STREAM_CONF) - .build(); + this.streamProps = new StreamProperties(); + this.streamProps.setStorageContainerId(System.currentTimeMillis()); + this.streamProps.setStreamId(streamId); + this.streamProps.setStreamName(streamName); + this.streamProps.setStreamConf().copyFrom(DEFAULT_STREAM_CONF); } @Override @@ -70,10 +69,10 @@ protected RootRangeServiceImplBase createRootRangeServiceForSuccess() { @Override public void getStream(GetStreamRequest request, StreamObserver responseObserver) { - responseObserver.onNext(GetStreamResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .setStreamProps(streamProps) - .build()); + GetStreamResponse resp = new GetStreamResponse(); + resp.setCode(StatusCode.SUCCESS); + resp.setStreamProps().copyFrom(streamProps); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; @@ -82,7 +81,7 @@ public void getStream(GetStreamRequest request, @Override protected void verifySuccess(RootRangeClient rootRangeClient) throws Exception { CompletableFuture getFuture = rootRangeClient.getStream(colName, streamName); - assertTrue(streamProps == getFuture.get()); + assertEquals(streamProps, getFuture.get()); } @Override @@ -91,9 +90,9 @@ protected RootRangeServiceImplBase createRootRangeServiceForRequestFailure() { @Override public void getStream(GetStreamRequest request, StreamObserver responseObserver) { - responseObserver.onNext(GetStreamResponse.newBuilder() - .setCode(StatusCode.STREAM_NOT_FOUND) - .build()); + GetStreamResponse resp = new GetStreamResponse(); + resp.setCode(StatusCode.STREAM_NOT_FOUND); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestStorageServerClientManagerImpl.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestStorageServerClientManagerImpl.java index d72a69e431e..cba0f08acfd 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestStorageServerClientManagerImpl.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestStorageServerClientManagerImpl.java @@ -52,15 +52,19 @@ protected void doSetup() throws Exception { protected void doTeardown() throws Exception { } + private static StreamProperties newStreamProperties(long streamId) { + StreamProperties props = new StreamProperties(); + props.setStorageContainerId(1234L); + props.setStreamId(streamId); + props.setStreamName("metaclient-stream"); + props.setStreamConf().copyFrom(new StreamConfiguration()); + return props; + } + @Test public void testGetMetaRangeClient() throws Exception { long streamId = 3456L; - StreamProperties props = StreamProperties.newBuilder() - .setStorageContainerId(1234L) - .setStreamId(streamId) - .setStreamName("metaclient-stream") - .setStreamConf(StreamConfiguration.newBuilder().build()) - .build(); + StreamProperties props = newStreamProperties(streamId); MetaRangeClientImpl metaRangeClient = serverManager.openMetaRangeClient(props); assertEquals(1234L, metaRangeClient.getStorageContainerClient().getStorageContainerId()); @@ -76,21 +80,16 @@ public void testGetMetaRangeClient() throws Exception { @Test public void testGetMetaRangeClientByStreamId() throws Exception { long streamId = 3456L; - StreamProperties props = StreamProperties.newBuilder() - .setStorageContainerId(1234L) - .setStreamId(streamId) - .setStreamName("metaclient-stream") - .setStreamConf(StreamConfiguration.newBuilder().build()) - .build(); + StreamProperties props = newStreamProperties(streamId); RootRangeServiceImplBase rootRangeService = new RootRangeServiceImplBase() { @Override public void getStream(GetStreamRequest request, StreamObserver responseObserver) { - responseObserver.onNext(GetStreamResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .setStreamProps(props) - .build()); + GetStreamResponse resp = new GetStreamResponse(); + resp.setCode(StatusCode.SUCCESS); + resp.setStreamProps().copyFrom(props); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; @@ -115,7 +114,7 @@ public void testGetLocationClient() throws Exception { assertEquals(1, responses.size()); assertEquals(StatusCode.SUCCESS, responses.get(0).getStatusCode()); assertEquals(ENDPOINT, responses.get(0).getEndpoint().getRwEndpoint()); - assertEquals(0, responses.get(0).getEndpoint().getRoEndpointCount()); + assertEquals(0, responses.get(0).getEndpoint().getRoEndpointsCount()); } diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestStreamMetadataCache.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestStreamMetadataCache.java index 54b902d5885..b889df15be7 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestStreamMetadataCache.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/TestStreamMetadataCache.java @@ -41,12 +41,16 @@ public class TestStreamMetadataCache { private final RootRangeClient scClient = mock(RootRangeClient.class); private final StreamMetadataCache cache = new StreamMetadataCache(scClient); - private final StreamProperties props = StreamProperties.newBuilder() - .setStorageContainerId(1234L) - .setStreamId(2345L) - .setStreamName("test-stream") - .setStreamConf(DEFAULT_STREAM_CONF) - .build(); + private final StreamProperties props = newStreamProperties(); + + private static StreamProperties newStreamProperties() { + StreamProperties props = new StreamProperties(); + props.setStorageContainerId(1234L); + props.setStreamId(2345L); + props.setStreamName("test-stream"); + props.setStreamConf().copyFrom(DEFAULT_STREAM_CONF); + return props; + } @Test public void testGetStreamProperties() throws Exception { diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/api/TestHashStreamRanges.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/api/TestHashStreamRanges.java index 5e215a55a09..5ff279dda57 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/api/TestHashStreamRanges.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/api/TestHashStreamRanges.java @@ -35,12 +35,11 @@ public class TestHashStreamRanges { public void testConstructor() { NavigableMap ranges = Maps.newTreeMap(); for (long hashKey = 0L; hashKey < 10L; hashKey++) { - RangeProperties props = RangeProperties.newBuilder() + RangeProperties props = new RangeProperties() .setStorageContainerId(hashKey) .setRangeId(hashKey) .setStartHashKey(hashKey) - .setEndHashKey(hashKey) - .build(); + .setEndHashKey(hashKey); ranges.put(hashKey, props); } diff --git a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/api/TestStreamRanges.java b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/api/TestStreamRanges.java index 59a74c6dfa3..3e1f4e001b2 100644 --- a/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/api/TestStreamRanges.java +++ b/stream/clients/java/base/src/test/java/org/apache/bookkeeper/clients/impl/internal/api/TestStreamRanges.java @@ -41,12 +41,11 @@ public void testInvalidRangeKeyType() { public void testConstructor() { NavigableMap ranges = Maps.newTreeMap(); for (long hashKey = 0L; hashKey < 10L; hashKey++) { - RangeProperties props = RangeProperties.newBuilder() + RangeProperties props = new RangeProperties() .setStorageContainerId(hashKey) .setRangeId(hashKey) .setStartHashKey(hashKey) - .setEndHashKey(hashKey) - .build(); + .setEndHashKey(hashKey); ranges.put(hashKey, props); } @@ -64,19 +63,17 @@ public void testEqual() { NavigableMap ranges1 = Maps.newTreeMap(); NavigableMap ranges2 = Maps.newTreeMap(); for (long hashKey = 0L; hashKey < 10L; hashKey++) { - RangeProperties props1 = RangeProperties.newBuilder() + RangeProperties props1 = new RangeProperties() .setStorageContainerId(hashKey) .setRangeId(hashKey) .setStartHashKey(hashKey) - .setEndHashKey(hashKey) - .build(); + .setEndHashKey(hashKey); ranges1.put(hashKey, props1); - RangeProperties props2 = RangeProperties.newBuilder() + RangeProperties props2 = new RangeProperties() .setStorageContainerId(hashKey) .setRangeId(hashKey) .setStartHashKey(hashKey) - .setEndHashKey(hashKey) - .build(); + .setEndHashKey(hashKey); ranges2.put(hashKey, props2); } @@ -95,19 +92,17 @@ public void testNotEqual() { NavigableMap ranges1 = Maps.newTreeMap(); NavigableMap ranges2 = Maps.newTreeMap(); for (long hashKey = 0L; hashKey < 10L; hashKey++) { - RangeProperties props1 = RangeProperties.newBuilder() + RangeProperties props1 = new RangeProperties() .setStorageContainerId(hashKey) .setRangeId(hashKey) .setStartHashKey(hashKey) - .setEndHashKey(hashKey) - .build(); + .setEndHashKey(hashKey); ranges1.put(hashKey, props1); - RangeProperties props2 = RangeProperties.newBuilder() + RangeProperties props2 = new RangeProperties() .setStorageContainerId(hashKey) .setRangeId(hashKey + 1) .setStartHashKey(hashKey + 1) - .setEndHashKey(hashKey + 1) - .build(); + .setEndHashKey(hashKey + 1); ranges2.put(hashKey, props2); } diff --git a/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/KvUtils.java b/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/KvUtils.java index 279219bc468..c6033cb7905 100644 --- a/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/KvUtils.java +++ b/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/KvUtils.java @@ -15,8 +15,6 @@ package org.apache.bookkeeper.clients.impl.kv; import com.google.common.collect.Lists; -import com.google.protobuf.ByteString; -import com.google.protobuf.UnsafeByteOperations; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import java.util.List; @@ -62,17 +60,13 @@ public final class KvUtils { private KvUtils() { } - public static ByteString toProtoKey(ByteBuf key) { - return UnsafeByteOperations.unsafeWrap(key.nioBuffer()); - } - public static org.apache.bookkeeper.api.kv.result.KeyValue fromProtoKeyValue( KeyValue kv, KeyValueFactory kvFactory) { return kvFactory.newKv() - .key(Unpooled.wrappedBuffer(kv.getKey().asReadOnlyByteBuffer())) - .value(Unpooled.wrappedBuffer(kv.getValue().asReadOnlyByteBuffer())) - .isNumber(kv.getIsNumber()) + .key(Unpooled.wrappedBuffer(kv.getKey())) + .value(Unpooled.wrappedBuffer(kv.getValue())) + .isNumber(kv.isIsNumber()) .numberValue(kv.getNumberValue()) .createRevision(kv.getCreateRevision()) .modifiedRevision(kv.getModRevision()) @@ -84,9 +78,13 @@ public static List fromProtoKeyValue(kv, kvFactory)); } - public static RangeRequest.Builder newRangeRequest(ByteBuf key, RangeOption option) { - RangeRequest.Builder builder = RangeRequest.newBuilder() - .setKey(toProtoKey(key)) + public static RangeRequest newRangeRequest(ByteBuf key, RangeOption option) { + // Always slice() ByteBufs before passing to lightproto setters: the + // generated serializer calls dst.writeBytes(src) which consumes src's + // readerIndex, so reusing the same buffer for another request (or having + // two fields alias the same buffer) silently produces empty fields. + RangeRequest request = new RangeRequest() + .setKey(key.slice()) .setCountOnly(option.countOnly()) .setKeysOnly(option.keysOnly()) .setLimit(option.limit()) @@ -95,9 +93,9 @@ public static RangeRequest.Builder newRangeRequest(ByteBuf key, RangeOption newRangeResult( @@ -106,16 +104,16 @@ public static RangeResult newRangeResult( KeyValueFactory kvFactory) { return resultFactory.newRangeResult(-1L) .count(response.getCount()) - .more(response.getMore()) + .more(response.isMore()) .kvs(fromProtoKeyValues(response.getKvsList(), kvFactory)); } - public static PutRequest.Builder newPutRequest(ByteBuf key, - ByteBuf value, - PutOption option) { - return PutRequest.newBuilder() - .setKey(UnsafeByteOperations.unsafeWrap(key.nioBuffer())) - .setValue(UnsafeByteOperations.unsafeWrap(value.nioBuffer())) + public static PutRequest newPutRequest(ByteBuf key, + ByteBuf value, + PutOption option) { + return new PutRequest() + .setKey(key.slice()) + .setValue(value.slice()) .setPrevKv(option.prevKv()); } @@ -130,11 +128,11 @@ public static PutResult newPutResult( return result; } - public static IncrementRequest.Builder newIncrementRequest(ByteBuf key, - long amount, - IncrementOption option) { - return IncrementRequest.newBuilder() - .setKey(UnsafeByteOperations.unsafeWrap(key.nioBuffer())) + public static IncrementRequest newIncrementRequest(ByteBuf key, + long amount, + IncrementOption option) { + return new IncrementRequest() + .setKey(key.slice()) .setAmount(amount) .setGetTotal(option.getTotal()); } @@ -148,14 +146,14 @@ public static IncrementResult newIncrementResult( return result; } - public static DeleteRangeRequest.Builder newDeleteRequest(ByteBuf key, DeleteOption option) { - DeleteRangeRequest.Builder builder = DeleteRangeRequest.newBuilder() - .setKey(UnsafeByteOperations.unsafeWrap(key.nioBuffer())) + public static DeleteRangeRequest newDeleteRequest(ByteBuf key, DeleteOption option) { + DeleteRangeRequest request = new DeleteRangeRequest() + .setKey(key.slice()) .setPrevKv(option.prevKv()); if (null != option.endKey()) { - builder = builder.setRangeEnd(UnsafeByteOperations.unsafeWrap(option.endKey().nioBuffer())); + request.setRangeEnd(option.endKey().slice()); } - return builder; + return request; } public static DeleteResult newDeleteResult( @@ -178,7 +176,7 @@ public static CompareTarget toProtoTarget(org.apache.bookkeeper.api.kv.op.Compar case VERSION: return CompareTarget.VERSION; default: - return CompareTarget.UNRECOGNIZED; + throw new IllegalArgumentException("Unknown compare target: " + target); } } @@ -193,83 +191,89 @@ public static CompareResult toProtoResult(org.apache.bookkeeper.api.kv.op.Compar case NOT_EQUAL: return CompareResult.NOT_EQUAL; default: - return CompareResult.UNRECOGNIZED; + throw new IllegalArgumentException("Unknown compare result: " + result); } } - public static Compare.Builder toProtoCompare(CompareOp cmp) { - Compare.Builder builder = Compare.newBuilder() - .setTarget(toProtoTarget(cmp.target())) - .setResult(toProtoResult(cmp.result())) - .setKey(toProtoKey(cmp.key())); + /** + * Populate {@code compare} from a {@link CompareOp}. + */ + public static void populateProtoCompare(Compare compare, CompareOp cmp) { + compare.setTarget(toProtoTarget(cmp.target())); + compare.setResult(toProtoResult(cmp.result())); + compare.setKey(cmp.key().slice()); switch (cmp.target()) { case VERSION: - builder.setVersion(cmp.revision()); + compare.setVersion(cmp.revision()); break; case MOD: - builder.setModRevision(cmp.revision()); + compare.setModRevision(cmp.revision()); break; case CREATE: - builder.setCreateRevision(cmp.revision()); + compare.setCreateRevision(cmp.revision()); break; case VALUE: ByteBuf value = cmp.value(); if (null == value) { value = Unpooled.wrappedBuffer(new byte[0]); } - builder.setValue(toProtoKey(value)); + compare.setValue(value.slice()); break; default: break; } - return builder; } - public static PutRequest.Builder toProtoPutRequest(PutOp op) { - return PutRequest.newBuilder() - .setPrevKv(op.option().prevKv()) - .setKey(toProtoKey(op.key())) - .setValue(toProtoKey(op.value())); + /** + * Populate {@code put} from a {@link PutOp}. + */ + public static void populateProtoPutRequest(PutRequest put, PutOp op) { + put.setPrevKv(op.option().prevKv()); + put.setKey(op.key().slice()); + put.setValue(op.value().slice()); } - public static DeleteRangeRequest.Builder toProtoDeleteRequest(DeleteOp op) { - DeleteRangeRequest.Builder builder = DeleteRangeRequest.newBuilder() - .setKey(toProtoKey(op.key())) - .setPrevKv(op.option().prevKv()); + /** + * Populate {@code req} from a {@link DeleteOp}. + */ + public static void populateProtoDeleteRequest(DeleteRangeRequest req, DeleteOp op) { + req.setKey(op.key().slice()); + req.setPrevKv(op.option().prevKv()); if (null != op.option().endKey()) { - builder.setRangeEnd(toProtoKey(op.option().endKey())); + req.setRangeEnd(op.option().endKey().slice()); } - return builder; } - public static RangeRequest.Builder toProtoRangeRequest(RangeOp op) { - RangeRequest.Builder builder = RangeRequest.newBuilder() - .setKey(toProtoKey(op.key())) - .setCountOnly(op.option().countOnly()) - .setKeysOnly(op.option().keysOnly()) - .setLimit(op.option().limit()); + /** + * Populate {@code req} from a {@link RangeOp}. + */ + public static void populateProtoRangeRequest(RangeRequest req, RangeOp op) { + req.setKey(op.key().slice()); + req.setCountOnly(op.option().countOnly()); + req.setKeysOnly(op.option().keysOnly()); + req.setLimit(op.option().limit()); if (null != op.option().endKey()) { - builder.setRangeEnd(toProtoKey(op.option().endKey())); + req.setRangeEnd(op.option().endKey().slice()); } - return builder; } - public static RequestOp.Builder toProtoRequest(Op op) { - RequestOp.Builder reqBuilder = RequestOp.newBuilder(); + /** + * Populate the inner request inside {@code reqOp} from {@code op}. + */ + public static void populateProtoRequest(RequestOp reqOp, Op op) { switch (op.type()) { case DELETE: - reqBuilder.setRequestDeleteRange(toProtoDeleteRequest((DeleteOp) op)); + populateProtoDeleteRequest(reqOp.setRequestDeleteRange(), (DeleteOp) op); break; case RANGE: - reqBuilder.setRequestRange(toProtoRangeRequest((RangeOp) op)); + populateProtoRangeRequest(reqOp.setRequestRange(), (RangeOp) op); break; case PUT: - reqBuilder.setRequestPut(toProtoPutRequest((PutOp) op)); + populateProtoPutRequest(reqOp.setRequestPut(), (PutOp) op); break; default: throw new IllegalArgumentException("Type '" + op.type() + "' is not supported in a txn yet."); } - return reqBuilder; } public static TxnResult newKvTxnResult( @@ -277,7 +281,7 @@ public static TxnResult newKvTxnResult( ResultFactory resultFactory, KeyValueFactory kvFactory) { TxnResultImpl result = resultFactory.newTxnResult(-1L); - result.isSuccess(txnResponse.getSucceeded()); + result.isSuccess(txnResponse.isSucceeded()); result.results(Lists.transform(txnResponse.getResponsesList(), op -> { switch (op.getResponseCase()) { case RESPONSE_PUT: diff --git a/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufSimpleTableImpl.java b/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufSimpleTableImpl.java index 596ee93a680..ae699c1f398 100644 --- a/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufSimpleTableImpl.java +++ b/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufSimpleTableImpl.java @@ -19,8 +19,8 @@ package org.apache.bookkeeper.clients.impl.kv; -import static org.apache.bookkeeper.clients.impl.kv.KvUtils.toProtoCompare; -import static org.apache.bookkeeper.clients.impl.kv.KvUtils.toProtoRequest; +import static org.apache.bookkeeper.clients.impl.kv.KvUtils.populateProtoCompare; +import static org.apache.bookkeeper.clients.impl.kv.KvUtils.populateProtoRequest; import static org.apache.bookkeeper.common.util.ListenableFutures.fromListenableFuture; import static org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.getDeleteMethod; import static org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.getIncrementMethod; @@ -31,7 +31,6 @@ import static org.apache.bookkeeper.stream.protocol.ProtocolConstants.SID_METADATA_KEY; import com.google.common.collect.Lists; -import com.google.protobuf.UnsafeByteOperations; import io.grpc.CallOptions; import io.grpc.Channel; import io.grpc.ClientCall; @@ -67,6 +66,10 @@ import org.apache.bookkeeper.api.kv.result.TxnResult; import org.apache.bookkeeper.clients.utils.RetryUtils; import org.apache.bookkeeper.stream.proto.StreamProperties; +import org.apache.bookkeeper.stream.proto.kv.rpc.DeleteRangeRequest; +import org.apache.bookkeeper.stream.proto.kv.rpc.IncrementRequest; +import org.apache.bookkeeper.stream.proto.kv.rpc.PutRequest; +import org.apache.bookkeeper.stream.proto.kv.rpc.RangeRequest; import org.apache.bookkeeper.stream.proto.kv.rpc.RoutingHeader; import org.apache.bookkeeper.stream.proto.kv.rpc.TxnRequest; @@ -123,10 +126,13 @@ public PByteBufSimpleTableImpl(StreamProperties streamProps, this.retryUtils = retryUtils; } - private RoutingHeader.Builder newRoutingHeader(ByteBuf pKey) { - return RoutingHeader.newBuilder() - .setStreamId(streamId) - .setRKey(UnsafeByteOperations.unsafeWrap(pKey.nioBuffer())); + private void populateRoutingHeader(RoutingHeader header, ByteBuf pKey) { + header.setStreamId(streamId); + // Use a slice so this header's rKey has its own readerIndex independent of the + // request's key/value fields when those alias the same underlying ByteBuf: + // lightproto's serializer calls ByteBuf#writeBytes(src) which advances src's + // readerIndex, so two fields backed by the same ByteBuf would clobber each other. + header.setRKey(pKey.slice()); } private Channel getChannel(ByteBuf pKey) { @@ -143,13 +149,14 @@ public CompletableFuture> get( if (null != option.endKey()) { option.endKey().retain(); } - return retryUtils.execute(() -> fromListenableFuture( - ClientCalls.futureUnaryCall( - getChannel(pKey).newCall(getRangeMethod(), getCallOptions()), - KvUtils.newRangeRequest(lKey, option) - .setHeader(newRoutingHeader(pKey)) - .build()) - )) + return retryUtils.execute(() -> { + RangeRequest request = KvUtils.newRangeRequest(lKey, option); + populateRoutingHeader(request.setHeader(), pKey); + return fromListenableFuture( + ClientCalls.futureUnaryCall( + getChannel(pKey).newCall(getRangeMethod(), getCallOptions()), + request)); + }) .thenApply(response -> KvUtils.newRangeResult(response, resultFactory, kvFactory)) .whenComplete((value, cause) -> { ReferenceCountUtil.release(pKey); @@ -167,19 +174,20 @@ public CompletableFuture> put( pKey.retain(); lKey.retain(); value.retain(); - return retryUtils.execute(() -> fromListenableFuture( - ClientCalls.futureUnaryCall( - getChannel(pKey).newCall(getPutMethod(), getCallOptions()), - KvUtils.newPutRequest(lKey, value, option) - .setHeader(newRoutingHeader(pKey)) - .build()) - )) - .thenApply(response -> KvUtils.newPutResult(response, resultFactory, kvFactory)) - .whenComplete((ignored, cause) -> { - ReferenceCountUtil.release(pKey); - ReferenceCountUtil.release(lKey); - ReferenceCountUtil.release(value); - }); + return retryUtils.execute(() -> { + PutRequest request = KvUtils.newPutRequest(lKey, value, option); + populateRoutingHeader(request.setHeader(), pKey); + return fromListenableFuture( + ClientCalls.futureUnaryCall( + getChannel(pKey).newCall(getPutMethod(), getCallOptions()), + request)); + }) + .thenApply(response -> KvUtils.newPutResult(response, resultFactory, kvFactory)) + .whenComplete((ignored, cause) -> { + ReferenceCountUtil.release(pKey); + ReferenceCountUtil.release(lKey); + ReferenceCountUtil.release(value); + }); } @Override @@ -191,13 +199,14 @@ public CompletableFuture> delete( if (null != option.endKey()) { option.endKey().retain(); } - return retryUtils.execute(() -> fromListenableFuture( - ClientCalls.futureUnaryCall( - getChannel(pKey).newCall(getDeleteMethod(), getCallOptions()), - KvUtils.newDeleteRequest(lKey, option) - .setHeader(newRoutingHeader(pKey)) - .build()) - )) + return retryUtils.execute(() -> { + DeleteRangeRequest request = KvUtils.newDeleteRequest(lKey, option); + populateRoutingHeader(request.setHeader(), pKey); + return fromListenableFuture( + ClientCalls.futureUnaryCall( + getChannel(pKey).newCall(getDeleteMethod(), getCallOptions()), + request)); + }) .thenApply(response -> KvUtils.newDeleteResult(response, resultFactory, kvFactory)) .whenComplete((ignored, cause) -> { ReferenceCountUtil.release(pKey); @@ -214,13 +223,14 @@ public CompletableFuture> increment( ) { pKey.retain(); lKey.retain(); - return retryUtils.execute(() -> fromListenableFuture( - ClientCalls.futureUnaryCall( - getChannel(pKey).newCall(getIncrementMethod(), getCallOptions()), - KvUtils.newIncrementRequest(lKey, amount, option) - .setHeader(newRoutingHeader(pKey)) - .build()) - )) + return retryUtils.execute(() -> { + IncrementRequest request = KvUtils.newIncrementRequest(lKey, amount, option); + populateRoutingHeader(request.setHeader(), pKey); + return fromListenableFuture( + ClientCalls.futureUnaryCall( + getChannel(pKey).newCall(getIncrementMethod(), getCallOptions()), + request)); + }) .thenApply(response -> KvUtils.newIncrementResult(response, resultFactory, kvFactory)) .whenComplete((ignored, cause) -> { ReferenceCountUtil.release(pKey); @@ -250,12 +260,12 @@ public void close() { class TxnImpl implements Txn { private final ByteBuf pKey; - private final TxnRequest.Builder txnBuilder; + private final TxnRequest txnRequest; private final List resourcesToRelease; TxnImpl(ByteBuf pKey) { this.pKey = pKey.retain(); - this.txnBuilder = TxnRequest.newBuilder(); + this.txnRequest = new TxnRequest(); this.resourcesToRelease = Lists.newArrayList(); } @@ -263,7 +273,7 @@ class TxnImpl implements Txn { @Override public Txn If(CompareOp... cmps) { for (CompareOp cmp : cmps) { - txnBuilder.addCompare(toProtoCompare(cmp)); + populateProtoCompare(txnRequest.addCompare(), cmp); resourcesToRelease.add(cmp); } return this; @@ -273,7 +283,7 @@ public Txn If(CompareOp... cmps) { @Override public Txn Then(Op... ops) { for (Op op : ops) { - txnBuilder.addSuccess(toProtoRequest(op)); + populateProtoRequest(txnRequest.addSuccess(), op); resourcesToRelease.add(op); } return this; @@ -283,7 +293,7 @@ public Txn Then(Op... ops) { @Override public Txn Else(Op... ops) { for (Op op : ops) { - txnBuilder.addFailure(toProtoRequest(op)); + populateProtoRequest(txnRequest.addFailure(), op); resourcesToRelease.add(op); } return this; @@ -291,10 +301,11 @@ public Txn Else(Op... ops) { @Override public CompletableFuture> commit() { + populateRoutingHeader(txnRequest.setHeader(), pKey); return retryUtils.execute(() -> fromListenableFuture( ClientCalls.futureUnaryCall( getChannel(pKey).newCall(getTxnMethod(), getCallOptions()), - txnBuilder.setHeader(newRoutingHeader(pKey)).build()) + txnRequest) )) .thenApply(response -> KvUtils.newKvTxnResult(response, resultFactory, kvFactory)) .whenComplete((ignored, cause) -> { diff --git a/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufTableRangeImpl.java b/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufTableRangeImpl.java index 0b25bbd0d65..2875d906759 100644 --- a/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufTableRangeImpl.java +++ b/stream/clients/java/kv/src/main/java/org/apache/bookkeeper/clients/impl/kv/PByteBufTableRangeImpl.java @@ -14,11 +14,10 @@ package org.apache.bookkeeper.clients.impl.kv; -import static org.apache.bookkeeper.clients.impl.kv.KvUtils.toProtoCompare; -import static org.apache.bookkeeper.clients.impl.kv.KvUtils.toProtoRequest; +import static org.apache.bookkeeper.clients.impl.kv.KvUtils.populateProtoCompare; +import static org.apache.bookkeeper.clients.impl.kv.KvUtils.populateProtoRequest; import com.google.common.collect.Lists; -import com.google.protobuf.UnsafeByteOperations; import io.netty.buffer.ByteBuf; import io.netty.util.ReferenceCountUtil; import java.util.List; @@ -44,6 +43,10 @@ import org.apache.bookkeeper.clients.impl.container.StorageContainerChannel; import org.apache.bookkeeper.common.util.Backoff; import org.apache.bookkeeper.stream.proto.RangeProperties; +import org.apache.bookkeeper.stream.proto.kv.rpc.DeleteRangeRequest; +import org.apache.bookkeeper.stream.proto.kv.rpc.IncrementRequest; +import org.apache.bookkeeper.stream.proto.kv.rpc.PutRequest; +import org.apache.bookkeeper.stream.proto.kv.rpc.RangeRequest; import org.apache.bookkeeper.stream.proto.kv.rpc.RoutingHeader; import org.apache.bookkeeper.stream.proto.kv.rpc.TxnRequest; @@ -80,11 +83,13 @@ class PByteBufTableRangeImpl implements PTable { this.backoffPolicy = backoffPolicy; } - private RoutingHeader.Builder newRoutingHeader(ByteBuf pKey) { - return RoutingHeader.newBuilder() - .setStreamId(streamId) - .setRangeId(rangeProps.getRangeId()) - .setRKey(UnsafeByteOperations.unsafeWrap(pKey.nioBuffer())); + private void populateRoutingHeader(RoutingHeader header, ByteBuf pKey) { + header.setStreamId(streamId); + header.setRangeId(rangeProps.getRangeId()); + // Slice so the rKey has its own readerIndex independent of any other ByteBuf + // fields that alias the same underlying buffer. See PByteBufSimpleTableImpl + // for details. + header.setRKey(pKey.slice()); } @Override @@ -95,10 +100,10 @@ public CompletableFuture> get( if (null != option.endKey()) { option.endKey().retain(); } + RangeRequest request = KvUtils.newRangeRequest(lKey, option); + populateRoutingHeader(request.setHeader(), pKey); return RangeRequestProcessor.of( - KvUtils.newRangeRequest(lKey, option) - .setHeader(newRoutingHeader(pKey)) - .build(), + request, response -> KvUtils.newRangeResult(response, resultFactory, kvFactory), scChannel, executor, @@ -120,10 +125,10 @@ public CompletableFuture> put(ByteBuf pKey, pKey.retain(); lKey.retain(); value.retain(); + PutRequest request = KvUtils.newPutRequest(lKey, value, option); + populateRoutingHeader(request.setHeader(), pKey); return PutRequestProcessor.of( - KvUtils.newPutRequest(lKey, value, option) - .setHeader(newRoutingHeader(pKey)) - .build(), + request, response -> KvUtils.newPutResult(response, resultFactory, kvFactory), scChannel, executor, @@ -144,10 +149,10 @@ public CompletableFuture> delete(ByteBuf pKey, if (null != option.endKey()) { option.endKey().retain(); } + DeleteRangeRequest request = KvUtils.newDeleteRequest(lKey, option); + populateRoutingHeader(request.setHeader(), pKey); return DeleteRequestProcessor.of( - KvUtils.newDeleteRequest(lKey, option) - .setHeader(newRoutingHeader(pKey)) - .build(), + request, response -> KvUtils.newDeleteResult(response, resultFactory, kvFactory), scChannel, executor, @@ -168,10 +173,10 @@ public CompletableFuture> increment(ByteBuf pK IncrementOption option) { pKey.retain(); lKey.retain(); + IncrementRequest request = KvUtils.newIncrementRequest(lKey, amount, option); + populateRoutingHeader(request.setHeader(), pKey); return IncrementRequestProcessor.of( - KvUtils.newIncrementRequest(lKey, amount, option) - .setHeader(newRoutingHeader(pKey)) - .build(), + request, response -> KvUtils.newIncrementResult(response, resultFactory, kvFactory), scChannel, executor, @@ -204,12 +209,12 @@ public OpFactory opFactory() { class TxnImpl implements Txn { private final ByteBuf pKey; - private final TxnRequest.Builder txnBuilder; + private final TxnRequest txnRequest; private final List resourcesToRelease; TxnImpl(ByteBuf pKey) { this.pKey = pKey.retain(); - this.txnBuilder = TxnRequest.newBuilder(); + this.txnRequest = new TxnRequest(); this.resourcesToRelease = Lists.newArrayList(); } @@ -217,7 +222,7 @@ class TxnImpl implements Txn { @Override public Txn If(CompareOp... cmps) { for (CompareOp cmp : cmps) { - txnBuilder.addCompare(toProtoCompare(cmp)); + populateProtoCompare(txnRequest.addCompare(), cmp); resourcesToRelease.add(cmp); } return this; @@ -227,7 +232,7 @@ public Txn If(CompareOp... cmps) { @Override public Txn Then(Op... ops) { for (Op op : ops) { - txnBuilder.addSuccess(toProtoRequest(op)); + populateProtoRequest(txnRequest.addSuccess(), op); resourcesToRelease.add(op); } return this; @@ -237,7 +242,7 @@ public Txn Then(Op... ops) { @Override public Txn Else(Op... ops) { for (Op op : ops) { - txnBuilder.addFailure(toProtoRequest(op)); + populateProtoRequest(txnRequest.addFailure(), op); resourcesToRelease.add(op); } return this; @@ -245,8 +250,9 @@ public Txn Else(Op... ops) { @Override public CompletableFuture> commit() { + populateRoutingHeader(txnRequest.setHeader(), pKey); return TxnRequestProcessor.of( - txnBuilder.setHeader(newRoutingHeader(pKey)).build(), + txnRequest, response -> KvUtils.newKvTxnResult(response, resultFactory, kvFactory), scChannel, executor, diff --git a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/DeleteRequestProcessorTest.java b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/DeleteRequestProcessorTest.java index 8b01b9b3e7e..d93b964ab64 100644 --- a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/DeleteRequestProcessorTest.java +++ b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/DeleteRequestProcessorTest.java @@ -18,7 +18,6 @@ package org.apache.bookkeeper.clients.impl.kv; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertSame; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -37,7 +36,6 @@ import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.stream.proto.kv.rpc.DeleteRangeRequest; import org.apache.bookkeeper.stream.proto.kv.rpc.DeleteRangeResponse; -import org.apache.bookkeeper.stream.proto.kv.rpc.ResponseHeader; import org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.TableServiceImplBase; import org.apache.bookkeeper.stream.proto.storage.StatusCode; import org.junit.Test; @@ -56,16 +54,13 @@ protected void doTeardown() throws Exception { } protected DeleteRangeResponse newSuccessResponse() { - return DeleteRangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.SUCCESS) - .build()) - .build(); + DeleteRangeResponse resp = new DeleteRangeResponse(); + resp.setHeader().setCode(StatusCode.SUCCESS); + return resp; } protected DeleteRangeRequest newRequest() { - return DeleteRangeRequest.newBuilder() - .build(); + return new DeleteRangeRequest(); } @Test @@ -109,7 +104,7 @@ private void complete(StreamObserver responseStreamObserver scheduler, ClientConstants.DEFAULT_INFINIT_BACKOFF_POLICY); assertEquals("test", FutureUtils.result(processor.process())); - assertSame(request, receivedRequest.get()); + assertEquals(request, receivedRequest.get()); } } diff --git a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/IncrementRequestProcessorTest.java b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/IncrementRequestProcessorTest.java index 1b1ec493397..66cf19aec22 100644 --- a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/IncrementRequestProcessorTest.java +++ b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/IncrementRequestProcessorTest.java @@ -18,7 +18,6 @@ package org.apache.bookkeeper.clients.impl.kv; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertSame; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -37,7 +36,6 @@ import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.stream.proto.kv.rpc.IncrementRequest; import org.apache.bookkeeper.stream.proto.kv.rpc.IncrementResponse; -import org.apache.bookkeeper.stream.proto.kv.rpc.ResponseHeader; import org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.TableServiceImplBase; import org.apache.bookkeeper.stream.proto.storage.StatusCode; import org.junit.Test; @@ -56,16 +54,13 @@ protected void doTeardown() throws Exception { } protected IncrementResponse newSuccessResponse() { - return IncrementResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.SUCCESS) - .build()) - .build(); + IncrementResponse resp = new IncrementResponse(); + resp.setHeader().setCode(StatusCode.SUCCESS); + return resp; } protected IncrementRequest newRequest() { - return IncrementRequest.newBuilder() - .build(); + return new IncrementRequest(); } @Test @@ -109,7 +104,7 @@ private void complete(StreamObserver responseStreamObserver) scheduler, ClientConstants.DEFAULT_INFINIT_BACKOFF_POLICY); assertEquals("test", FutureUtils.result(processor.process())); - assertSame(request, receivedRequest.get()); + assertEquals(request, receivedRequest.get()); } } diff --git a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/PutRequestProcessorTest.java b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/PutRequestProcessorTest.java index 3481a26b938..f5261245a5f 100644 --- a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/PutRequestProcessorTest.java +++ b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/PutRequestProcessorTest.java @@ -18,7 +18,6 @@ package org.apache.bookkeeper.clients.impl.kv; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertSame; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -37,7 +36,6 @@ import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.stream.proto.kv.rpc.PutRequest; import org.apache.bookkeeper.stream.proto.kv.rpc.PutResponse; -import org.apache.bookkeeper.stream.proto.kv.rpc.ResponseHeader; import org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.TableServiceImplBase; import org.apache.bookkeeper.stream.proto.storage.StatusCode; import org.junit.Test; @@ -56,16 +54,13 @@ protected void doTeardown() throws Exception { } protected PutResponse newSuccessResponse() { - return PutResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.SUCCESS) - .build()) - .build(); + PutResponse resp = new PutResponse(); + resp.setHeader().setCode(StatusCode.SUCCESS); + return resp; } protected PutRequest newRequest() { - return PutRequest.newBuilder() - .build(); + return new PutRequest(); } @Test @@ -109,7 +104,7 @@ private void complete(StreamObserver responseStreamObserver) { scheduler, ClientConstants.DEFAULT_INFINIT_BACKOFF_POLICY); assertEquals("test", FutureUtils.result(processor.process())); - assertSame(request, receivedRequest.get()); + assertEquals(request, receivedRequest.get()); } } diff --git a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/RangeRequestProcessorTest.java b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/RangeRequestProcessorTest.java index 9e53df7f329..9d1f947b4fc 100644 --- a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/RangeRequestProcessorTest.java +++ b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/RangeRequestProcessorTest.java @@ -18,7 +18,6 @@ package org.apache.bookkeeper.clients.impl.kv; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertSame; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -37,7 +36,6 @@ import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.stream.proto.kv.rpc.RangeRequest; import org.apache.bookkeeper.stream.proto.kv.rpc.RangeResponse; -import org.apache.bookkeeper.stream.proto.kv.rpc.ResponseHeader; import org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.TableServiceImplBase; import org.apache.bookkeeper.stream.proto.storage.StatusCode; import org.junit.Test; @@ -56,16 +54,13 @@ protected void doTeardown() throws Exception { } protected RangeResponse newSuccessResponse() { - return RangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.SUCCESS) - .build()) - .build(); + RangeResponse resp = new RangeResponse(); + resp.setHeader().setCode(StatusCode.SUCCESS); + return resp; } protected RangeRequest newRequest() { - return RangeRequest.newBuilder() - .build(); + return new RangeRequest(); } @Test @@ -109,7 +104,7 @@ private void complete(StreamObserver responseStreamObserver) { scheduler, ClientConstants.DEFAULT_INFINIT_BACKOFF_POLICY); assertEquals("test", FutureUtils.result(processor.process())); - assertSame(request, receivedRequest.get()); + assertEquals(request, receivedRequest.get()); } } diff --git a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TestKvUtils.java b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TestKvUtils.java index 50887ac7ae4..e63ee03cb5f 100644 --- a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TestKvUtils.java +++ b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TestKvUtils.java @@ -19,11 +19,11 @@ import static org.apache.bookkeeper.clients.impl.kv.KvUtils.newIncrementRequest; import static org.apache.bookkeeper.clients.impl.kv.KvUtils.newPutRequest; import static org.apache.bookkeeper.clients.impl.kv.KvUtils.newRangeRequest; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import com.google.protobuf.ByteString; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import org.apache.bookkeeper.api.kv.impl.options.OptionFactoryImpl; @@ -44,12 +44,10 @@ */ public class TestKvUtils { - private static final long scId = System.currentTimeMillis(); - private static final ByteString routingKey = ByteString.copyFrom("test-routing-key", UTF_8); private static final ByteBuf key = Unpooled.wrappedBuffer("test-key".getBytes(UTF_8)); private static final ByteBuf value = Unpooled.wrappedBuffer("test-value".getBytes(UTF_8)); - private static final ByteString keyBs = ByteString.copyFrom("test-key".getBytes(UTF_8)); - private static final ByteString valueBs = ByteString.copyFrom("test-value".getBytes(UTF_8)); + private static final byte[] keyBytes = "test-key".getBytes(UTF_8); + private static final byte[] valueBytes = "test-value".getBytes(UTF_8); private final OptionFactory optionFactory = new OptionFactoryImpl<>(); @@ -65,11 +63,11 @@ public void testNewRangeRequest() { .maxModRev(2345L) .minModRev(1235L) .build()) { - RangeRequest rr = newRangeRequest(key, rangeOption).build(); - assertEquals(keyBs, rr.getKey()); - assertEquals(keyBs, rr.getRangeEnd()); - assertTrue(rr.getCountOnly()); - assertTrue(rr.getKeysOnly()); + RangeRequest rr = newRangeRequest(key, rangeOption); + assertArrayEquals(keyBytes, rr.getKey()); + assertArrayEquals(keyBytes, rr.getRangeEnd()); + assertTrue(rr.isCountOnly()); + assertTrue(rr.isKeysOnly()); assertEquals(10, rr.getLimit()); assertEquals(1234L, rr.getMaxCreateRevision()); assertEquals(234L, rr.getMinCreateRevision()); @@ -82,10 +80,10 @@ public void testNewRangeRequest() { @Test public void testNewPutRequest() { try (PutOption option = Options.putAndGet()) { - PutRequest rr = newPutRequest(key, value, option).build(); - assertEquals(keyBs, rr.getKey()); - assertEquals(valueBs, rr.getValue()); - assertTrue(rr.getPrevKv()); + PutRequest rr = newPutRequest(key, value, option); + assertArrayEquals(keyBytes, rr.getKey()); + assertArrayEquals(valueBytes, rr.getValue()); + assertTrue(rr.isPrevKv()); assertFalse(rr.hasHeader()); } } @@ -93,10 +91,10 @@ public void testNewPutRequest() { @Test public void testNewIncrementRequest() { try (IncrementOption option = Options.incrementAndGet()) { - IncrementRequest rr = newIncrementRequest(key, 100L, option).build(); - assertEquals(keyBs, rr.getKey()); + IncrementRequest rr = newIncrementRequest(key, 100L, option); + assertArrayEquals(keyBytes, rr.getKey()); assertEquals(100L, rr.getAmount()); - assertTrue(rr.getGetTotal()); + assertTrue(rr.isGetTotal()); assertFalse(rr.hasHeader()); } } @@ -107,10 +105,10 @@ public void testNewDeleteRequest() { .endKey(key.retainedDuplicate()) .prevKv(true) .build()) { - DeleteRangeRequest rr = newDeleteRequest(key, option).build(); - assertEquals(keyBs, rr.getKey()); - assertEquals(keyBs, rr.getRangeEnd()); - assertTrue(rr.getPrevKv()); + DeleteRangeRequest rr = newDeleteRequest(key, option); + assertArrayEquals(keyBytes, rr.getKey()); + assertArrayEquals(keyBytes, rr.getRangeEnd()); + assertTrue(rr.isPrevKv()); assertFalse(rr.hasHeader()); } } diff --git a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TestPByteBufTableImpl.java b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TestPByteBufTableImpl.java index d25c3c93540..72cacb70d83 100644 --- a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TestPByteBufTableImpl.java +++ b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TestPByteBufTableImpl.java @@ -93,12 +93,16 @@ public Long getRoutingKey(Integer key) { return keys.get(idx); } }; - private final StreamProperties streamProps = StreamProperties.newBuilder() - .setStorageContainerId(12345L) - .setStreamConf(DEFAULT_STREAM_CONF) - .setStreamId(streamId) - .setStreamName("test-stream") - .build(); + private final StreamProperties streamProps = newStreamProperties(); + + private static StreamProperties newStreamProperties() { + StreamProperties props = new StreamProperties(); + props.setStorageContainerId(12345L); + props.setStreamConf().copyFrom(DEFAULT_STREAM_CONF); + props.setStreamId(streamId); + props.setStreamName("test-stream"); + return props; + } private final MetaRangeClient mockMetaRangeClient = mock(MetaRangeClient.class); private final StorageServerClientManager mockClientManager = mock(StorageServerClientManager.class); private final OptionFactory optionFactory = new OptionFactoryImpl<>(); diff --git a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TxnRequestProcessorTest.java b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TxnRequestProcessorTest.java index 2efecd0e373..6da33ad4d1d 100644 --- a/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TxnRequestProcessorTest.java +++ b/stream/clients/java/kv/src/test/java/org/apache/bookkeeper/clients/impl/kv/TxnRequestProcessorTest.java @@ -18,7 +18,6 @@ package org.apache.bookkeeper.clients.impl.kv; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertSame; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -35,7 +34,6 @@ import org.apache.bookkeeper.clients.impl.container.StorageContainerChannel; import org.apache.bookkeeper.clients.utils.ClientConstants; import org.apache.bookkeeper.common.concurrent.FutureUtils; -import org.apache.bookkeeper.stream.proto.kv.rpc.ResponseHeader; import org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.TableServiceImplBase; import org.apache.bookkeeper.stream.proto.kv.rpc.TxnRequest; import org.apache.bookkeeper.stream.proto.kv.rpc.TxnResponse; @@ -56,16 +54,13 @@ protected void doTeardown() throws Exception { } protected TxnResponse newSuccessResponse() { - return TxnResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.SUCCESS) - .build()) - .build(); + TxnResponse resp = new TxnResponse(); + resp.setHeader().setCode(StatusCode.SUCCESS); + return resp; } protected TxnRequest newRequest() { - return TxnRequest.newBuilder() - .build(); + return new TxnRequest(); } @Test @@ -109,7 +104,7 @@ private void complete(StreamObserver responseStreamObserver) { scheduler, ClientConstants.DEFAULT_INFINIT_BACKOFF_POLICY); assertEquals("test", FutureUtils.result(processor.process())); - assertSame(request, receivedRequest.get()); + assertEquals(request, receivedRequest.get()); } } diff --git a/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/proxy/PingPongServiceTestBase.java b/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/proxy/PingPongServiceTestBase.java index 6f7da168962..d2685575948 100644 --- a/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/proxy/PingPongServiceTestBase.java +++ b/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/proxy/PingPongServiceTestBase.java @@ -145,9 +145,8 @@ public void testUnary() { PingPongServiceBlockingStub clientBlocking = PingPongServiceGrpc.newBlockingStub(clientChannel); long sequence = ThreadLocalRandom.current().nextLong(); - PingRequest request = PingRequest.newBuilder() - .setSequence(sequence) - .build(); + PingRequest request = new PingRequest() + .setSequence(sequence); PongResponse response = clientBlocking.pingPong(request); assertEquals(sequence, response.getLastSequence()); assertEquals(1, response.getNumPingReceived()); @@ -159,9 +158,8 @@ public void testServerStreaming() { PingPongServiceBlockingStub clientBlocking = PingPongServiceGrpc.newBlockingStub(clientChannel); long sequence = ThreadLocalRandom.current().nextLong(100000); - PingRequest request = PingRequest.newBuilder() - .setSequence(sequence) - .build(); + PingRequest request = new PingRequest() + .setSequence(sequence); Iterator respIter = clientBlocking.lotsOfPongs(request); int count = 0; while (respIter.hasNext()) { @@ -197,9 +195,8 @@ public void onCompleted() { }); for (int i = 0; i < numPings; i++) { - PingRequest request = PingRequest.newBuilder() - .setSequence(sequence + i) - .build(); + PingRequest request = new PingRequest() + .setSequence(sequence + i); pinger.onNext(request); } pinger.onCompleted(); @@ -241,9 +238,8 @@ public void onCompleted() { final LinkedBlockingQueue reqQueue = new LinkedBlockingQueue<>(); for (int i = 0; i < numPings; i++) { final long sequence = ThreadLocalRandom.current().nextLong(100000); - PingRequest request = PingRequest.newBuilder() - .setSequence(sequence) - .build(); + PingRequest request = new PingRequest() + .setSequence(sequence); reqQueue.put(request); pinger.onNext(request); } diff --git a/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/GrpcStatsIntegrationTest.java b/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/GrpcStatsIntegrationTest.java index e34f30e1d2d..ecbab6dc426 100644 --- a/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/GrpcStatsIntegrationTest.java +++ b/stream/common/src/test/java/org/apache/bookkeeper/common/grpc/stats/GrpcStatsIntegrationTest.java @@ -167,9 +167,8 @@ private void assertStats(String methodName, @Test public void testUnary() { long sequence = ThreadLocalRandom.current().nextLong(); - PingRequest request = PingRequest.newBuilder() - .setSequence(sequence) - .build(); + PingRequest request = new PingRequest() + .setSequence(sequence); PongResponse response = client.pingPong(request); assertEquals(sequence, response.getLastSequence()); assertEquals(1, response.getNumPingReceived()); @@ -188,9 +187,8 @@ public void testUnary() { @Test public void testServerStreaming() { long sequence = ThreadLocalRandom.current().nextLong(100000); - PingRequest request = PingRequest.newBuilder() - .setSequence(sequence) - .build(); + PingRequest request = new PingRequest() + .setSequence(sequence); Iterator respIter = client.lotsOfPongs(request); int count = 0; while (respIter.hasNext()) { @@ -234,9 +232,8 @@ public void onCompleted() { }); for (int i = 0; i < numPings; i++) { - PingRequest request = PingRequest.newBuilder() - .setSequence(sequence + i) - .build(); + PingRequest request = new PingRequest() + .setSequence(sequence + i); pinger.onNext(request); } pinger.onCompleted(); @@ -287,9 +284,8 @@ public void onCompleted() { final LinkedBlockingQueue reqQueue = new LinkedBlockingQueue<>(); for (int i = 0; i < numPings; i++) { final long sequence = ThreadLocalRandom.current().nextLong(100000); - PingRequest request = PingRequest.newBuilder() - .setSequence(sequence) - .build(); + PingRequest request = new PingRequest() + .setSequence(sequence); reqQueue.put(request); pinger.onNext(request); } diff --git a/stream/proto/pom.xml b/stream/proto/pom.xml index d4e0fe6844d..999ffac5818 100644 --- a/stream/proto/pom.xml +++ b/stream/proto/pom.xml @@ -37,8 +37,20 @@ commons-lang3 - com.google.protobuf - protobuf-java + io.netty + netty-buffer + + + io.grpc + grpc-api + + + io.grpc + grpc-stub + + + io.grpc + grpc-protobuf javax.annotation @@ -61,13 +73,6 @@ - - - kr.motd.maven - os-maven-plugin - ${os-maven-plugin.version} - - org.apache.rat @@ -86,33 +91,11 @@ false - - org.xolstice.maven.plugins - protobuf-maven-plugin - ${protobuf-maven-plugin.version} - - com.google.protobuf:protoc:${protoc.version}:exe:${os.detected.classifier} - grpc-java - io.grpc:protoc-gen-grpc-java:${protoc-gen-grpc-java.version}:exe:${os.detected.classifier} - true - - - - - compile - compile-custom - - - - io.streamnative.lightproto lightproto-maven-plugin ${lightproto-maven-plugin.version} - - ${project.basedir}/src/main/proto-lightproto/cluster.proto - generated-sources/lightproto/java diff --git a/stream/proto/src/main/java/org/apache/bookkeeper/stream/protocol/ProtocolConstants.java b/stream/proto/src/main/java/org/apache/bookkeeper/stream/protocol/ProtocolConstants.java index e8c14efed5a..e47d8e4525e 100644 --- a/stream/proto/src/main/java/org/apache/bookkeeper/stream/protocol/ProtocolConstants.java +++ b/stream/proto/src/main/java/org/apache/bookkeeper/stream/protocol/ProtocolConstants.java @@ -21,14 +21,11 @@ import io.grpc.Metadata; import org.apache.bookkeeper.common.grpc.netty.IdentityBinaryMarshaller; import org.apache.bookkeeper.common.grpc.netty.LongBinaryMarshaller; -import org.apache.bookkeeper.stream.proto.FixedRangeSplitPolicy; import org.apache.bookkeeper.stream.proto.RangeKeyType; import org.apache.bookkeeper.stream.proto.RetentionPolicy; import org.apache.bookkeeper.stream.proto.SegmentRollingPolicy; -import org.apache.bookkeeper.stream.proto.SizeBasedSegmentRollingPolicy; import org.apache.bookkeeper.stream.proto.SplitPolicy; import org.apache.bookkeeper.stream.proto.StreamConfiguration; -import org.apache.bookkeeper.stream.proto.TimeBasedRetentionPolicy; /** * Protocol related constants used across the project. @@ -77,38 +74,47 @@ private ProtocolConstants() { public static final long MIN_DATA_STREAM_ID = 1024L; // default split policy - public static final SplitPolicy DEFAULT_SPLIT_POLICY = - SplitPolicy.newBuilder() - .setFixedRangePolicy( - FixedRangeSplitPolicy.newBuilder() - .setNumRanges(2)) - .build(); + public static final SplitPolicy DEFAULT_SPLIT_POLICY = newDefaultSplitPolicy(); + + private static SplitPolicy newDefaultSplitPolicy() { + SplitPolicy policy = new SplitPolicy(); + policy.setFixedRangePolicy().setNumRanges(2); + return policy; + } + // default rolling policy - public static final SegmentRollingPolicy DEFAULT_SEGMENT_ROLLING_POLICY = - SegmentRollingPolicy.newBuilder() - .setSizePolicy( - SizeBasedSegmentRollingPolicy.newBuilder() - .setMaxSegmentSize(128 * 1024 * 1024)) - .build(); + public static final SegmentRollingPolicy DEFAULT_SEGMENT_ROLLING_POLICY = newDefaultSegmentRollingPolicy(); + + private static SegmentRollingPolicy newDefaultSegmentRollingPolicy() { + SegmentRollingPolicy policy = new SegmentRollingPolicy(); + policy.setSizePolicy().setMaxSegmentSize(128 * 1024 * 1024); + return policy; + } + // default retention policy - public static final RetentionPolicy DEFAULT_RETENTION_POLICY = - RetentionPolicy.newBuilder() - .setTimePolicy( - TimeBasedRetentionPolicy.newBuilder() - .setRetentionMinutes(-1)) - .build(); + public static final RetentionPolicy DEFAULT_RETENTION_POLICY = newDefaultRetentionPolicy(); + + private static RetentionPolicy newDefaultRetentionPolicy() { + RetentionPolicy policy = new RetentionPolicy(); + policy.setTimePolicy().setRetentionMinutes(-1); + return policy; + } + // default stream configuration public static final int INIT_NUM_RANGES = 24; public static final int MIN_NUM_RANGES = 24; - public static final StreamConfiguration DEFAULT_STREAM_CONF = - StreamConfiguration.newBuilder() + public static final StreamConfiguration DEFAULT_STREAM_CONF = newDefaultStreamConf(); + + private static StreamConfiguration newDefaultStreamConf() { + StreamConfiguration conf = new StreamConfiguration() .setKeyType(RangeKeyType.HASH) .setInitialNumRanges(INIT_NUM_RANGES) - .setMinNumRanges(MIN_NUM_RANGES) - .setRetentionPolicy(DEFAULT_RETENTION_POLICY) - .setRollingPolicy(DEFAULT_SEGMENT_ROLLING_POLICY) - .setSplitPolicy(DEFAULT_SPLIT_POLICY) - .build(); + .setMinNumRanges(MIN_NUM_RANGES); + conf.setRetentionPolicy().copyFrom(DEFAULT_RETENTION_POLICY); + conf.setRollingPolicy().copyFrom(DEFAULT_SEGMENT_ROLLING_POLICY); + conf.setSplitPolicy().copyFrom(DEFAULT_SPLIT_POLICY); + return conf; + } // storage container request metadata key public static final String SC_ID_KEY = "bk-rt-sc-id" + Metadata.BINARY_HEADER_SUFFIX; diff --git a/stream/proto/src/main/java/org/apache/bookkeeper/stream/protocol/util/ProtoUtils.java b/stream/proto/src/main/java/org/apache/bookkeeper/stream/protocol/util/ProtoUtils.java index baa735a867d..1ebf64ce078 100644 --- a/stream/proto/src/main/java/org/apache/bookkeeper/stream/protocol/util/ProtoUtils.java +++ b/stream/proto/src/main/java/org/apache/bookkeeper/stream/protocol/util/ProtoUtils.java @@ -29,7 +29,6 @@ import org.apache.bookkeeper.stream.proto.StreamConfiguration; import org.apache.bookkeeper.stream.proto.StreamMetadata.LifecycleState; import org.apache.bookkeeper.stream.proto.StreamMetadata.ServingState; -import org.apache.bookkeeper.stream.proto.StreamName; import org.apache.bookkeeper.stream.proto.StreamProperties; import org.apache.bookkeeper.stream.proto.storage.CreateNamespaceRequest; import org.apache.bookkeeper.stream.proto.storage.CreateStreamRequest; @@ -40,7 +39,6 @@ import org.apache.bookkeeper.stream.proto.storage.GetStorageContainerEndpointRequest; import org.apache.bookkeeper.stream.proto.storage.GetStorageContainerEndpointResponse; import org.apache.bookkeeper.stream.proto.storage.GetStreamRequest; -import org.apache.bookkeeper.stream.proto.storage.OneStorageContainerEndpointRequest; import org.apache.bookkeeper.stream.proto.storage.OneStorageContainerEndpointResponse; import org.apache.bookkeeper.stream.proto.storage.StatusCode; import org.apache.bookkeeper.stream.proto.storage.StorageContainerEndpoint; @@ -142,12 +140,11 @@ public static List split(long streamId, endKey = Long.MAX_VALUE; } long rangeId = nextRangeId++; - RangeProperties props = RangeProperties.newBuilder() + RangeProperties props = new RangeProperties() .setStartHashKey(startKey) .setEndHashKey(endKey) .setStorageContainerId(placementPolicy.placeStreamRange(streamId, rangeId)) - .setRangeId(rangeId) - .build(); + .setRangeId(rangeId); startKey = endKey; ranges.add(props); @@ -162,7 +159,7 @@ public static List split(long streamId, * @return true if the stream is in created state. otherwise, false. */ public static boolean isStreamCreated(LifecycleState state) { - checkArgument(state != LifecycleState.UNRECOGNIZED); + checkArgument(state != null); return LifecycleState.UNINIT != state && LifecycleState.CREATING != state; } @@ -174,7 +171,7 @@ public static boolean isStreamCreated(LifecycleState state) { * @return true if the stream is writable. otherwise, false. */ public static boolean isStreamWritable(ServingState state) { - checkArgument(state != ServingState.UNRECOGNIZED); + checkArgument(state != null); return ServingState.WRITABLE == state; } @@ -190,14 +187,13 @@ public static boolean isStreamWritable(ServingState state) { */ public static GetStorageContainerEndpointRequest createGetStorageContainerEndpointRequest( List> storageContainers) { - GetStorageContainerEndpointRequest.Builder builder = GetStorageContainerEndpointRequest.newBuilder(); + GetStorageContainerEndpointRequest request = new GetStorageContainerEndpointRequest(); for (Revisioned storageContainer : storageContainers) { - builder.addRequests( - OneStorageContainerEndpointRequest.newBuilder() - .setStorageContainer(storageContainer.getValue()) - .setRevision(storageContainer.getRevision())); + request.addRequest() + .setStorageContainer(storageContainer.getValue()) + .setRevision(storageContainer.getRevision()); } - return builder.build(); + return request; } /** @@ -208,15 +204,14 @@ public static GetStorageContainerEndpointRequest createGetStorageContainerEndpoi */ public static GetStorageContainerEndpointResponse createGetStorageContainerEndpointResponse( List endpoints) { - GetStorageContainerEndpointResponse.Builder builder = GetStorageContainerEndpointResponse.newBuilder(); - builder.setStatusCode(StatusCode.SUCCESS); + GetStorageContainerEndpointResponse response = new GetStorageContainerEndpointResponse(); + response.setStatusCode(StatusCode.SUCCESS); for (StorageContainerEndpoint endpoint : endpoints) { - builder.addResponses( - OneStorageContainerEndpointResponse.newBuilder() - .setStatusCode(StatusCode.SUCCESS) - .setEndpoint(endpoint)); + OneStorageContainerEndpointResponse one = response.addResponse(); + one.setStatusCode(StatusCode.SUCCESS); + one.setEndpoint().copyFrom(endpoint); } - return builder.build(); + return response; } @@ -225,16 +220,14 @@ public static GetStorageContainerEndpointResponse createGetStorageContainerEndpo // public static GetActiveRangesRequest createGetActiveRangesRequest(long streamId) { - return GetActiveRangesRequest.newBuilder() - .setStreamId(streamId) - .build(); + return new GetActiveRangesRequest().setStreamId(streamId); } public static GetActiveRangesRequest createGetActiveRangesRequest(StreamProperties streamProps) { - return GetActiveRangesRequest.newBuilder() - .setStreamId(streamProps.getStreamId()) - .setStreamProps(streamProps) - .build(); + GetActiveRangesRequest request = new GetActiveRangesRequest() + .setStreamId(streamProps.getStreamId()); + request.setStreamProps().copyFrom(streamProps); + return request; } // @@ -250,10 +243,9 @@ public static GetActiveRangesRequest createGetActiveRangesRequest(StreamProperti */ public static CreateNamespaceRequest createCreateNamespaceRequest(String nsName, NamespaceConfiguration nsConf) { - return CreateNamespaceRequest.newBuilder() - .setName(nsName) - .setNsConf(nsConf) - .build(); + CreateNamespaceRequest request = new CreateNamespaceRequest().setName(nsName); + request.setNsConf().copyFrom(nsConf); + return request; } /** @@ -263,9 +255,7 @@ public static CreateNamespaceRequest createCreateNamespaceRequest(String nsName, * @return a delete namespace request. */ public static DeleteNamespaceRequest createDeleteNamespaceRequest(String colName) { - return DeleteNamespaceRequest.newBuilder() - .setName(colName) - .build(); + return new DeleteNamespaceRequest().setName(colName); } /** @@ -275,9 +265,7 @@ public static DeleteNamespaceRequest createDeleteNamespaceRequest(String colName * @return a get namespace request. */ public static GetNamespaceRequest createGetNamespaceRequest(String colName) { - return GetNamespaceRequest.newBuilder() - .setName(colName) - .build(); + return new GetNamespaceRequest().setName(colName); } // @@ -295,11 +283,11 @@ public static GetNamespaceRequest createGetNamespaceRequest(String colName) { public static CreateStreamRequest createCreateStreamRequest(String nsName, String streamName, StreamConfiguration streamConf) { - return CreateStreamRequest.newBuilder() + CreateStreamRequest request = new CreateStreamRequest() .setNsName(nsName) - .setName(streamName) - .setStreamConf(streamConf) - .build(); + .setName(streamName); + request.setStreamConf().copyFrom(streamConf); + return request; } /** @@ -311,11 +299,11 @@ public static CreateStreamRequest createCreateStreamRequest(String nsName, */ public static GetStreamRequest createGetStreamRequest(String nsName, String streamName) { - return GetStreamRequest.newBuilder() - .setStreamName(StreamName.newBuilder() - .setNamespaceName(nsName) - .setStreamName(streamName)) - .build(); + GetStreamRequest request = new GetStreamRequest(); + request.setStreamName() + .setNamespaceName(nsName) + .setStreamName(streamName); + return request; } /** @@ -325,9 +313,7 @@ public static GetStreamRequest createGetStreamRequest(String nsName, * @return a create stream request. */ public static GetStreamRequest createGetStreamRequest(long streamId) { - return GetStreamRequest.newBuilder() - .setStreamId(streamId) - .build(); + return new GetStreamRequest().setStreamId(streamId); } /** @@ -339,10 +325,7 @@ public static GetStreamRequest createGetStreamRequest(long streamId) { */ public static DeleteStreamRequest createDeleteStreamRequest(String nsName, String streamName) { - return DeleteStreamRequest.newBuilder() - .setName(streamName) - .setNsName(nsName) - .build(); + return new DeleteStreamRequest().setName(streamName).setNsName(nsName); } } diff --git a/stream/proto/src/main/proto-lightproto/cluster.proto b/stream/proto/src/main/proto/cluster.proto similarity index 100% rename from stream/proto/src/main/proto-lightproto/cluster.proto rename to stream/proto/src/main/proto/cluster.proto diff --git a/stream/proto/src/test/java/org/apache/bookkeeper/stream/protocol/util/TestProtoUtils.java b/stream/proto/src/test/java/org/apache/bookkeeper/stream/protocol/util/TestProtoUtils.java index 9d1b7289246..cd6d0c67d31 100644 --- a/stream/proto/src/test/java/org/apache/bookkeeper/stream/protocol/util/TestProtoUtils.java +++ b/stream/proto/src/test/java/org/apache/bookkeeper/stream/protocol/util/TestProtoUtils.java @@ -108,12 +108,11 @@ public void testKeyRangeOverlaps2() { } private static RangeProperties createRangeMeta(long startKey, long endKey) { - return RangeProperties.newBuilder() + return new RangeProperties() .setStartHashKey(startKey) .setEndHashKey(endKey) .setStorageContainerId(1234L) - .setRangeId(1234L) - .build(); + .setRangeId(1234L); } @Test @@ -194,9 +193,8 @@ public void testIsStreamWritable() { @Test public void testCreateCreateNamespaceRequest() { - NamespaceConfiguration nsConf = NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build(); + NamespaceConfiguration nsConf = new NamespaceConfiguration(); + nsConf.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); CreateNamespaceRequest request = createCreateNamespaceRequest( name.getMethodName(), nsConf); diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java index 6b6d68be5ec..5b9de67f585 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/cluster/StreamCluster.java @@ -273,12 +273,10 @@ private void createDefaultNamespaces() throws Exception { .attr("namespace", namespaceName) .log("Creating namespace ..."); try { + NamespaceConfiguration nsConf = new NamespaceConfiguration(); + nsConf.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); NamespaceProperties nsProps = result( - admin.createNamespace( - namespaceName, - NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build())); + admin.createNamespace(namespaceName, nsConf)); log.info() .attr("namespace", namespaceName) .log("Successfully created namespace"); diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/StorageServer.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/StorageServer.java index 3132f20f0e3..1a9ef6e1f1a 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/StorageServer.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/StorageServer.java @@ -128,10 +128,9 @@ public static Endpoint createLocalEndpoint(int port, boolean useHostname) throws log.warn() .attr("hostname", hostname) .log("Decided to use hostname"); - return Endpoint.newBuilder() + return new Endpoint() .setHostname(hostname) - .setPort(port) - .build(); + .setPort(port); } public static void main(String[] args) { diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcStorageContainerService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcStorageContainerService.java index bffbbb0a7fa..c64776a8249 100644 --- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcStorageContainerService.java +++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/grpc/GrpcStorageContainerService.java @@ -40,28 +40,24 @@ class GrpcStorageContainerService extends StorageContainerServiceImplBase { @Override public void getStorageContainerEndpoint(GetStorageContainerEndpointRequest request, StreamObserver responseObserver) { - GetStorageContainerEndpointResponse.Builder responseBuilder = GetStorageContainerEndpointResponse.newBuilder() + GetStorageContainerEndpointResponse response = new GetStorageContainerEndpointResponse() .setStatusCode(StatusCode.SUCCESS); for (int i = 0; i < request.getRequestsCount(); i++) { Endpoint endpoint = storageContainerStore .getRoutingService() - .getStorageContainer(request.getRequests(i).getStorageContainer()); - OneStorageContainerEndpointResponse.Builder oneRespBuilder; + .getStorageContainer(request.getRequestAt(i).getStorageContainer()); + OneStorageContainerEndpointResponse oneResp = response.addResponse(); if (null != endpoint) { - oneRespBuilder = OneStorageContainerEndpointResponse.newBuilder() - .setStatusCode(StatusCode.SUCCESS) - .setEndpoint( - StorageContainerEndpoint.newBuilder() - .setRwEndpoint(endpoint) - .addRoEndpoint(endpoint) - .setRevision(0L)); + oneResp.setStatusCode(StatusCode.SUCCESS); + StorageContainerEndpoint sce = oneResp.setEndpoint(); + sce.setRwEndpoint().copyFrom(endpoint); + sce.addRoEndpoint().copyFrom(endpoint); + sce.setRevision(0L); } else { - oneRespBuilder = OneStorageContainerEndpointResponse.newBuilder() - .setStatusCode(StatusCode.INTERNAL_SERVER_ERROR); + oneResp.setStatusCode(StatusCode.INTERNAL_SERVER_ERROR); } - responseBuilder = responseBuilder.addResponses(oneRespBuilder); } - responseObserver.onNext(responseBuilder.build()); + responseObserver.onNext(response); responseObserver.onCompleted(); } diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCRecordCoder.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCRecordCoder.java index 44628a22b72..b3d405c4073 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCRecordCoder.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCRecordCoder.java @@ -18,13 +18,9 @@ package org.apache.bookkeeper.statelib.impl.mvcc; -import com.google.protobuf.CodedOutputStream; -import com.google.protobuf.InvalidProtocolBufferException; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; import io.netty.util.ReferenceCountUtil; -import java.io.IOException; -import java.nio.ByteBuffer; import lombok.AccessLevel; import lombok.NoArgsConstructor; import org.apache.bookkeeper.common.coder.Coder; @@ -44,15 +40,18 @@ public static MVCCRecordCoder of() { private static final MVCCRecordCoder INSTANCE = new MVCCRecordCoder(); - @Override - public byte[] encode(MVCCRecord record) { - KeyMeta meta = KeyMeta.newBuilder() + private static KeyMeta toKeyMeta(MVCCRecord record) { + return new KeyMeta() .setCreateRevision(record.getCreateRev()) .setModRevision(record.getModRev()) .setVersion(record.getVersion()) .setValueType(record.getValueType()) - .setExpireTime(record.getExpireTime()) - .build(); + .setExpireTime(record.getExpireTime()); + } + + @Override + public byte[] encode(MVCCRecord record) { + KeyMeta meta = toKeyMeta(record); int metaLen = meta.getSerializedSize(); int valLen = record.getValue().readableBytes(); @@ -68,13 +67,12 @@ public byte[] encode(MVCCRecord record) { ByteBuf buf = Unpooled.wrappedBuffer(data); buf.writerIndex(0); buf.writeInt(metaLen); - CodedOutputStream out = CodedOutputStream.newInstance(data, Integer.BYTES, metaLen); try { - meta.writeTo(out); - } catch (IOException e) { + meta.writeTo(buf); + } catch (RuntimeException e) { + ReferenceCountUtil.release(buf); throw new StateStoreRuntimeException("Failed to serialize key metadata", e); } - buf.writerIndex(buf.writerIndex() + metaLen); buf.writeInt(valLen); buf.writeBytes(record.getValue().slice()); ReferenceCountUtil.release(buf); @@ -89,14 +87,7 @@ public void encode(MVCCRecord value, ByteBuf destBuf) { @Override public int getSerializedSize(MVCCRecord record) { - KeyMeta meta = KeyMeta.newBuilder() - .setCreateRevision(record.getCreateRev()) - .setModRevision(record.getModRev()) - .setVersion(record.getVersion()) - .setValueType(record.getValueType()) - .setExpireTime(record.getExpireTime()) - .build(); - int metaLen = meta.getSerializedSize(); + int metaLen = toKeyMeta(record).getSerializedSize(); int valLen = record.getValue().readableBytes(); return Integer.BYTES // meta len @@ -110,14 +101,12 @@ public MVCCRecord decode(ByteBuf data) { ByteBuf copy = data.slice(); int metaLen = copy.readInt(); - ByteBuffer metaBuf = copy.slice(copy.readerIndex(), metaLen).nioBuffer(); - KeyMeta meta; + KeyMeta meta = new KeyMeta(); try { - meta = KeyMeta.parseFrom(metaBuf); - } catch (InvalidProtocolBufferException e) { + meta.parseFrom(copy, metaLen); + } catch (RuntimeException e) { throw new StateStoreRuntimeException("Failed to deserialize key metadata", e); } - copy.skipBytes(metaLen); int valLen = copy.readInt(); ByteBuf valBuf = copy.retainedSlice(copy.readerIndex(), valLen); diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCUtils.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCUtils.java index fa2a4f14125..145276fbe60 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCUtils.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/MVCCUtils.java @@ -18,15 +18,8 @@ package org.apache.bookkeeper.statelib.impl.mvcc; -import com.google.common.collect.Lists; -import com.google.protobuf.InvalidProtocolBufferException; -import com.google.protobuf.UnsafeByteOperations; import io.netty.buffer.ByteBuf; -import io.netty.buffer.ByteBufOutputStream; import io.netty.buffer.Unpooled; -import java.io.IOException; -import java.util.Collections; -import java.util.List; import java.util.concurrent.CompletableFuture; import lombok.AccessLevel; import lombok.CustomLog; @@ -56,7 +49,6 @@ import org.apache.bookkeeper.stream.proto.kv.rpc.RequestOp; import org.apache.bookkeeper.stream.proto.kv.rpc.TxnRequest; import org.apache.bookkeeper.stream.proto.kv.store.Command; -import org.apache.bookkeeper.stream.proto.kv.store.NopRequest; /** * Utils for mvcc stores. @@ -65,17 +57,20 @@ @NoArgsConstructor(access = AccessLevel.PRIVATE) public final class MVCCUtils { - public static final Command NOP_CMD = Command.newBuilder() - .setNopReq(NopRequest.newBuilder().build()) - .build(); + public static final Command NOP_CMD = newNopCommand(); + + private static Command newNopCommand() { + Command cmd = new Command(); + cmd.setNopReq(); + return cmd; + } static PutRequest toPutRequest(PutOp op) { - PutRequest.Builder reqBuilder = PutRequest.newBuilder() - .setKey(UnsafeByteOperations.unsafeWrap(op.key())) - .setValue(UnsafeByteOperations.unsafeWrap(op.value())) + return new PutRequest() + .setKey(op.key()) + .setValue(op.value()) .setLease(0) .setPrevKv(op.option().prevKv()); - return reqBuilder.build(); } static DeleteRangeRequest toDeleteRequest(DeleteOp op) { @@ -88,11 +83,10 @@ static DeleteRangeRequest toDeleteRequest(DeleteOp op) { endKey = Constants.NULL_END_KEY; } - DeleteRangeRequest.Builder reqBuilder = DeleteRangeRequest.newBuilder() - .setKey(UnsafeByteOperations.unsafeWrap(key)) - .setRangeEnd(UnsafeByteOperations.unsafeWrap(endKey)); - - return reqBuilder.setPrevKv(op.option().prevKv()).build(); + return new DeleteRangeRequest() + .setKey(key) + .setRangeEnd(endKey) + .setPrevKv(op.option().prevKv()); } static RangeRequest toRangeRequest(RangeOp op) { @@ -105,46 +99,37 @@ static RangeRequest toRangeRequest(RangeOp op) { endKey = Constants.NULL_END_KEY; } - RangeRequest.Builder reqBuilder = RangeRequest.newBuilder() - .setKey(UnsafeByteOperations.unsafeWrap(key)) - .setRangeEnd(UnsafeByteOperations.unsafeWrap(endKey)) + return new RangeRequest() + .setKey(key) + .setRangeEnd(endKey) .setMaxCreateRevision(op.option().maxCreateRev()) .setMinCreateRevision(op.option().minCreateRev()) .setMaxModRevision(op.option().maxModRev()) .setMinModRevision(op.option().minModRev()) .setCountOnly(false); - - return reqBuilder.build(); } - private static List toRequestOpList(List> ops) { + private static void appendRequestOps(TxnRequest target, java.util.List> ops, boolean success) { if (ops == null) { - return Collections.emptyList(); + return; } - List requestOps = Lists.newArrayListWithExpectedSize(ops.size()); for (Op op : ops) { + RequestOp reqOp = success ? target.addSuccess() : target.addFailure(); switch (op.type()) { case PUT: - requestOps.add(RequestOp.newBuilder() - .setRequestPut(toPutRequest((PutOp) op)) - .build()); + reqOp.setRequestPut().copyFrom(toPutRequest((PutOp) op)); break; case DELETE: - requestOps.add(RequestOp.newBuilder() - .setRequestDeleteRange(toDeleteRequest((DeleteOp) op)) - .build()); + reqOp.setRequestDeleteRange().copyFrom(toDeleteRequest((DeleteOp) op)); break; case RANGE: - requestOps.add(RequestOp.newBuilder() - .setRequestRange(toRangeRequest((RangeOp) op)) - .build()); + reqOp.setRequestRange().copyFrom(toRangeRequest((RangeOp) op)); break; default: throw new IllegalArgumentException("Unknown request " + op.type() + " found in a txn request"); } } - return requestOps; } public static Op toApiOp(RequestOp protoOp) { @@ -161,38 +146,37 @@ public static Op toApiOp(RequestOp protoOp) { } } - private static List toCompareList(List> ops) { - List compares = Lists.newArrayListWithExpectedSize(ops.size()); + private static void appendCompares(TxnRequest target, java.util.List> ops) { + if (ops == null) { + return; + } for (CompareOp op : ops) { - compares.add(toCompare(op)); + populateCompare(target.addCompare(), op); } - return compares; } - private static Compare toCompare(CompareOp op) { - Compare.Builder compareBuilder = Compare.newBuilder(); - compareBuilder.setTarget(toProtoCompareTarget(op.target())); - compareBuilder.setResult(toProtoCompareResult(op.result())); - compareBuilder.setKey(UnsafeByteOperations.unsafeWrap(op.key())); + private static void populateCompare(Compare compare, CompareOp op) { + compare.setTarget(toProtoCompareTarget(op.target())); + compare.setResult(toProtoCompareResult(op.result())); + compare.setKey(op.key()); switch (op.target()) { case MOD: - compareBuilder.setModRevision(op.revision()); + compare.setModRevision(op.revision()); break; case CREATE: - compareBuilder.setCreateRevision(op.revision()); + compare.setCreateRevision(op.revision()); break; case VERSION: - compareBuilder.setVersion(op.revision()); + compare.setVersion(op.revision()); break; case VALUE: if (op.value() != null) { - compareBuilder.setValue(UnsafeByteOperations.unsafeWrap(op.value())); + compare.setValue(op.value()); } break; default: throw new IllegalArgumentException("Invalid compare target " + op.target()); } - return compareBuilder.build(); } private static Compare.CompareTarget toProtoCompareTarget(CompareTarget target) { @@ -256,47 +240,46 @@ public static CompareResult toApiCompareResult(Compare.CompareResult result) { } static TxnRequest toTxnRequest(TxnOp op) { - return TxnRequest.newBuilder() - .addAllSuccess(toRequestOpList(op.successOps())) - .addAllFailure(toRequestOpList(op.failureOps())) - .addAllCompare(toCompareList(op.compareOps())) - .build(); + TxnRequest req = new TxnRequest(); + appendRequestOps(req, op.successOps(), true); + appendRequestOps(req, op.failureOps(), false); + appendCompares(req, op.compareOps()); + return req; } static IncrementRequest toIncrementRequest(IncrementOp op) { - return IncrementRequest.newBuilder() - .setKey(UnsafeByteOperations.unsafeWrap(op.key())) + return new IncrementRequest() + .setKey(op.key()) .setAmount(op.amount()) - .setGetTotal(op.option().getTotal()) - .build(); + .setGetTotal(op.option().getTotal()); } static Command toCommand(Op op) { - Command.Builder cmdBuilder = Command.newBuilder(); + Command cmd = new Command(); switch (op.type()) { case PUT: - cmdBuilder.setPutReq(toPutRequest((PutOp) op)); + cmd.setPutReq().copyFrom(toPutRequest((PutOp) op)); break; case DELETE: - cmdBuilder.setDeleteReq(toDeleteRequest((DeleteOp) op)); + cmd.setDeleteReq().copyFrom(toDeleteRequest((DeleteOp) op)); break; case TXN: - cmdBuilder.setTxnReq(toTxnRequest((TxnOp) op)); + cmd.setTxnReq().copyFrom(toTxnRequest((TxnOp) op)); break; case INCREMENT: - cmdBuilder.setIncrReq(toIncrementRequest((IncrementOp) op)); + cmd.setIncrReq().copyFrom(toIncrementRequest((IncrementOp) op)); break; default: throw new IllegalArgumentException("Unknown command type " + op.type()); } - return cmdBuilder.build(); + return cmd; } public static ByteBuf newLogRecordBuf(Command command) { ByteBuf buf = Unpooled.buffer(command.getSerializedSize()); try { - command.writeTo(new ByteBufOutputStream(buf)); - } catch (IOException e) { + command.writeTo(buf); + } catch (RuntimeException e) { throw new StateStoreRuntimeException("Invalid command : " + command, e); } return buf; @@ -304,8 +287,10 @@ public static ByteBuf newLogRecordBuf(Command command) { static Command newCommand(ByteBuf recordBuf) { try { - return Command.parseFrom(recordBuf.nioBuffer()); - } catch (InvalidProtocolBufferException e) { + Command cmd = new Command(); + cmd.parseFrom(recordBuf, recordBuf.readableBytes()); + return cmd; + } catch (RuntimeException e) { log.error().exception(e).log("Found a corrupted record on replaying log stream"); throw new StateStoreRuntimeException("Found a corrupted record on replaying log stream", e); } diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoCompareImpl.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoCompareImpl.java index 569f9d19b4f..c091d9af987 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoCompareImpl.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoCompareImpl.java @@ -22,7 +22,6 @@ import static org.apache.bookkeeper.statelib.impl.mvcc.MVCCUtils.toApiCompareResult; import static org.apache.bookkeeper.statelib.impl.mvcc.MVCCUtils.toApiCompareTarget; -import com.google.protobuf.ByteString; import io.netty.util.Recycler; import io.netty.util.Recycler.Handle; import lombok.AccessLevel; @@ -93,10 +92,11 @@ public byte[] key() { if (null != key) { return key; } - if (ByteString.EMPTY == request.getKey()) { + byte[] reqKey = request.getKey(); + if (reqKey == null || reqKey.length == 0) { key = null; } else { - key = request.getKey().toByteArray(); + key = reqKey; } return key; } @@ -106,10 +106,11 @@ public byte[] value() { if (null != value) { return value; } - if (ByteString.EMPTY == request.getValue()) { + byte[] reqValue = request.getValue(); + if (reqValue == null || reqValue.length == 0) { value = null; } else { - value = request.getValue().toByteArray(); + value = reqValue; } return value; } diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoDeleteOpImpl.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoDeleteOpImpl.java index 130d06cb468..9ae6398cab5 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoDeleteOpImpl.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoDeleteOpImpl.java @@ -70,7 +70,7 @@ public byte[] key() { if (null == req.getKey()) { key = null; } else { - key = req.getKey().toByteArray(); + key = req.getKey(); } return key; } @@ -80,19 +80,20 @@ public byte[] endKey() { if (null != endKey) { return endKey; } - if (null == req.getRangeEnd() - || 0 == req.getRangeEnd().size() - || (1 == req.getRangeEnd().size() && req.getRangeEnd().byteAt(0) == 0)) { + byte[] rangeEnd = req.getRangeEnd(); + if (null == rangeEnd + || 0 == rangeEnd.length + || (1 == rangeEnd.length && rangeEnd[0] == 0)) { endKey = null; } else { - endKey = req.getRangeEnd().toByteArray(); + endKey = rangeEnd; } return endKey; } @Override public boolean prevKv() { - return req.getPrevKv(); + return req.isPrevKv(); } @Override diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoIncrementOpImpl.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoIncrementOpImpl.java index 99ae5df4f75..d881a4bbb30 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoIncrementOpImpl.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoIncrementOpImpl.java @@ -73,7 +73,7 @@ public byte[] key() { if (null != key) { return key; } - key = req.getKey().toByteArray(); + key = req.getKey(); return key; } @@ -90,6 +90,6 @@ public void close() { @Override public boolean getTotal() { - return req.getGetTotal(); + return req.isGetTotal(); } } diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoPutOpImpl.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoPutOpImpl.java index cdde2458b12..98ec14bbff1 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoPutOpImpl.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoPutOpImpl.java @@ -66,7 +66,7 @@ public byte[] value() { if (null != value) { return value; } - value = req.getValue().toByteArray(); + value = req.getValue(); return value; } @@ -85,7 +85,7 @@ public void setPutRequest(PutRequest request) { @Override public boolean prevKv() { - return req.getPrevKv(); + return req.isPrevKv(); } @Override @@ -98,7 +98,7 @@ public byte[] key() { if (null != key) { return key; } - key = req.getKey().toByteArray(); + key = req.getKey(); return key; } diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoRangeOpImpl.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoRangeOpImpl.java index 74a849b6788..3098a055c7c 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoRangeOpImpl.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoRangeOpImpl.java @@ -76,7 +76,7 @@ public byte[] key() { if (null == req.getKey()) { key = null; } else { - key = req.getKey().toByteArray(); + key = req.getKey(); } return key; } @@ -91,12 +91,13 @@ public byte[] endKey() { if (null != endKey) { return endKey; } - if (null == req.getRangeEnd() - || 0 == req.getRangeEnd().size() - || (1 == req.getRangeEnd().size() && req.getRangeEnd().byteAt(0) == 0)) { + byte[] rangeEnd = req.getRangeEnd(); + if (null == rangeEnd + || 0 == rangeEnd.length + || (1 == rangeEnd.length && rangeEnd[0] == 0)) { endKey = null; } else { - endKey = req.getRangeEnd().toByteArray(); + endKey = rangeEnd; } return endKey; } @@ -134,7 +135,7 @@ public boolean keysOnly() { @Override public boolean countOnly() { - return req.getCountOnly(); + return req.isCountOnly(); } @Override diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoTxnOpImpl.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoTxnOpImpl.java index 37f675c4e5e..cb9228a5032 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoTxnOpImpl.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoTxnOpImpl.java @@ -48,17 +48,17 @@ public static ProtoTxnOpImpl newTxnOp(TxnRequest request) { ProtoTxnOpImpl op = RECYCLER.get(); op.setRequest(request); RecyclableArrayList> compareOps = COMPARE_OPS_RECYCLER.newInstance(); - for (Compare compare : request.getCompareList()) { + for (Compare compare : request.getComparesList()) { compareOps.add(ProtoCompareImpl.newCompareOp(compare)); } op.setCompareOps(compareOps); RecyclableArrayList> successOps = OPS_RECYCLER.newInstance(); - for (RequestOp reqOp : request.getSuccessList()) { + for (RequestOp reqOp : request.getSuccessesList()) { successOps.add(toApiOp(reqOp)); } op.setSuccessOps(successOps); RecyclableArrayList> failureOps = OPS_RECYCLER.newInstance(); - for (RequestOp reqOp : request.getFailureList()) { + for (RequestOp reqOp : request.getFailuresList()) { failureOps.add(toApiOp(reqOp)); } return op; diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointFile.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointFile.java index 8df0fda4250..40b41f3229c 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointFile.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointFile.java @@ -264,9 +264,8 @@ public void copyFromRemote(CheckpointStore checkpointStore, public FileInfo getFileInfo() { - return FileInfo.newBuilder() + return new FileInfo() .setName(file.getName()) - .setChecksum(checksum) - .build(); + .setChecksum(checksum); } } diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointInfo.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointInfo.java index b598a1f227e..2b84b1283c9 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointInfo.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/CheckpointInfo.java @@ -48,7 +48,8 @@ public class CheckpointInfo implements Comparable { public CheckpointInfo(String id, InputStream is) throws IOException { this.id = id; - this.metadata = CheckpointMetadata.parseFrom(is); + this.metadata = new CheckpointMetadata(); + this.metadata.parseFrom(is.readAllBytes()); } public CheckpointInfo(String id) { diff --git a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksdbCheckpointTask.java b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksdbCheckpointTask.java index 6e1d5be0d5f..d93d3a63670 100644 --- a/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksdbCheckpointTask.java +++ b/stream/statelib/src/main/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksdbCheckpointTask.java @@ -20,7 +20,6 @@ import com.google.common.collect.Sets; import com.google.common.io.MoreFiles; import com.google.common.io.RecursiveDeleteOption; -import com.google.protobuf.UnsafeByteOperations; import java.io.File; import java.io.IOException; import java.io.OutputStream; @@ -177,21 +176,21 @@ private void finalizeCheckpoint(List files, String checkpointId, byte[] txid) throws IOException { - CheckpointMetadata.Builder metadataBuilder = CheckpointMetadata.newBuilder(); + CheckpointMetadata metadata = new CheckpointMetadata(); for (CheckpointFile file : files) { if (checkpointChecksumEnable) { - metadataBuilder.addFileInfos(file.getFileInfo()); + metadata.addFileInfo().copyFrom(file.getFileInfo()); } - metadataBuilder.addFiles(file.getName()); + metadata.addFile(file.getName()); } if (null != txid) { - metadataBuilder.setTxid(UnsafeByteOperations.unsafeWrap(txid)); + metadata.setTxid(txid); } - metadataBuilder.setCreatedAt(System.currentTimeMillis()); + metadata.setCreatedAt(System.currentTimeMillis()); String destCheckpointPath = RocksUtils.getDestCheckpointMetadataPath(dbPrefix, checkpointId); try (OutputStream os = checkpointStore.openOutputStream(destCheckpointPath)) { - os.write(metadataBuilder.build().toByteArray()); + os.write(metadata.toByteArray()); } } diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoCompareImplTest.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoCompareImplTest.java index 6e50bafacd1..d1b5ddea2c0 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoCompareImplTest.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/mvcc/op/proto/ProtoCompareImplTest.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; -import com.google.protobuf.ByteString; import lombok.Cleanup; import org.apache.bookkeeper.stream.proto.kv.rpc.Compare; import org.apache.bookkeeper.stream.proto.kv.rpc.Compare.CompareResult; @@ -35,19 +34,18 @@ */ public class ProtoCompareImplTest { - private static final ByteString KEY = ByteString.copyFromUtf8("test-key"); - private static final ByteString VAL = ByteString.copyFromUtf8("test-value"); + private static final byte[] KEY = "test-key".getBytes(UTF_8); + private static final byte[] VAL = "test-value".getBytes(UTF_8); private static final long MOD_REV = System.currentTimeMillis(); private static final long CREATE_REV = MOD_REV + 1; private static final long VERSION = CREATE_REV + 1; @Test public void testCompareEmptyValue() { - Compare compare = Compare.newBuilder() + Compare compare = new Compare() .setKey(KEY) .setResult(CompareResult.EQUAL) - .setTarget(CompareTarget.VALUE) - .build(); + .setTarget(CompareTarget.VALUE); @Cleanup ProtoCompareImpl protoCompare = ProtoCompareImpl.newCompareOp(compare); assertArrayEquals("test-key".getBytes(UTF_8), protoCompare.key()); @@ -58,12 +56,11 @@ public void testCompareEmptyValue() { @Test public void testCompareValue() { - Compare compare = Compare.newBuilder() + Compare compare = new Compare() .setKey(KEY) .setValue(VAL) .setResult(CompareResult.EQUAL) - .setTarget(CompareTarget.VALUE) - .build(); + .setTarget(CompareTarget.VALUE); @Cleanup ProtoCompareImpl protoCompare = ProtoCompareImpl.newCompareOp(compare); assertArrayEquals("test-key".getBytes(UTF_8), protoCompare.key()); @@ -74,12 +71,11 @@ public void testCompareValue() { @Test public void testCompareMod() { - Compare compare = Compare.newBuilder() + Compare compare = new Compare() .setKey(KEY) .setModRevision(MOD_REV) .setResult(CompareResult.EQUAL) - .setTarget(CompareTarget.MOD) - .build(); + .setTarget(CompareTarget.MOD); @Cleanup ProtoCompareImpl protoCompare = ProtoCompareImpl.newCompareOp(compare); assertArrayEquals("test-key".getBytes(UTF_8), protoCompare.key()); @@ -90,12 +86,11 @@ public void testCompareMod() { @Test public void testCompareCreate() { - Compare compare = Compare.newBuilder() + Compare compare = new Compare() .setKey(KEY) .setCreateRevision(CREATE_REV) .setResult(CompareResult.EQUAL) - .setTarget(CompareTarget.CREATE) - .build(); + .setTarget(CompareTarget.CREATE); @Cleanup ProtoCompareImpl protoCompare = ProtoCompareImpl.newCompareOp(compare); assertArrayEquals("test-key".getBytes(UTF_8), protoCompare.key()); @@ -106,12 +101,11 @@ public void testCompareCreate() { @Test public void testCompareVersion() { - Compare compare = Compare.newBuilder() + Compare compare = new Compare() .setKey(KEY) .setVersion(VERSION) .setResult(CompareResult.EQUAL) - .setTarget(CompareTarget.VERSION) - .build(); + .setTarget(CompareTarget.VERSION); @Cleanup ProtoCompareImpl protoCompare = ProtoCompareImpl.newCompareOp(compare); assertArrayEquals("test-key".getBytes(UTF_8), protoCompare.key()); diff --git a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointerTest.java b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointerTest.java index e14b38b4781..5689ef14886 100644 --- a/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointerTest.java +++ b/stream/statelib/src/test/java/org/apache/bookkeeper/statelib/impl/rocksdb/checkpoint/RocksCheckpointerTest.java @@ -269,8 +269,9 @@ public void testCheckpointRestore() throws Exception { ByteStreams.readFully(fileIn, checkpointMetadataBytes); // verify the checkpointed metadata exists - CheckpointMetadata metadata = CheckpointMetadata.parseFrom(checkpointMetadataBytes); - assertArrayEquals(txid, metadata.getTxid().toByteArray()); + CheckpointMetadata metadata = new CheckpointMetadata(); + metadata.parseFrom(checkpointMetadataBytes); + assertArrayEquals(txid, metadata.getTxid()); verifyCheckpointMetadata(checkpointedDir, metadata); verifyRemoteFiles(checkpointId, checkpointedDir); @@ -321,7 +322,7 @@ public void testRestoreCleanupCheckpoints() throws Exception { localDir, checkpointStore); assertNotNull(metadata); - assertArrayEquals("checkpoint-2".getBytes(UTF_8), metadata.getTxid().toByteArray()); + assertArrayEquals("checkpoint-2".getBytes(UTF_8), metadata.getTxid()); for (int i = 0; i < 3; i++) { String checkpoint = checkpointIds.get(i); @@ -610,7 +611,7 @@ public void testRestoreCheckpointMissingLocally() throws Exception { localDir, checkpointStore); assertNotNull(metadata); - assertArrayEquals("checkpoint-2".getBytes(UTF_8), metadata.getTxid().toByteArray()); + assertArrayEquals("checkpoint-2".getBytes(UTF_8), metadata.getTxid()); for (int i = 0; i < 3; i++) { String checkpoint = checkpointIds.get(i); @@ -656,7 +657,7 @@ public void testRestoreLocalCheckpointCorrupted() throws Exception { localDir, checkpointStore); assertNotNull(metadata); - assertArrayEquals("checkpoint-0".getBytes(UTF_8), metadata.getTxid().toByteArray()); + assertArrayEquals("checkpoint-0".getBytes(UTF_8), metadata.getTxid()); String checkpoint = checkpointIds.get(0); assertTrue(new File(localCheckpointsDir, checkpoint).exists()); diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcMetaRangeService.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcMetaRangeService.java index ff269b3d5fc..2fe57680ecd 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcMetaRangeService.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcMetaRangeService.java @@ -50,9 +50,8 @@ public void getActiveRanges(GetActiveRangesRequest request, new ResponseHandler(responseObserver) { @Override protected GetActiveRangesResponse createErrorResp(Throwable cause) { - return GetActiveRangesResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build(); + return new GetActiveRangesResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR); } }); } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcRootRangeService.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcRootRangeService.java index 035ba4b1a8a..dc68c47e20d 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcRootRangeService.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcRootRangeService.java @@ -57,9 +57,8 @@ public void createNamespace(CreateNamespaceRequest request, new ResponseHandler(responseObserver) { @Override protected CreateNamespaceResponse createErrorResp(Throwable cause) { - return CreateNamespaceResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build(); + return new CreateNamespaceResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR); } }); } @@ -71,9 +70,8 @@ public void deleteNamespace(DeleteNamespaceRequest request, new ResponseHandler(responseObserver) { @Override protected DeleteNamespaceResponse createErrorResp(Throwable cause) { - return DeleteNamespaceResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build(); + return new DeleteNamespaceResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR); } }); } @@ -85,9 +83,8 @@ public void getNamespace(GetNamespaceRequest request, new ResponseHandler(responseObserver) { @Override protected GetNamespaceResponse createErrorResp(Throwable cause) { - return GetNamespaceResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build(); + return new GetNamespaceResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR); } }); } @@ -103,9 +100,8 @@ public void createStream(CreateStreamRequest request, new ResponseHandler(responseObserver) { @Override protected CreateStreamResponse createErrorResp(Throwable cause) { - return CreateStreamResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build(); + return new CreateStreamResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR); } }); } @@ -117,9 +113,8 @@ public void deleteStream(DeleteStreamRequest request, new ResponseHandler(responseObserver) { @Override protected DeleteStreamResponse createErrorResp(Throwable cause) { - return DeleteStreamResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build(); + return new DeleteStreamResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR); } }); } @@ -131,9 +126,8 @@ public void getStream(GetStreamRequest request, new ResponseHandler(responseObserver) { @Override protected GetStreamResponse createErrorResp(Throwable cause) { - return GetStreamResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build(); + return new GetStreamResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR); } }); } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcTableService.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcTableService.java index d7fdfdb4b02..8b832dfe3fe 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcTableService.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/grpc/GrpcTableService.java @@ -55,12 +55,11 @@ public void range(RangeRequest request, new ResponseHandler(responseObserver) { @Override protected RangeResponse createErrorResp(Throwable cause) { - return RangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .setRoutingHeader(request.getHeader()) - .build()) - .build(); + RangeResponse resp = new RangeResponse(); + ResponseHeader header = resp.setHeader(); + header.setCode(StatusCode.INTERNAL_SERVER_ERROR); + header.setRoutingHeader().copyFrom(request.getHeader()); + return resp; } }); } @@ -72,12 +71,11 @@ public void put(PutRequest request, new ResponseHandler(responseObserver) { @Override protected PutResponse createErrorResp(Throwable cause) { - return PutResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .setRoutingHeader(request.getHeader()) - .build()) - .build(); + PutResponse resp = new PutResponse(); + ResponseHeader header = resp.setHeader(); + header.setCode(StatusCode.INTERNAL_SERVER_ERROR); + header.setRoutingHeader().copyFrom(request.getHeader()); + return resp; } }); } @@ -89,12 +87,11 @@ public void delete(DeleteRangeRequest request, new ResponseHandler(responseObserver) { @Override protected DeleteRangeResponse createErrorResp(Throwable cause) { - return DeleteRangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .setRoutingHeader(request.getHeader()) - .build()) - .build(); + DeleteRangeResponse resp = new DeleteRangeResponse(); + ResponseHeader header = resp.setHeader(); + header.setCode(StatusCode.INTERNAL_SERVER_ERROR); + header.setRoutingHeader().copyFrom(request.getHeader()); + return resp; } }); } @@ -106,12 +103,11 @@ public void txn(TxnRequest request, new ResponseHandler(responseObserver) { @Override protected TxnResponse createErrorResp(Throwable cause) { - return TxnResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .setRoutingHeader(request.getHeader()) - .build()) - .build(); + TxnResponse resp = new TxnResponse(); + ResponseHeader header = resp.setHeader(); + header.setCode(StatusCode.INTERNAL_SERVER_ERROR); + header.setRoutingHeader().copyFrom(request.getHeader()); + return resp; } }); } @@ -123,12 +119,11 @@ public void increment(IncrementRequest request, new ResponseHandler(responseObserver) { @Override protected IncrementResponse createErrorResp(Throwable cause) { - return IncrementResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .setRoutingHeader(request.getHeader()) - .build()) - .build(); + IncrementResponse resp = new IncrementResponse(); + ResponseHeader header = resp.setHeader(); + header.setCode(StatusCode.INTERNAL_SERVER_ERROR); + header.setRoutingHeader().copyFrom(request.getHeader()); + return resp; } }); } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImpl.java index 13379656c76..5a2d4bd8d3f 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImpl.java @@ -26,7 +26,6 @@ import static org.apache.bookkeeper.stream.storage.impl.kv.TableStoreUtils.processRangeResult; import static org.apache.bookkeeper.stream.storage.impl.kv.TableStoreUtils.processTxnResult; -import com.google.protobuf.ByteString; import java.util.concurrent.CompletableFuture; import lombok.CustomLog; import org.apache.bookkeeper.api.kv.op.DeleteOp; @@ -87,12 +86,11 @@ public CompletableFuture range(RangeRequest rangeReq) { }) .exceptionally(cause -> { log.error().attr("request", rangeReq).exception(cause).log("Failed to process range request"); - return RangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(handleCause(cause)) - .setRoutingHeader(rangeReq.getHeader()) - .build()) - .build(); + RangeResponse resp = new RangeResponse(); + ResponseHeader header = resp.setHeader(); + header.setCode(handleCause(cause)); + header.setRoutingHeader().copyFrom(rangeReq.getHeader()); + return resp; }); } @@ -103,12 +101,12 @@ private CompletableFuture> doRange(RangeRequest requ } private RangeOp buildRangeOp(RoutingHeader header, RangeRequest request) { - ByteString rKey = header.getRKey(); - ByteString lKey = request.getKey(); - ByteString lEndKey = request.getRangeEnd(); + byte[] rKey = header.getRKey(); + byte[] lKey = request.getKey(); + byte[] lEndKey = request.getRangeEnd(); byte[] storeKey = newStoreKey(rKey, lKey); byte[] storeEndKey = null; - if (null != lEndKey && lEndKey.size() > 0) { + if (null != lEndKey && lEndKey.length > 0) { storeEndKey = newStoreKey(rKey, lEndKey); } @@ -151,12 +149,11 @@ public CompletableFuture put(PutRequest putReq) { }) .exceptionally(cause -> { log.error().attr("request", putReq).exception(cause).log("Failed to process put request"); - return PutResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(handleCause(cause)) - .setRoutingHeader(putReq.getHeader()) - .build()) - .build(); + PutResponse resp = new PutResponse(); + ResponseHeader header = resp.setHeader(); + header.setCode(handleCause(cause)); + header.setRoutingHeader().copyFrom(putReq.getHeader()); + return resp; }); } @@ -167,14 +164,14 @@ private CompletableFuture> doPut(PutRequest request) { } private PutOp buildPutOp(RoutingHeader header, PutRequest request) { - ByteString rKey = header.getRKey(); - ByteString lKey = request.getKey(); + byte[] rKey = header.getRKey(); + byte[] lKey = request.getKey(); byte[] storeKey = newStoreKey(rKey, lKey); return store.getOpFactory().newPut( storeKey, - request.getValue().toByteArray(), + request.getValue(), store.getOpFactory().optionFactory().newPutOption() - .prevKv(request.getPrevKv()) + .prevKv(request.isPrevKv()) .build()); } @@ -192,12 +189,11 @@ public CompletableFuture incr(IncrementRequest incrementReq) }) .exceptionally(cause -> { log.error().attr("request", incrementReq).exception(cause).log("Failed to process increment request"); - return IncrementResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(handleCause(cause)) - .setRoutingHeader(incrementReq.getHeader()) - .build()) - .build(); + IncrementResponse resp = new IncrementResponse(); + ResponseHeader header = resp.setHeader(); + header.setCode(handleCause(cause)); + header.setRoutingHeader().copyFrom(incrementReq.getHeader()); + return resp; }); } @@ -208,14 +204,14 @@ private CompletableFuture> doIncrement(Increment } private IncrementOp buildIncrementOp(RoutingHeader header, IncrementRequest request) { - ByteString rKey = header.getRKey(); - ByteString lKey = request.getKey(); + byte[] rKey = header.getRKey(); + byte[] lKey = request.getKey(); byte[] storeKey = newStoreKey(rKey, lKey); return store.getOpFactory().newIncrement( storeKey, request.getAmount(), store.getOpFactory().optionFactory().newIncrementOption() - .getTotal(request.getGetTotal()) + .getTotal(request.isGetTotal()) .build()); } @@ -231,12 +227,13 @@ public CompletableFuture delete(DeleteRangeRequest deleteRe result.close(); } }) - .exceptionally(cause -> DeleteRangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(handleCause(cause)) - .setRoutingHeader(deleteReq.getHeader()) - .build()) - .build()); + .exceptionally(cause -> { + DeleteRangeResponse resp = new DeleteRangeResponse(); + ResponseHeader header = resp.setHeader(); + header.setCode(handleCause(cause)); + header.setRoutingHeader().copyFrom(deleteReq.getHeader()); + return resp; + }); } private CompletableFuture> doDelete(DeleteRangeRequest request) { @@ -246,18 +243,18 @@ private CompletableFuture> doDelete(DeleteRangeRequ } private DeleteOp buildDeleteOp(RoutingHeader header, DeleteRangeRequest request) { - ByteString rKey = header.getRKey(); - ByteString lKey = request.getKey(); - ByteString lEndKey = request.getRangeEnd(); + byte[] rKey = header.getRKey(); + byte[] lKey = request.getKey(); + byte[] lEndKey = request.getRangeEnd(); byte[] storeKey = newStoreKey(rKey, lKey); byte[] storeEndKey = null; - if (null != lEndKey && lEndKey.size() > 0) { + if (null != lEndKey && lEndKey.length > 0) { storeEndKey = newStoreKey(rKey, lEndKey); } return store.getOpFactory().newDelete( storeKey, store.getOpFactory().optionFactory().newDeleteOption() - .prevKv(request.getPrevKv()) + .prevKv(request.isPrevKv()) .endKey(storeEndKey) .build()); } @@ -273,12 +270,13 @@ public CompletableFuture txn(TxnRequest txnReq) { txnResult.close(); } }) - .exceptionally(cause -> TxnResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(handleCause(cause)) - .setRoutingHeader(txnReq.getHeader()) - .build()) - .build()); + .exceptionally(cause -> { + TxnResponse resp = new TxnResponse(); + ResponseHeader header = resp.setHeader(); + header.setCode(handleCause(cause)); + header.setRoutingHeader().copyFrom(txnReq.getHeader()); + return resp; + }); } private CompletableFuture> doTxn(TxnRequest request) { @@ -290,13 +288,13 @@ private CompletableFuture> doTxn(TxnRequest request) { private TxnOp buildTxnOp(TxnRequest request) { RoutingHeader header = request.getHeader(); TxnOpBuilder txnBuilder = store.getOpFactory().newTxn(); - for (RequestOp requestOp : request.getSuccessList()) { + for (RequestOp requestOp : request.getSuccessesList()) { txnBuilder.Then(buildTxnOp(header, requestOp)); } - for (RequestOp requestOp : request.getFailureList()) { + for (RequestOp requestOp : request.getFailuresList()) { txnBuilder.Else(buildTxnOp(header, requestOp)); } - for (Compare compare : request.getCompareList()) { + for (Compare compare : request.getComparesList()) { txnBuilder.If(fromProtoCompare(store.getOpFactory(), header, compare)); } return txnBuilder.build(); diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreUtils.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreUtils.java index c7eb2a42e8b..0e5eac57d79 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreUtils.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreUtils.java @@ -17,12 +17,10 @@ */ package org.apache.bookkeeper.stream.storage.impl.kv; -import com.google.common.collect.Lists; -import com.google.protobuf.ByteString; -import com.google.protobuf.UnsafeByteOperations; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; import io.netty.buffer.Unpooled; +import java.util.Arrays; import java.util.concurrent.ExecutionException; import lombok.AccessLevel; import lombok.CustomLog; @@ -61,44 +59,44 @@ final class TableStoreUtils { static final byte NO_ROUTING_KEY = 0x0; static final byte HAS_ROUTING_KEY = 0x1; - static boolean hasRKey(ByteString rKey) { - return null != rKey && rKey.size() > 0; + static boolean hasRKey(byte[] rKey) { + return null != rKey && rKey.length > 0; } - static byte[] newStoreKey(ByteString rKey, ByteString lKey) { + static byte[] newStoreKey(byte[] rKey, byte[] lKey) { boolean hasRkey = hasRKey(rKey); int keyLen; if (hasRkey) { - keyLen = rKey.size() + lKey.size() + 2; + keyLen = rKey.length + lKey.length + 2; } else { - keyLen = lKey.size() + 1; + keyLen = lKey.length + 1; } ByteBuf keyBuf = Unpooled.buffer(keyLen); if (hasRkey) { keyBuf.writeByte(HAS_ROUTING_KEY); - keyBuf.writeBytes(rKey.asReadOnlyByteBuffer()); + keyBuf.writeBytes(rKey); keyBuf.writeByte(SEP); - keyBuf.writeBytes(lKey.asReadOnlyByteBuffer()); + keyBuf.writeBytes(lKey); } else { keyBuf.writeByte(NO_ROUTING_KEY); - keyBuf.writeBytes(lKey.asReadOnlyByteBuffer()); + keyBuf.writeBytes(lKey); } return ByteBufUtil.getBytes(keyBuf); } - static ByteString getLKey(byte[] storeKey, ByteString rKey) { + static byte[] getLKey(byte[] storeKey, byte[] rKey) { boolean hasRKey = hasRKey(rKey); int lKeyOffset; if (hasRKey) { - lKeyOffset = rKey.size() + 2; + lKeyOffset = rKey.length + 2; } else { lKeyOffset = 1; } - return UnsafeByteOperations.unsafeWrap(storeKey, lKeyOffset, storeKey.length - lKeyOffset); + return Arrays.copyOfRange(storeKey, lKeyOffset, storeKey.length); } static StatusCode handleCause(Throwable cause) { @@ -137,118 +135,114 @@ static StatusCode mvccCodeToStatusCode(Code code) { } } - static KeyValue newKeyValue(ByteString rKey, + static KeyValue newKeyValue(byte[] rKey, org.apache.bookkeeper.api.kv.result.KeyValue kv) { if (null == kv) { return null; } - return KeyValue.newBuilder() + return new KeyValue() .setKey(getLKey(kv.key(), rKey)) - .setValue(UnsafeByteOperations.unsafeWrap(kv.value())) + .setValue(kv.value()) .setCreateRevision(kv.createRevision()) .setModRevision(kv.modifiedRevision()) .setVersion(kv.version()) .setIsNumber(kv.isNumber()) - .setNumberValue(kv.numberValue()) - .build(); + .setNumberValue(kv.numberValue()); } static PutResponse processPutResult(RoutingHeader routingHeader, PutResult result) { - ByteString rKey = routingHeader.getRKey(); - PutResponse.Builder putRespBuilder = PutResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(mvccCodeToStatusCode(result.code())) - .setRoutingHeader(routingHeader) - .build()); + byte[] rKey = routingHeader.getRKey(); + PutResponse putResp = new PutResponse(); + ResponseHeader header = putResp.setHeader(); + header.setCode(mvccCodeToStatusCode(result.code())); + header.setRoutingHeader().copyFrom(routingHeader); if (null != result.prevKv()) { - putRespBuilder = putRespBuilder.setPrevKv(newKeyValue(rKey, result.prevKv())); + putResp.setPrevKv().copyFrom(newKeyValue(rKey, result.prevKv())); } - return putRespBuilder.build(); + return putResp; } static IncrementResponse processIncrementResult(RoutingHeader routingHeader, IncrementResult result) { - IncrementResponse.Builder putRespBuilder = IncrementResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(mvccCodeToStatusCode(result.code())) - .setRoutingHeader(routingHeader) - .build()) - .setTotalAmount(result.totalAmount()); - return putRespBuilder.build(); + IncrementResponse putResp = new IncrementResponse(); + ResponseHeader header = putResp.setHeader(); + header.setCode(mvccCodeToStatusCode(result.code())); + header.setRoutingHeader().copyFrom(routingHeader); + putResp.setTotalAmount(result.totalAmount()); + return putResp; } static RangeResponse processRangeResult(RoutingHeader routingHeader, RangeResult result) { - ByteString rKey = routingHeader.getRKey(); - return RangeResponse.newBuilder() - .setCount(result.count()) - .setHeader(ResponseHeader.newBuilder() - .setCode(mvccCodeToStatusCode(result.code())) - .setRoutingHeader(routingHeader) - .build()) - .addAllKvs(Lists.transform(result.kvs(), kv -> newKeyValue(rKey, kv))) - .setMore(result.more()) - .build(); + byte[] rKey = routingHeader.getRKey(); + RangeResponse rangeResp = new RangeResponse(); + rangeResp.setCount(result.count()); + ResponseHeader header = rangeResp.setHeader(); + header.setCode(mvccCodeToStatusCode(result.code())); + header.setRoutingHeader().copyFrom(routingHeader); + for (org.apache.bookkeeper.api.kv.result.KeyValue kv : result.kvs()) { + rangeResp.addKv().copyFrom(newKeyValue(rKey, kv)); + } + rangeResp.setMore(result.more()); + return rangeResp; } static DeleteRangeResponse processDeleteResult(RoutingHeader routingHeader, DeleteResult result) { - ByteString rKey = routingHeader.getRKey(); - return DeleteRangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(mvccCodeToStatusCode(result.code())) - .setRoutingHeader(routingHeader) - .build()) - .setDeleted(result.numDeleted()) - .addAllPrevKvs(Lists.transform(result.prevKvs(), kv -> newKeyValue(rKey, kv))) - .build(); + byte[] rKey = routingHeader.getRKey(); + DeleteRangeResponse delResp = new DeleteRangeResponse(); + ResponseHeader header = delResp.setHeader(); + header.setCode(mvccCodeToStatusCode(result.code())); + header.setRoutingHeader().copyFrom(routingHeader); + delResp.setDeleted(result.numDeleted()); + for (org.apache.bookkeeper.api.kv.result.KeyValue kv : result.prevKvs()) { + delResp.addPrevKv().copyFrom(newKeyValue(rKey, kv)); + } + return delResp; } static TxnResponse processTxnResult(RoutingHeader routingHeader, TxnResult txnResult) { - return TxnResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(mvccCodeToStatusCode(txnResult.code())) - .setRoutingHeader(routingHeader) - .build()) - .setSucceeded(txnResult.isSuccess()) - .addAllResponses(Lists.transform(txnResult.results(), result -> processTxnResult( - routingHeader, result))) - .build(); + TxnResponse txnResp = new TxnResponse(); + ResponseHeader header = txnResp.setHeader(); + header.setCode(mvccCodeToStatusCode(txnResult.code())); + header.setRoutingHeader().copyFrom(routingHeader); + txnResp.setSucceeded(txnResult.isSuccess()); + for (Result result : txnResult.results()) { + txnResp.addResponse().copyFrom(processTxnResult(routingHeader, result)); + } + return txnResp; } static ResponseOp processTxnResult(RoutingHeader routingHeader, Result result) { - ResponseOp.Builder respBuilder = ResponseOp.newBuilder(); + ResponseOp respOp = new ResponseOp(); switch (result.type()) { case PUT: PutResult putResult = (PutResult) result; - respBuilder.setResponsePut( - processPutResult(routingHeader, putResult)); + respOp.setResponsePut().copyFrom(processPutResult(routingHeader, putResult)); break; case DELETE: DeleteResult delResult = (DeleteResult) result; - respBuilder.setResponseDeleteRange( - processDeleteResult(routingHeader, delResult)); + respOp.setResponseDeleteRange().copyFrom(processDeleteResult(routingHeader, delResult)); break; case RANGE: RangeResult rangeResult = (RangeResult) result; - respBuilder.setResponseRange( - processRangeResult(routingHeader, rangeResult)); + respOp.setResponseRange().copyFrom(processRangeResult(routingHeader, rangeResult)); break; default: break; } - return respBuilder.build(); + return respOp; } static CompareOp fromProtoCompare(OpFactory opFactory, RoutingHeader header, Compare compare) { - ByteString rKey = header.getRKey(); - ByteString lKey = compare.getKey(); + byte[] rKey = header.getRKey(); + byte[] lKey = compare.getKey(); byte[] storeKey = newStoreKey(rKey, lKey); CompareResult result = fromProtoCompareResult(compare.getResult()); switch (compare.getTarget()) { @@ -268,11 +262,12 @@ static CompareOp fromProtoCompare(OpFactory opFa storeKey, compare.getVersion()); case VALUE: + byte[] cv = compare.getValue(); return opFactory.compareValue( result, storeKey, - (null == compare.getValue() || compare.getValue().size() == 0) - ? null : compare.getValue().toByteArray()); + (null == cv || cv.length == 0) + ? null : cv); default: throw new IllegalArgumentException("Invalid compare target " + compare.getTarget()); } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/CreateNamespaceProcessor.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/CreateNamespaceProcessor.java index 9b3722cd607..cfcd66800e2 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/CreateNamespaceProcessor.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/CreateNamespaceProcessor.java @@ -43,9 +43,8 @@ protected StatusCode verifyRequest(RootRangeStoreImpl state, @Override protected CreateNamespaceResponse failRequest(StatusCode code) { - return CreateNamespaceResponse.newBuilder() - .setCode(code) - .build(); + return new CreateNamespaceResponse() + .setCode(code); } @Override diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/DeleteNamespaceProcessor.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/DeleteNamespaceProcessor.java index 70c7e6c3273..0a83015f366 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/DeleteNamespaceProcessor.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/DeleteNamespaceProcessor.java @@ -43,9 +43,8 @@ protected StatusCode verifyRequest(RootRangeStoreImpl state, @Override protected DeleteNamespaceResponse failRequest(StatusCode code) { - return DeleteNamespaceResponse.newBuilder() - .setCode(code) - .build(); + return new DeleteNamespaceResponse() + .setCode(code); } @Override diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImpl.java index 117da800657..ac2a72a5208 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImpl.java @@ -74,9 +74,8 @@ private CompletableFuture createStreamIfMissing(long st MetaRangeImpl metaRange, StreamProperties streamProps) { if (null == streamProps) { - return FutureUtils.value(GetActiveRangesResponse.newBuilder() - .setCode(StatusCode.STREAM_NOT_FOUND) - .build()); + return FutureUtils.value(new GetActiveRangesResponse() + .setCode(StatusCode.STREAM_NOT_FOUND)); } return metaRange.create(streamProps).thenCompose(created -> { @@ -86,9 +85,8 @@ private CompletableFuture createStreamIfMissing(long st } return getActiveRanges(metaRange); } else { - return FutureUtils.value(GetActiveRangesResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build()); + return FutureUtils.value(new GetActiveRangesResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR)); } }); } @@ -121,21 +119,21 @@ public CompletableFuture getActiveRanges(GetActiveRange } private CompletableFuture getActiveRanges(MetaRange metaRange) { - GetActiveRangesResponse.Builder respBuilder = GetActiveRangesResponse.newBuilder(); + GetActiveRangesResponse resp = new GetActiveRangesResponse(); return metaRange.getActiveRanges() .thenApplyAsync(ranges -> { for (RangeMetadata range : ranges) { - RelatedRanges.Builder rrBuilder = RelatedRanges.newBuilder() - .setProps(range.getProps()) - .setType(RelationType.PARENTS) - .addAllRelatedRanges(range.getParentsList()); - respBuilder.addRanges(rrBuilder); + RelatedRanges rr = resp.addRange(); + rr.setProps().copyFrom(range.getProps()); + rr.setType(RelationType.PARENTS); + for (int i = 0; i < range.getParentsCount(); i++) { + rr.addRelatedRange(range.getParentAt(i)); + } } - return respBuilder - .setCode(StatusCode.SUCCESS) - .build(); + resp.setCode(StatusCode.SUCCESS); + return resp; }, executor) - .exceptionally(cause -> respBuilder.setCode(StatusCode.INTERNAL_SERVER_ERROR).build()); + .exceptionally(cause -> new GetActiveRangesResponse().setCode(StatusCode.INTERNAL_SERVER_ERROR)); } } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java index 284258f70cf..340d7aad41e 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/RootRangeStoreImpl.java @@ -23,7 +23,6 @@ import static org.apache.bookkeeper.stream.protocol.util.ProtoUtils.validateNamespaceName; import static org.apache.bookkeeper.stream.protocol.util.ProtoUtils.validateStreamName; -import com.google.protobuf.InvalidProtocolBufferException; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledExecutorService; import lombok.CustomLog; @@ -221,12 +220,11 @@ private CompletableFuture executeCreateNamespaceTxn(lon long namespaceId = currentNsId + 1; String nsName = request.getName(); - NamespaceMetadata metadata = NamespaceMetadata.newBuilder() - .setProps(NamespaceProperties.newBuilder() - .setNamespaceId(namespaceId) - .setNamespaceName(nsName) - .setDefaultStreamConf(request.getNsConf().getDefaultStreamConf())) - .build(); + NamespaceMetadata metadata = new NamespaceMetadata(); + NamespaceProperties props = metadata.setProps(); + props.setNamespaceId(namespaceId); + props.setNamespaceName(nsName); + props.setDefaultStreamConf().copyFrom(request.getNsConf().getDefaultStreamConf()); byte[] nsNameKey = getNamespaceNameKey(nsName); byte[] nsNameVal = Bytes.toBytes(namespaceId); @@ -248,15 +246,15 @@ private CompletableFuture executeCreateNamespaceTxn(lon return store.txn(txn) .thenApply(txnResult -> { try { - CreateNamespaceResponse.Builder respBuilder = CreateNamespaceResponse.newBuilder(); + CreateNamespaceResponse resp = new CreateNamespaceResponse(); if (txnResult.isSuccess()) { - respBuilder.setCode(StatusCode.SUCCESS); - respBuilder.setNsProps(metadata.getProps()); + resp.setCode(StatusCode.SUCCESS); + resp.setNsProps().copyFrom(metadata.getProps()); } else { // TODO: differentiate the error code - respBuilder.setCode(StatusCode.INTERNAL_SERVER_ERROR); + resp.setCode(StatusCode.INTERNAL_SERVER_ERROR); } - return respBuilder.build(); + return resp; } finally { txnResult.close(); } @@ -293,9 +291,8 @@ CompletableFuture doProcessDeleteNamespaceRequest(Delet CompletableFuture deleteNamespace(String nsName, NamespaceMetadata nsMetadata) { if (null == nsMetadata) { return FutureUtils.value( - DeleteNamespaceResponse.newBuilder() - .setCode(StatusCode.NAMESPACE_NOT_FOUND) - .build()); + new DeleteNamespaceResponse() + .setCode(StatusCode.NAMESPACE_NOT_FOUND)); } byte[] nsNameKey = getNamespaceNameKey(nsName); @@ -316,14 +313,14 @@ CompletableFuture deleteNamespace(String nsName, Namesp return store.txn(txnOp).thenApply(txnResult -> { try { - DeleteNamespaceResponse.Builder respBuilder = DeleteNamespaceResponse.newBuilder(); + DeleteNamespaceResponse resp = new DeleteNamespaceResponse(); if (txnResult.isSuccess()) { - respBuilder.setCode(StatusCode.SUCCESS); + resp.setCode(StatusCode.SUCCESS); } else { // TODO: differentiate the error code - respBuilder.setCode(StatusCode.INTERNAL_SERVER_ERROR); + resp.setCode(StatusCode.INTERNAL_SERVER_ERROR); } - return respBuilder.build(); + return resp; } finally { txnResult.close(); } @@ -336,19 +333,18 @@ public CompletableFuture getNamespace(GetNamespaceRequest if (!validateNamespaceName(nsName)) { log.error().attr("nsName", nsName).log("Failed to get namespace due to invalid namespace name"); return FutureUtils.value( - GetNamespaceResponse.newBuilder() - .setCode(StatusCode.INVALID_NAMESPACE_NAME) - .build()); + new GetNamespaceResponse() + .setCode(StatusCode.INVALID_NAMESPACE_NAME)); } else { return getNamespace(request.getName()).thenApply(nsMetadata -> { - GetNamespaceResponse.Builder nsRespBuilder = GetNamespaceResponse.newBuilder(); + GetNamespaceResponse nsResp = new GetNamespaceResponse(); if (null == nsMetadata) { - nsRespBuilder.setCode(StatusCode.NAMESPACE_NOT_FOUND); + nsResp.setCode(StatusCode.NAMESPACE_NOT_FOUND); } else { - nsRespBuilder.setCode(StatusCode.SUCCESS); - nsRespBuilder.setNsProps(nsMetadata.getProps()); + nsResp.setCode(StatusCode.SUCCESS); + nsResp.setNsProps().copyFrom(nsMetadata.getProps()); } - return nsRespBuilder.build(); + return nsResp; }); } } @@ -357,8 +353,13 @@ private CompletableFuture getNamespace(long nsId) { byte[] nsIdKey = getNamespaceIdKey(nsId); return store.get(nsIdKey).thenCompose(nsIdVal -> { try { - return FutureUtils.value(null != nsIdVal ? NamespaceMetadata.parseFrom(nsIdVal) : null); - } catch (InvalidProtocolBufferException e) { + if (null == nsIdVal) { + return FutureUtils.value(null); + } + NamespaceMetadata nsMetadata = new NamespaceMetadata(); + nsMetadata.parseFrom(nsIdVal); + return FutureUtils.value(nsMetadata); + } catch (RuntimeException e) { return FutureUtils.exception(e); } }); @@ -388,7 +389,7 @@ public CompletableFuture createStream(CreateStreamRequest StatusCode code = verifyStreamRequest(nsName, streamName); if (StatusCode.SUCCESS != code) { - return FutureUtils.value(CreateStreamResponse.newBuilder().setCode(code).build()); + return FutureUtils.value(new CreateStreamResponse().setCode(code)); } return createStream(nsName, streamName, request.getStreamConf()); @@ -416,14 +417,14 @@ private CompletableFuture createStream(String nsName, streamConf )) .exceptionally(cause -> - CreateStreamResponse.newBuilder().setCode(StatusCode.INTERNAL_SERVER_ERROR).build()); + new CreateStreamResponse().setCode(StatusCode.INTERNAL_SERVER_ERROR)); } private CompletableFuture createStream(NamespaceMetadata nsMetadata, String streamName, StreamConfiguration streamConf) { if (null == nsMetadata) { - return FutureUtils.value(CreateStreamResponse.newBuilder().setCode(StatusCode.NAMESPACE_NOT_FOUND).build()); + return FutureUtils.value(new CreateStreamResponse().setCode(StatusCode.NAMESPACE_NOT_FOUND)); } return getValue(STREAM_ID_KEY) @@ -471,12 +472,11 @@ private CompletableFuture executeCreateStreamTxn(long nsId long scId = placementPolicy.placeStreamRange(streamId, 0L); - StreamProperties streamProps = StreamProperties.newBuilder() + StreamProperties streamProps = new StreamProperties() .setStreamId(streamId) .setStreamName(streamName) - .setStorageContainerId(scId) - .setStreamConf(streamConf) - .build(); + .setStorageContainerId(scId); + streamProps.setStreamConf().copyFrom(streamConf); byte[] nsIdKey = getNamespaceIdKey(nsId); byte[] streamNameKey = getStreamNameKey(nsId, streamName); @@ -505,16 +505,16 @@ private CompletableFuture executeCreateStreamTxn(long nsId return store.txn(txn) .thenApply(txnResult -> { try { - CreateStreamResponse.Builder respBuilder = CreateStreamResponse.newBuilder(); + CreateStreamResponse resp = new CreateStreamResponse(); if (txnResult.isSuccess()) { - respBuilder.setCode(StatusCode.SUCCESS); - respBuilder.setStreamProps(streamProps); + resp.setCode(StatusCode.SUCCESS); + resp.setStreamProps().copyFrom(streamProps); } else { // TODO: differentiate the error codes - respBuilder.setCode(StatusCode.INTERNAL_SERVER_ERROR); + resp.setCode(StatusCode.INTERNAL_SERVER_ERROR); } - logStreamOp("create", nsId, streamId, streamName, respBuilder.getCode()); - return respBuilder.build(); + logStreamOp("create", nsId, streamId, streamName, resp.getCode()); + return resp; } finally { txnResult.close(); txn.close(); @@ -522,7 +522,7 @@ private CompletableFuture executeCreateStreamTxn(long nsId }) .exceptionally(cause -> { txn.close(); - return CreateStreamResponse.newBuilder().setCode(StatusCode.INTERNAL_SERVER_ERROR).build(); + return new CreateStreamResponse().setCode(StatusCode.INTERNAL_SERVER_ERROR); }); } @@ -533,7 +533,7 @@ public CompletableFuture deleteStream(DeleteStreamRequest StatusCode code = verifyStreamRequest(nsName, streamName); if (StatusCode.SUCCESS != code) { - return FutureUtils.value(DeleteStreamResponse.newBuilder().setCode(code).build()); + return FutureUtils.value(new DeleteStreamResponse().setCode(code)); } return deleteStream(nsName, streamName); @@ -549,7 +549,7 @@ private CompletableFuture deleteStream(NamespaceMetadata n String streamName) { if (null == nsMetadata) { - return FutureUtils.value(DeleteStreamResponse.newBuilder().setCode(StatusCode.NAMESPACE_NOT_FOUND).build()); + return FutureUtils.value(new DeleteStreamResponse().setCode(StatusCode.NAMESPACE_NOT_FOUND)); } long nsId = nsMetadata.getProps().getNamespaceId(); @@ -558,9 +558,8 @@ private CompletableFuture deleteStream(NamespaceMetadata n return store.get(streamNameKey).thenCompose(streamIdBytes -> { if (null == streamIdBytes) { return FutureUtils.value( - DeleteStreamResponse.newBuilder() - .setCode(StatusCode.STREAM_NOT_FOUND) - .build()); + new DeleteStreamResponse() + .setCode(StatusCode.STREAM_NOT_FOUND)); } long streamId = Bytes.toLong(streamIdBytes, 0); @@ -595,14 +594,14 @@ private CompletableFuture deleteStream(long nsId, return store.txn(txnOp).thenApply(txnResult -> { try { - DeleteStreamResponse.Builder respBuilder = DeleteStreamResponse.newBuilder(); + DeleteStreamResponse resp = new DeleteStreamResponse(); if (txnResult.isSuccess()) { - respBuilder.setCode(StatusCode.SUCCESS); + resp.setCode(StatusCode.SUCCESS); } else { - respBuilder.setCode(StatusCode.INTERNAL_SERVER_ERROR); + resp.setCode(StatusCode.INTERNAL_SERVER_ERROR); } - logStreamOp("delete", nsId, streamId, streamName, respBuilder.getCode()); - return respBuilder.build(); + logStreamOp("delete", nsId, streamId, streamName, resp.getCode()); + return resp; } finally { txnResult.close(); } @@ -612,20 +611,18 @@ private CompletableFuture deleteStream(long nsId, private CompletableFuture streamPropertiesToResponse( CompletableFuture propsFuture ) { - GetStreamResponse.Builder respBuilder = GetStreamResponse.newBuilder(); return propsFuture.thenCompose(streamProps -> { + GetStreamResponse resp = new GetStreamResponse(); if (null == streamProps) { - return FutureUtils.value(respBuilder.setCode(StatusCode.STREAM_NOT_FOUND).build()); + return FutureUtils.value(resp.setCode(StatusCode.STREAM_NOT_FOUND)); } else { - return FutureUtils.value(respBuilder - .setCode(StatusCode.SUCCESS) - .setStreamProps(streamProps) - .build()); + resp.setCode(StatusCode.SUCCESS); + resp.setStreamProps().copyFrom(streamProps); + return FutureUtils.value(resp); } }).exceptionally(cause -> - respBuilder + new GetStreamResponse() .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build() ); } @@ -637,9 +634,8 @@ public CompletableFuture getStream(GetStreamRequest request) } else if (IdCase.STREAM_NAME == request.getIdCase()) { return getStreamProps(request.getStreamName()); } else { - return FutureUtils.value(GetStreamResponse.newBuilder() - .setCode(StatusCode.ILLEGAL_OP) - .build()); + return FutureUtils.value(new GetStreamResponse() + .setCode(StatusCode.ILLEGAL_OP)); } } @@ -661,8 +657,8 @@ CompletableFuture getStreamProps(StreamName streamName) { streamName.getNamespaceName(), streamName.getStreamName()); if (StatusCode.SUCCESS != code) { - return FutureUtils.value(GetStreamResponse.newBuilder() - .setCode(code).build()); + return FutureUtils.value(new GetStreamResponse() + .setCode(code)); } byte[] nsNameKey = getNamespaceNameKey(streamName.getNamespaceName()); @@ -671,8 +667,8 @@ CompletableFuture getStreamProps(StreamName streamName) { return store.get(nsNameKey) .thenCompose(nsIdBytes -> { if (null == nsIdBytes) { - return FutureUtils.value(GetStreamResponse.newBuilder() - .setCode(StatusCode.NAMESPACE_NOT_FOUND).build()); + return FutureUtils.value(new GetStreamResponse() + .setCode(StatusCode.NAMESPACE_NOT_FOUND)); } long nsId = Bytes.toLong(nsIdBytes, 0); return streamPropertiesToResponse( @@ -701,8 +697,10 @@ CompletableFuture getStreamProps(long nsId, long streamId) { return FutureUtils.value(null); } else { try { - return FutureUtils.value(StreamProperties.parseFrom(streamPropBytes)); - } catch (InvalidProtocolBufferException e) { + StreamProperties props = new StreamProperties(); + props.parseFrom(streamPropBytes); + return FutureUtils.value(props); + } catch (RuntimeException e) { return FutureUtils.exception(e); } } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/MetaRangeImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/MetaRangeImpl.java index e1c12537e09..6057d54ee34 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/MetaRangeImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/MetaRangeImpl.java @@ -28,7 +28,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.protobuf.InvalidProtocolBufferException; import java.util.List; import java.util.NavigableMap; import java.util.concurrent.CompletableFuture; @@ -135,12 +134,10 @@ public MetaRangeImpl(MVCCAsyncStore store, placementPolicy, Maps.newTreeMap(), Lists.newArrayList(), - StreamMetadata - .newBuilder() + new StreamMetadata() .setLifecycleState(LifecycleState.UNINIT) .setServingState(ServingState.WRITABLE) - .setNextRangeId(MIN_DATA_RANGE_ID) - .build(), + .setNextRangeId(MIN_DATA_RANGE_ID), 0L, 0L); } @@ -200,29 +197,31 @@ public NavigableMap unsafeGetRanges() { @VisibleForTesting private synchronized StreamMetadata toStreamMetadata(LifecycleState state) { - StreamMetadata.Builder builder = StreamMetadata - .newBuilder() - .setProps(streamProps) - .setLifecycleState(state) - .setServingState(servingState) - .setNextRangeId(nextRangeId) - .setCTime(cTime) - .setMTime(mTime) - .addAllCurrentRanges(currentRanges); - return builder.build(); + StreamMetadata metadata = new StreamMetadata(); + metadata.setProps().copyFrom(streamProps); + metadata.setLifecycleState(state); + metadata.setServingState(servingState); + metadata.setNextRangeId(nextRangeId); + metadata.setCTime(cTime); + metadata.setMTime(mTime); + for (long currentRange : currentRanges) { + metadata.addCurrentRange(currentRange); + } + return metadata; } private synchronized StreamMetadata toStreamMetadata(ServingState state, long mTime) { - StreamMetadata.Builder builder = StreamMetadata - .newBuilder() - .setProps(streamProps) - .setLifecycleState(lifecycleState) - .setServingState(state) - .setNextRangeId(nextRangeId) - .setCTime(cTime) - .setMTime(mTime) - .addAllCurrentRanges(currentRanges); - return builder.build(); + StreamMetadata metadata = new StreamMetadata(); + metadata.setProps().copyFrom(streamProps); + metadata.setLifecycleState(lifecycleState); + metadata.setServingState(state); + metadata.setNextRangeId(nextRangeId); + metadata.setCTime(cTime); + metadata.setMTime(mTime); + for (long currentRange : currentRanges) { + metadata.addCurrentRange(currentRange); + } + return metadata; } @VisibleForTesting @@ -294,13 +293,11 @@ private void unsafeCreate(CompletableFuture createFuture, List> successOps = Lists.newArrayListWithExpectedSize( propertiesList.size() + 1); for (RangeProperties props : propertiesList) { - RangeMetadata meta = RangeMetadata.newBuilder() - .setProps(props) - .setCreateTime(cTime) - .setFenceTime(Long.MAX_VALUE) - .setState(RangeState.RANGE_ACTIVE) - .addAllParents(Lists.newArrayList()) - .build(); + RangeMetadata meta = new RangeMetadata(); + meta.setProps().copyFrom(props); + meta.setCreateTime(cTime); + meta.setFenceTime(Long.MAX_VALUE); + meta.setState(RangeState.RANGE_ACTIVE); ranges.put(props.getRangeId(), meta); currentRanges.add(props.getRangeId()); @@ -382,10 +379,10 @@ private void loadMetadata(List> kvs) { private synchronized void loadStreamMetadata(long streamId, byte[] streamMetadataBytes) { this.streamId = streamId; - StreamMetadata metadata; + StreamMetadata metadata = new StreamMetadata(); try { - metadata = StreamMetadata.parseFrom(streamMetadataBytes); - } catch (InvalidProtocolBufferException e) { + metadata.parseFrom(streamMetadataBytes); + } catch (RuntimeException e) { throw new RuntimeException("Invalid stream metadata of stream " + streamId, e); } @@ -393,7 +390,9 @@ private synchronized void loadStreamMetadata(long streamId, byte[] streamMetadat this.lifecycleState = metadata.getLifecycleState(); this.servingState = metadata.getServingState(); this.currentRanges.clear(); - this.currentRanges.addAll(metadata.getCurrentRangesList()); + for (int i = 0; i < metadata.getCurrentRangesCount(); i++) { + this.currentRanges.add(metadata.getCurrentRangeAt(i)); + } this.nextRangeId = metadata.getNextRangeId(); this.cTime = metadata.getCTime(); this.mTime = metadata.getMTime(); @@ -403,10 +402,10 @@ private void loadRangeMetadata(long streamId, long rangeId, byte[] rangeMetadata checkArgument(this.streamId == streamId); checkArgument(rangeId >= 0L); - RangeMetadata metadata; + RangeMetadata metadata = new RangeMetadata(); try { - metadata = RangeMetadata.parseFrom(rangeMetadataBytes); - } catch (InvalidProtocolBufferException e) { + metadata.parseFrom(rangeMetadataBytes); + } catch (RuntimeException e) { throw new RuntimeException("Invalid range metadata of range (" + streamId + ", " + rangeId + ")", e); } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptor.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptor.java index 34cad9967c8..664a40da71a 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptor.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptor.java @@ -23,10 +23,6 @@ import static org.apache.bookkeeper.stream.protocol.ProtocolConstants.RK_METADATA_KEY; import static org.apache.bookkeeper.stream.protocol.ProtocolConstants.SID_METADATA_KEY; -import com.google.protobuf.CodedOutputStream; -import com.google.protobuf.MessageLite; -import com.google.protobuf.Parser; -import com.google.protobuf.UnsafeByteOperations; import io.grpc.CallOptions; import io.grpc.Channel; import io.grpc.ClientCall; @@ -43,6 +39,7 @@ import java.io.InputStream; import java.util.HashMap; import java.util.Map; +import java.util.function.Supplier; import lombok.CustomLog; import lombok.Data; import org.apache.bookkeeper.stream.proto.kv.rpc.DeleteRangeRequest; @@ -82,80 +79,173 @@ ReqT intercept(ReqT request, } - private static RoutingHeader.Builder newRoutingHeaderBuilder(RoutingHeader header, - Long sid, - Long rid, - byte[] rk) { - return RoutingHeader.newBuilder(header) - .setStreamId(sid) - .setRangeId(rid) - .setRKey(UnsafeByteOperations.unsafeWrap(rk)); + private static RoutingHeader newRoutingHeader(RoutingHeader header, + Long sid, + Long rid, + byte[] rk) { + RoutingHeader newHeader = new RoutingHeader(); + newHeader.copyFrom(header); + newHeader.setStreamId(sid); + newHeader.setRangeId(rid); + newHeader.setRKey(rk); + return newHeader; } private static final TableRequestMutator PUT_INTERCEPTOR = - (request, sid, rid, rk) -> PutRequest.newBuilder(request) - .setHeader(newRoutingHeaderBuilder(request.getHeader(), sid, rid, rk)) - .build(); + (request, sid, rid, rk) -> { + PutRequest mutated = new PutRequest(); + mutated.copyFrom(request); + mutated.setHeader().copyFrom(newRoutingHeader(request.getHeader(), sid, rid, rk)); + return mutated; + }; private static final TableRequestMutator RANGE_INTERCEPTOR = - (request, sid, rid, rk) -> RangeRequest.newBuilder(request) - .setHeader(newRoutingHeaderBuilder(request.getHeader(), sid, rid, rk)) - .build(); + (request, sid, rid, rk) -> { + RangeRequest mutated = new RangeRequest(); + mutated.copyFrom(request); + mutated.setHeader().copyFrom(newRoutingHeader(request.getHeader(), sid, rid, rk)); + return mutated; + }; private static final TableRequestMutator DELETE_INTERCEPTOR = - (request, sid, rid, rk) -> DeleteRangeRequest.newBuilder(request) - .setHeader(newRoutingHeaderBuilder(request.getHeader(), sid, rid, rk)) - .build(); + (request, sid, rid, rk) -> { + DeleteRangeRequest mutated = new DeleteRangeRequest(); + mutated.copyFrom(request); + mutated.setHeader().copyFrom(newRoutingHeader(request.getHeader(), sid, rid, rk)); + return mutated; + }; private static final TableRequestMutator INCR_INTERCEPTOR = - (request, sid, rid, rk) -> IncrementRequest.newBuilder(request) - .setHeader(newRoutingHeaderBuilder(request.getHeader(), sid, rid, rk)) - .build(); + (request, sid, rid, rk) -> { + IncrementRequest mutated = new IncrementRequest(); + mutated.copyFrom(request); + mutated.setHeader().copyFrom(newRoutingHeader(request.getHeader(), sid, rid, rk)); + return mutated; + }; private static final TableRequestMutator TXN_INTERCEPTOR = - (request, sid, rid, rk) -> TxnRequest.newBuilder(request) - .setHeader(newRoutingHeaderBuilder(request.getHeader(), sid, rid, rk)) - .build(); + (request, sid, rid, rk) -> { + TxnRequest mutated = new TxnRequest(); + mutated.copyFrom(request); + mutated.setHeader().copyFrom(newRoutingHeader(request.getHeader(), sid, rid, rk)); + return mutated; + }; + + /** + * Parser that creates a new instance of a lightproto message from a byte array. + */ + private interface LightProtoParser { + T parseFrom(byte[] data); + } + + /** + * Serializer that converts a lightproto message to a byte array. + */ + private interface LightProtoSerializer { + byte[] toByteArray(T msg); + } @Data(staticConstructor = "of") - private static class InterceptorDescriptor { + private static class InterceptorDescriptor { private final Class clz; - private final Parser parser; + private final Supplier factory; + private final LightProtoParser parser; + private final LightProtoSerializer serializer; private final TableRequestMutator interceptor; } + private static InterceptorDescriptor descriptor( + Class clz, + Supplier factory, + LightProtoParser parser, + LightProtoSerializer serializer, + TableRequestMutator interceptor + ) { + return InterceptorDescriptor.of(clz, factory, parser, serializer, interceptor); + } + private static Map> kvRpcMethods = new HashMap<>(); + private static PutRequest parsePutRequest(byte[] bytes) { + PutRequest m = new PutRequest(); + m.parseFrom(bytes); + return m; + } + + private static RangeRequest parseRangeRequest(byte[] bytes) { + RangeRequest m = new RangeRequest(); + m.parseFrom(bytes); + return m; + } + + private static DeleteRangeRequest parseDeleteRangeRequest(byte[] bytes) { + DeleteRangeRequest m = new DeleteRangeRequest(); + m.parseFrom(bytes); + return m; + } + + private static IncrementRequest parseIncrementRequest(byte[] bytes) { + IncrementRequest m = new IncrementRequest(); + m.parseFrom(bytes); + return m; + } + + private static TxnRequest parseTxnRequest(byte[] bytes) { + TxnRequest m = new TxnRequest(); + m.parseFrom(bytes); + return m; + } + static { kvRpcMethods.put( TableServiceGrpc.getPutMethod().getFullMethodName(), - InterceptorDescriptor.of( - PutRequest.class, PutRequest.parser(), PUT_INTERCEPTOR + descriptor( + PutRequest.class, + PutRequest::new, + RoutingHeaderProxyInterceptor::parsePutRequest, + PutRequest::toByteArray, + PUT_INTERCEPTOR ) ); kvRpcMethods.put( TableServiceGrpc.getRangeMethod().getFullMethodName(), - InterceptorDescriptor.of( - RangeRequest.class, RangeRequest.parser(), RANGE_INTERCEPTOR + descriptor( + RangeRequest.class, + RangeRequest::new, + RoutingHeaderProxyInterceptor::parseRangeRequest, + RangeRequest::toByteArray, + RANGE_INTERCEPTOR ) ); kvRpcMethods.put( TableServiceGrpc.getDeleteMethod().getFullMethodName(), - InterceptorDescriptor.of( - DeleteRangeRequest.class, DeleteRangeRequest.parser(), DELETE_INTERCEPTOR + descriptor( + DeleteRangeRequest.class, + DeleteRangeRequest::new, + RoutingHeaderProxyInterceptor::parseDeleteRangeRequest, + DeleteRangeRequest::toByteArray, + DELETE_INTERCEPTOR ) ); kvRpcMethods.put( TableServiceGrpc.getIncrementMethod().getFullMethodName(), - InterceptorDescriptor.of( - IncrementRequest.class, IncrementRequest.parser(), INCR_INTERCEPTOR + descriptor( + IncrementRequest.class, + IncrementRequest::new, + RoutingHeaderProxyInterceptor::parseIncrementRequest, + IncrementRequest::toByteArray, + INCR_INTERCEPTOR ) ); kvRpcMethods.put( TableServiceGrpc.getTxnMethod().getFullMethodName(), - InterceptorDescriptor.of( - TxnRequest.class, TxnRequest.parser(), TXN_INTERCEPTOR + descriptor( + TxnRequest.class, + TxnRequest::new, + RoutingHeaderProxyInterceptor::parseTxnRequest, + TxnRequest::toByteArray, + TXN_INTERCEPTOR ) ); } @@ -236,7 +326,7 @@ private ReqT interceptMessage(MethodDescriptor method .parse(new ByteBufInputStream(buffer, true)); } - private ReqT interceptMessage( + private ReqT interceptMessage( MethodDescriptor method, InterceptorDescriptor descriptor, ReqT message, @@ -262,7 +352,7 @@ private ReqT interceptMessage( } @SuppressWarnings("unchecked") - private ReqT interceptTableRequest( + private ReqT interceptTableRequest( MethodDescriptor method, InterceptorDescriptor interceptor, ReqT message, @@ -274,7 +364,8 @@ private ReqT interceptTableRequest( request = (TableReqT) message; } else { InputStream is = method.getRequestMarshaller().stream(message); - request = interceptor.getParser().parseFrom(is); + byte[] bytes = is.readAllBytes(); + request = interceptor.getParser().parseFrom(bytes); } TableReqT interceptedMessage = interceptor.getInterceptor().intercept( request, sid, rid, rk @@ -282,8 +373,7 @@ private ReqT interceptTableRequest( if (message.getClass() == interceptor.getClz()) { return (ReqT) interceptedMessage; } else { - byte[] reqBytes = new byte[interceptedMessage.getSerializedSize()]; - interceptedMessage.writeTo(CodedOutputStream.newInstance(reqBytes)); + byte[] reqBytes = interceptor.getSerializer().toByteArray(interceptedMessage); return method.getRequestMarshaller().parse(new ByteArrayInputStream(reqBytes)); } diff --git a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java index 186b2865469..e34023585e7 100644 --- a/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java +++ b/stream/storage/impl/src/main/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImpl.java @@ -271,12 +271,11 @@ public CompletableFuture range(RangeRequest request) { RoutingHeader header = request.getHeader(); if (header.getRangeId() <= 0L) { - return FutureUtils.value(RangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.BAD_REQUEST) - .setRoutingHeader(request.getHeader()) - .build()) - .build()); + RangeResponse resp = new RangeResponse(); + ResponseHeader responseHeader = resp.setHeader(); + responseHeader.setCode(StatusCode.BAD_REQUEST); + responseHeader.setRoutingHeader().copyFrom(request.getHeader()); + return FutureUtils.value(resp); } RangeId rid = RangeId.of(header.getStreamId(), header.getRangeId()); @@ -289,12 +288,11 @@ public CompletableFuture put(PutRequest request) { RoutingHeader header = request.getHeader(); if (header.getRangeId() <= 0L) { - return CompletableFuture.completedFuture(PutResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.BAD_REQUEST) - .setRoutingHeader(request.getHeader()) - .build()) - .build()); + PutResponse resp = new PutResponse(); + ResponseHeader responseHeader = resp.setHeader(); + responseHeader.setCode(StatusCode.BAD_REQUEST); + responseHeader.setRoutingHeader().copyFrom(request.getHeader()); + return CompletableFuture.completedFuture(resp); } RangeId rid = RangeId.of(header.getStreamId(), header.getRangeId()); @@ -307,12 +305,11 @@ public CompletableFuture delete(DeleteRangeRequest request) RoutingHeader header = request.getHeader(); if (header.getRangeId() <= 0L) { - return CompletableFuture.completedFuture(DeleteRangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.BAD_REQUEST) - .setRoutingHeader(request.getHeader()) - .build()) - .build()); + DeleteRangeResponse resp = new DeleteRangeResponse(); + ResponseHeader responseHeader = resp.setHeader(); + responseHeader.setCode(StatusCode.BAD_REQUEST); + responseHeader.setRoutingHeader().copyFrom(request.getHeader()); + return CompletableFuture.completedFuture(resp); } RangeId rid = RangeId.of(header.getStreamId(), header.getRangeId()); @@ -325,12 +322,11 @@ public CompletableFuture txn(TxnRequest request) { RoutingHeader header = request.getHeader(); if (header.getRangeId() <= 0L) { - return CompletableFuture.completedFuture(TxnResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.BAD_REQUEST) - .setRoutingHeader(request.getHeader()) - .build()) - .build()); + TxnResponse resp = new TxnResponse(); + ResponseHeader responseHeader = resp.setHeader(); + responseHeader.setCode(StatusCode.BAD_REQUEST); + responseHeader.setRoutingHeader().copyFrom(request.getHeader()); + return CompletableFuture.completedFuture(resp); } RangeId rid = RangeId.of(header.getStreamId(), header.getRangeId()); @@ -343,12 +339,11 @@ public CompletableFuture incr(IncrementRequest request) { RoutingHeader header = request.getHeader(); if (header.getRangeId() <= 0L) { - return CompletableFuture.completedFuture(IncrementResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.BAD_REQUEST) - .setRoutingHeader(request.getHeader()) - .build()) - .build()); + IncrementResponse resp = new IncrementResponse(); + ResponseHeader responseHeader = resp.setHeader(); + responseHeader.setCode(StatusCode.BAD_REQUEST); + responseHeader.setRoutingHeader().copyFrom(request.getHeader()); + return CompletableFuture.completedFuture(resp); } RangeId rid = RangeId.of(header.getStreamId(), header.getRangeId()); diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/TestStorageContainerStoreImpl.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/TestStorageContainerStoreImpl.java index 456f0bd3710..07959974d3d 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/TestStorageContainerStoreImpl.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/TestStorageContainerStoreImpl.java @@ -33,16 +33,20 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import com.google.protobuf.ByteString; +import com.google.common.util.concurrent.ListenableFuture; +import io.grpc.CallOptions; import io.grpc.Channel; import io.grpc.ClientInterceptors; import io.grpc.ManagedChannel; +import io.grpc.MethodDescriptor; import io.grpc.Server; import io.grpc.ServerServiceDefinition; import io.grpc.Status; import io.grpc.StatusRuntimeException; import io.grpc.inprocess.InProcessChannelBuilder; import io.grpc.inprocess.InProcessServerBuilder; +import io.grpc.stub.ClientCalls; +import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -66,7 +70,6 @@ import org.apache.bookkeeper.stream.proto.kv.rpc.ResponseHeader; import org.apache.bookkeeper.stream.proto.kv.rpc.RoutingHeader; import org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc; -import org.apache.bookkeeper.stream.proto.kv.rpc.TableServiceGrpc.TableServiceFutureStub; import org.apache.bookkeeper.stream.proto.storage.CreateNamespaceRequest; import org.apache.bookkeeper.stream.proto.storage.CreateNamespaceResponse; import org.apache.bookkeeper.stream.proto.storage.CreateStreamRequest; @@ -82,9 +85,7 @@ import org.apache.bookkeeper.stream.proto.storage.GetStreamRequest; import org.apache.bookkeeper.stream.proto.storage.GetStreamResponse; import org.apache.bookkeeper.stream.proto.storage.MetaRangeServiceGrpc; -import org.apache.bookkeeper.stream.proto.storage.MetaRangeServiceGrpc.MetaRangeServiceFutureStub; import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc; -import org.apache.bookkeeper.stream.proto.storage.RootRangeServiceGrpc.RootRangeServiceFutureStub; import org.apache.bookkeeper.stream.proto.storage.StatusCode; import org.apache.bookkeeper.stream.storage.StorageResources; import org.apache.bookkeeper.stream.storage.api.metadata.RangeStoreService; @@ -105,107 +106,112 @@ @CustomLog public class TestStorageContainerStoreImpl { - private static final StreamProperties streamProps = StreamProperties.newBuilder() - .setStorageContainerId(System.currentTimeMillis()) - .setStreamId(System.currentTimeMillis()) - .setStreamName("test-create-add-stream-request") - .setStreamConf(DEFAULT_STREAM_CONF) - .build(); + private static final StreamProperties streamProps = createStreamProps(); + + private static StreamProperties createStreamProps() { + StreamProperties props = new StreamProperties() + .setStorageContainerId(System.currentTimeMillis()) + .setStreamId(System.currentTimeMillis()) + .setStreamName("test-create-add-stream-request"); + props.setStreamConf().copyFrom(DEFAULT_STREAM_CONF); + return props; + } private static StreamName createStreamName(String name) { - return StreamName.newBuilder() + return new StreamName() .setNamespaceName(name + "_col") - .setStreamName(name + "_stream") - .build(); + .setStreamName(name + "_stream"); } private static Endpoint createEndpoint(String hostname, int port) { - return Endpoint.newBuilder() + return new Endpoint() .setHostname(hostname) - .setPort(port) - .build(); + .setPort(port); } private final CompositeConfiguration compConf = new CompositeConfiguration(); private final StorageConfiguration storageConf = new StorageConfiguration(compConf); - private final NamespaceConfiguration namespaceConf = - NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build(); + private final NamespaceConfiguration namespaceConf = createNamespaceConf(); + + private static NamespaceConfiguration createNamespaceConf() { + NamespaceConfiguration conf = new NamespaceConfiguration(); + conf.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); + return conf; + } private final StorageResources resources = StorageResources.create(); private RangeStoreService mockRangeStoreService; private StorageContainerStoreImpl rangeStore; private Server server; private Channel channel; - private TableServiceFutureStub tableService; - private RootRangeServiceFutureStub rootRangeService; - private MetaRangeServiceFutureStub metaRangeService; private long scId; + private ListenableFuture futureCall(MethodDescriptor method, ReqT request) { + return ClientCalls.futureUnaryCall(channel.newCall(method, CallOptions.DEFAULT), request); + } + // // Utils for table api // - private static final ByteString TEST_ROUTING_KEY = ByteString.copyFromUtf8("test-routing-key"); - private static final ByteString TEST_KEY = ByteString.copyFromUtf8("test-key"); - private static final ByteString TEST_VAL = ByteString.copyFromUtf8("test-val"); - private static final RoutingHeader TEST_ROUTING_HEADER = RoutingHeader.newBuilder() - .setRKey(TEST_ROUTING_KEY) - .setStreamId(1234L) - .setRangeId(1256L) - .build(); - private static final ResponseHeader TEST_RESP_HEADER = ResponseHeader.newBuilder() - .setRoutingHeader(TEST_ROUTING_HEADER) - .build(); + private static final byte[] TEST_ROUTING_KEY = "test-routing-key".getBytes(StandardCharsets.UTF_8); + private static final byte[] TEST_KEY = "test-key".getBytes(StandardCharsets.UTF_8); + private static final byte[] TEST_VAL = "test-val".getBytes(StandardCharsets.UTF_8); + + private static RoutingHeader testRoutingHeader() { + return new RoutingHeader() + .setRKey(TEST_ROUTING_KEY) + .setStreamId(1234L) + .setRangeId(1256L); + } private static PutRequest createPutRequest() { - return PutRequest.newBuilder() - .setHeader(TEST_ROUTING_HEADER) - .setKey(TEST_KEY) - .setValue(TEST_VAL) - .build(); + PutRequest req = new PutRequest() + .setKey(TEST_KEY) + .setValue(TEST_VAL); + req.setHeader().copyFrom(testRoutingHeader()); + return req; } private static PutResponse createPutResponse(StatusCode code) { - return PutResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder(TEST_RESP_HEADER) - .setCode(code) - .build()) - .build(); + PutResponse resp = new PutResponse(); + ResponseHeader header = resp.setHeader(); + header.setRoutingHeader().copyFrom(testRoutingHeader()); + header.setCode(code); + return resp; } private static RangeRequest createRangeRequest() { - return RangeRequest.newBuilder() - .setHeader(TEST_ROUTING_HEADER) - .setKey(TEST_KEY) - .build(); + RangeRequest req = new RangeRequest() + .setKey(TEST_KEY); + req.setHeader().copyFrom(testRoutingHeader()); + return req; } private static RangeResponse createRangeResponse(StatusCode code) { - return RangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder(TEST_RESP_HEADER) - .setCode(code) - .build()) - .setCount(0) - .build(); + RangeResponse resp = new RangeResponse(); + ResponseHeader header = resp.setHeader(); + header.setRoutingHeader().copyFrom(testRoutingHeader()); + header.setCode(code); + resp.setCount(0); + return resp; } private static DeleteRangeRequest createDeleteRequest() { - return DeleteRangeRequest.newBuilder() - .setHeader(TEST_ROUTING_HEADER) - .setKey(TEST_KEY) - .build(); + DeleteRangeRequest req = new DeleteRangeRequest() + .setKey(TEST_KEY); + req.setHeader().copyFrom(testRoutingHeader()); + return req; } private static DeleteRangeResponse createDeleteResponse(StatusCode code) { - return DeleteRangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder(TEST_RESP_HEADER) - .setCode(code) - .build()) - .setDeleted(0) - .build(); + DeleteRangeResponse resp = new DeleteRangeResponse(); + ResponseHeader header = resp.setHeader(); + header.setRoutingHeader().copyFrom(testRoutingHeader()); + header.setCode(code); + resp.setDeleted(0); + return resp; } @SuppressWarnings("unchecked") @@ -262,11 +268,6 @@ public void setUp() throws Exception { channel = ClientInterceptors.intercept( channel, new StorageContainerClientInterceptor(scId)); - - - tableService = TableServiceGrpc.newFutureStub(channel); - metaRangeService = MetaRangeServiceGrpc.newFutureStub(channel); - rootRangeService = RootRangeServiceGrpc.newFutureStub(channel); } @After @@ -306,7 +307,8 @@ public void testCreateNamespaceNoRootStorageContainerStore() throws Exception { String colName = "test-create-namespace-no-root-storage-container-store"; verifyNotFoundException(fromListenableFuture( - rootRangeService.createNamespace(createCreateNamespaceRequest(colName, namespaceConf))), + futureCall(RootRangeServiceGrpc.getCreateNamespaceMethod(), + createCreateNamespaceRequest(colName, namespaceConf))), Status.NOT_FOUND); } @@ -316,7 +318,7 @@ public void testDeleteNamespaceNoRootStorageContainerStore() throws Exception { String colName = "test-delete-namespace-no-root-storage-container-store"; verifyNotFoundException(fromListenableFuture( - rootRangeService.deleteNamespace(createDeleteNamespaceRequest(colName))), + futureCall(RootRangeServiceGrpc.getDeleteNamespaceMethod(), createDeleteNamespaceRequest(colName))), Status.NOT_FOUND); } @@ -326,7 +328,7 @@ public void testGetNamespaceNoRootStorageContainerStore() throws Exception { String colName = "test-get-namespace-no-root-storage-container-store"; verifyNotFoundException(fromListenableFuture( - rootRangeService.getNamespace(createGetNamespaceRequest(colName))), + futureCall(RootRangeServiceGrpc.getGetNamespaceMethod(), createGetNamespaceRequest(colName))), Status.NOT_FOUND); } @@ -334,17 +336,16 @@ public void testGetNamespaceNoRootStorageContainerStore() throws Exception { public void testCreateNamespaceMockRootStorageContainerStore() throws Exception { String colName = "test-create-namespace-mock-root-storage-container-store"; - CreateNamespaceResponse createResp = CreateNamespaceResponse.newBuilder() - .setCode(StatusCode.NAMESPACE_EXISTS) - .build(); + CreateNamespaceResponse createResp = new CreateNamespaceResponse() + .setCode(StatusCode.NAMESPACE_EXISTS); CreateNamespaceRequest request = createCreateNamespaceRequest(colName, namespaceConf); when(mockRangeStoreService.createNamespace(request)) .thenReturn(CompletableFuture.completedFuture(createResp)); CompletableFuture createRespFuture = - fromListenableFuture(rootRangeService.createNamespace(request)); - assertTrue(createResp == createRespFuture.get()); + fromListenableFuture(futureCall(RootRangeServiceGrpc.getCreateNamespaceMethod(), request)); + assertEquals(createResp, createRespFuture.get()); verify(mockRangeStoreService, times(1)).createNamespace(request); } @@ -352,36 +353,34 @@ public void testCreateNamespaceMockRootStorageContainerStore() throws Exception public void testDeleteNamespaceMockRootStorageContainerStore() throws Exception { String colName = "test-delete-namespace-no-root-storage-container-store"; - DeleteNamespaceResponse deleteResp = DeleteNamespaceResponse.newBuilder() - .setCode(StatusCode.NAMESPACE_NOT_FOUND) - .build(); + DeleteNamespaceResponse deleteResp = new DeleteNamespaceResponse() + .setCode(StatusCode.NAMESPACE_NOT_FOUND); DeleteNamespaceRequest request = createDeleteNamespaceRequest(colName); when(mockRangeStoreService.deleteNamespace(request)) .thenReturn(CompletableFuture.completedFuture(deleteResp)); CompletableFuture deleteRespFuture = - fromListenableFuture(rootRangeService.deleteNamespace(request)); + fromListenableFuture(futureCall(RootRangeServiceGrpc.getDeleteNamespaceMethod(), request)); verify(mockRangeStoreService, times(1)).deleteNamespace(request); - assertTrue(deleteResp == deleteRespFuture.get()); + assertEquals(deleteResp, deleteRespFuture.get()); } @Test public void testGetNamespaceMockRootStorageContainerStore() throws Exception { String colName = "test-get-namespace-no-root-storage-container-store"; - GetNamespaceResponse getResp = GetNamespaceResponse.newBuilder() - .setCode(StatusCode.NAMESPACE_NOT_FOUND) - .build(); + GetNamespaceResponse getResp = new GetNamespaceResponse() + .setCode(StatusCode.NAMESPACE_NOT_FOUND); GetNamespaceRequest request = createGetNamespaceRequest(colName); when(mockRangeStoreService.getNamespace(request)).thenReturn( CompletableFuture.completedFuture(getResp)); CompletableFuture getRespFuture = - fromListenableFuture(rootRangeService.getNamespace(request)); + fromListenableFuture(futureCall(RootRangeServiceGrpc.getGetNamespaceMethod(), request)); verify(mockRangeStoreService, times(1)).getNamespace(request); - assertTrue(getResp == getRespFuture.get()); + assertEquals(getResp, getRespFuture.get()); } // @@ -395,7 +394,8 @@ public void testCreateStreamNoRootStorageContainerStore() throws Exception { String colName = "test-create-namespace-no-root-storage-container-store"; String streamName = colName; verifyNotFoundException(fromListenableFuture( - rootRangeService.createStream(createCreateStreamRequest(colName, streamName, DEFAULT_STREAM_CONF))), + futureCall(RootRangeServiceGrpc.getCreateStreamMethod(), + createCreateStreamRequest(colName, streamName, DEFAULT_STREAM_CONF))), Status.NOT_FOUND); } @@ -406,7 +406,7 @@ public void testDeleteStreamNoRootStorageContainerStore() throws Exception { String colName = "test-delete-namespace-no-root-storage-container-store"; String streamName = colName; verifyNotFoundException(fromListenableFuture( - rootRangeService.deleteStream(createDeleteStreamRequest(colName, streamName))), + futureCall(RootRangeServiceGrpc.getDeleteStreamMethod(), createDeleteStreamRequest(colName, streamName))), Status.NOT_FOUND); } @@ -417,7 +417,7 @@ public void testGetStreamNoRootStorageContainerStore() throws Exception { String colName = "test-get-namespace-no-root-storage-container-store"; String streamName = colName; verifyNotFoundException(fromListenableFuture( - rootRangeService.getStream(createGetStreamRequest(colName, streamName))), + futureCall(RootRangeServiceGrpc.getGetStreamMethod(), createGetStreamRequest(colName, streamName))), Status.NOT_FOUND); } @@ -426,17 +426,16 @@ public void testCreateStreamMockRootStorageContainerStore() throws Exception { String colName = "test-create-namespace-mock-root-storage-container-store"; String streamName = colName; - CreateStreamResponse createResp = CreateStreamResponse.newBuilder() - .setCode(StatusCode.STREAM_EXISTS) - .build(); + CreateStreamResponse createResp = new CreateStreamResponse() + .setCode(StatusCode.STREAM_EXISTS); CreateStreamRequest createReq = createCreateStreamRequest(colName, streamName, DEFAULT_STREAM_CONF); when(mockRangeStoreService.createStream(createReq)).thenReturn( CompletableFuture.completedFuture(createResp)); CompletableFuture createRespFuture = - fromListenableFuture(rootRangeService.createStream(createReq)); + fromListenableFuture(futureCall(RootRangeServiceGrpc.getCreateStreamMethod(), createReq)); verify(mockRangeStoreService, times(1)).createStream(createReq); - assertTrue(createResp == createRespFuture.get()); + assertEquals(createResp, createRespFuture.get()); } @Test @@ -444,17 +443,16 @@ public void testDeleteStreamMockRootStorageContainerStore() throws Exception { String colName = "test-delete-namespace-no-root-storage-container-store"; String streamName = colName; - DeleteStreamResponse deleteResp = DeleteStreamResponse.newBuilder() - .setCode(StatusCode.STREAM_NOT_FOUND) - .build(); + DeleteStreamResponse deleteResp = new DeleteStreamResponse() + .setCode(StatusCode.STREAM_NOT_FOUND); DeleteStreamRequest deleteReq = createDeleteStreamRequest(colName, streamName); when(mockRangeStoreService.deleteStream(deleteReq)).thenReturn( CompletableFuture.completedFuture(deleteResp)); CompletableFuture deleteRespFuture = - fromListenableFuture(rootRangeService.deleteStream(deleteReq)); + fromListenableFuture(futureCall(RootRangeServiceGrpc.getDeleteStreamMethod(), deleteReq)); verify(mockRangeStoreService, times(1)).deleteStream(deleteReq); - assertTrue(deleteResp == deleteRespFuture.get()); + assertEquals(deleteResp, deleteRespFuture.get()); } @Test @@ -462,17 +460,16 @@ public void testGetStreamMockRootStorageContainerStore() throws Exception { String colName = "test-get-namespace-no-root-storage-container-store"; String streamName = colName; - GetStreamResponse getResp = GetStreamResponse.newBuilder() - .setCode(StatusCode.STREAM_NOT_FOUND) - .build(); + GetStreamResponse getResp = new GetStreamResponse() + .setCode(StatusCode.STREAM_NOT_FOUND); GetStreamRequest getReq = createGetStreamRequest(colName, streamName); when(mockRangeStoreService.getStream(getReq)).thenReturn( CompletableFuture.completedFuture(getResp)); CompletableFuture getRespFuture = - fromListenableFuture(rootRangeService.getStream(getReq)); + fromListenableFuture(futureCall(RootRangeServiceGrpc.getGetStreamMethod(), getReq)); verify(mockRangeStoreService, times(1)).getStream(getReq); - assertTrue(getResp == getRespFuture.get()); + assertEquals(getResp, getRespFuture.get()); } @Test @@ -480,24 +477,23 @@ public void testGetActiveRangesNoManager() throws Exception { rangeStore.getRegistry().stopStorageContainer(scId).join(); verifyNotFoundException(fromListenableFuture( - metaRangeService.getActiveRanges(createGetActiveRangesRequest(34L))), + futureCall(MetaRangeServiceGrpc.getGetActiveRangesMethod(), createGetActiveRangesRequest(34L))), Status.NOT_FOUND); } @Test public void testGetActiveRangesMockManager() throws Exception { - GetActiveRangesResponse resp = GetActiveRangesResponse.newBuilder() - .setCode(StatusCode.STREAM_NOT_FOUND) - .build(); + GetActiveRangesResponse resp = new GetActiveRangesResponse() + .setCode(StatusCode.STREAM_NOT_FOUND); GetActiveRangesRequest request = createGetActiveRangesRequest(34L); when(mockRangeStoreService.getActiveRanges(request)) .thenReturn(CompletableFuture.completedFuture(resp)); CompletableFuture future = fromListenableFuture( - metaRangeService.getActiveRanges(request)); + futureCall(MetaRangeServiceGrpc.getGetActiveRangesMethod(), request)); verify(mockRangeStoreService, times(1)).getActiveRanges(request); - assertTrue(resp == future.get()); + assertEquals(resp, future.get()); } @@ -510,7 +506,7 @@ public void testPutNoStorageContainer() throws Exception { rangeStore.getRegistry().stopStorageContainer(scId).join(); verifyNotFoundException(fromListenableFuture( - tableService.put(createPutRequest())), + futureCall(TableServiceGrpc.getPutMethod(), createPutRequest())), Status.NOT_FOUND); } @@ -519,7 +515,7 @@ public void testDeleteNoStorageContainer() throws Exception { rangeStore.getRegistry().stopStorageContainer(scId).join(); verifyNotFoundException(fromListenableFuture( - tableService.delete(createDeleteRequest())), + futureCall(TableServiceGrpc.getDeleteMethod(), createDeleteRequest())), Status.NOT_FOUND); } @@ -528,7 +524,7 @@ public void testRangeNoStorageContainer() throws Exception { rangeStore.getRegistry().stopStorageContainer(scId).join(); verifyNotFoundException(fromListenableFuture( - tableService.range(createRangeRequest())), + futureCall(TableServiceGrpc.getRangeMethod(), createRangeRequest())), Status.NOT_FOUND); } @@ -541,9 +537,9 @@ public void testRangeMockStorageContainer() throws Exception { .thenReturn(CompletableFuture.completedFuture(response)); CompletableFuture future = fromListenableFuture( - tableService.range(request)); + futureCall(TableServiceGrpc.getRangeMethod(), request)); verify(mockRangeStoreService, times(1)).range(eq(request)); - assertTrue(response == future.get()); + assertEquals(response, future.get()); } @Test @@ -555,9 +551,9 @@ public void testDeleteMockStorageContainer() throws Exception { .thenReturn(CompletableFuture.completedFuture(response)); CompletableFuture future = fromListenableFuture( - tableService.delete(request)); + futureCall(TableServiceGrpc.getDeleteMethod(), request)); verify(mockRangeStoreService, times(1)).delete(eq(request)); - assertTrue(response == future.get()); + assertEquals(response, future.get()); } @Test @@ -569,9 +565,9 @@ public void testPutMockStorageContainer() throws Exception { .thenReturn(CompletableFuture.completedFuture(response)); CompletableFuture future = fromListenableFuture( - tableService.put(request)); + futureCall(TableServiceGrpc.getPutMethod(), request)); verify(mockRangeStoreService, times(1)).put(eq(request)); - assertTrue(response == future.get()); + assertEquals(response, future.get()); } } diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/grpc/TestGrpcMetaRangeService.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/grpc/TestGrpcMetaRangeService.java index 3f87b9e9d5b..008ae0a78a9 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/grpc/TestGrpcMetaRangeService.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/grpc/TestGrpcMetaRangeService.java @@ -49,14 +49,11 @@ public void testGetActiveRangesSuccess() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcMetaRangeService grpcService = new GrpcMetaRangeService(rangeService); - GetActiveRangesRequest request = GetActiveRangesRequest - .newBuilder() - .setStreamId(23456L) - .build(); + GetActiveRangesRequest request = new GetActiveRangesRequest() + .setStreamId(23456L); - GetActiveRangesResponse response = GetActiveRangesResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .build(); + GetActiveRangesResponse response = new GetActiveRangesResponse() + .setCode(StatusCode.SUCCESS); when(rangeService.getActiveRanges(request)).thenReturn( CompletableFuture.completedFuture(response)); @@ -76,14 +73,11 @@ public void testGetActiveRangesFailure() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcMetaRangeService grpcService = new GrpcMetaRangeService(rangeService); - GetActiveRangesRequest request = GetActiveRangesRequest - .newBuilder() - .setStreamId(23456L) - .build(); + GetActiveRangesRequest request = new GetActiveRangesRequest() + .setStreamId(23456L); - GetActiveRangesResponse response = GetActiveRangesResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build(); + GetActiveRangesResponse response = new GetActiveRangesResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR); when(rangeService.getActiveRanges(request)).thenReturn( FutureUtils.exception(CAUSE)); @@ -103,10 +97,8 @@ public void testGetActiveRangesException() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcMetaRangeService grpcService = new GrpcMetaRangeService(rangeService); - GetActiveRangesRequest request = GetActiveRangesRequest - .newBuilder() - .setStreamId(23456L) - .build(); + GetActiveRangesRequest request = new GetActiveRangesRequest() + .setStreamId(23456L); when(rangeService.getActiveRanges(request)).thenReturn( FutureUtils.exception(new StatusRuntimeException(Status.NOT_FOUND))); diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/grpc/TestGrpcRootRangeService.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/grpc/TestGrpcRootRangeService.java index f22e52ae1b5..a589d367192 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/grpc/TestGrpcRootRangeService.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/grpc/TestGrpcRootRangeService.java @@ -66,27 +66,77 @@ public class TestGrpcRootRangeService { private static final long colId = 12345L; private static final String nsName = "test-namespace-name"; - private static final NamespaceConfiguration namespaceConf = - NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build(); - private static final NamespaceProperties namespaceProps = - NamespaceProperties.newBuilder() + private static final NamespaceConfiguration namespaceConf = createNamespaceConf(); + + private static NamespaceConfiguration createNamespaceConf() { + NamespaceConfiguration conf = new NamespaceConfiguration(); + conf.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); + return conf; + } + + private static final NamespaceProperties namespaceProps = createNamespaceProps(); + + private static NamespaceProperties createNamespaceProps() { + NamespaceProperties props = new NamespaceProperties() .setNamespaceId(colId) - .setNamespaceName(nsName) - .setDefaultStreamConf(namespaceConf.getDefaultStreamConf()) - .build(); + .setNamespaceName(nsName); + props.setDefaultStreamConf().copyFrom(namespaceConf.getDefaultStreamConf()); + return props; + } + private static final String streamName = "test-stream-name"; - private static final StreamProperties streamProps = - StreamProperties.newBuilder() + private static final StreamProperties streamProps = createStreamProps(); + + private static StreamProperties createStreamProps() { + StreamProperties props = new StreamProperties() .setStorageContainerId(1234L) - .setStreamConf(DEFAULT_STREAM_CONF) .setStreamName(streamName) - .setStreamId(1234L) - .build(); + .setStreamId(1234L); + props.setStreamConf().copyFrom(DEFAULT_STREAM_CONF); + return props; + } private static final Throwable CAUSE = new StorageException("test-grpc-root-range-service"); + private static CreateNamespaceRequest createNsRequest() { + CreateNamespaceRequest req = new CreateNamespaceRequest() + .setName(nsName); + req.setNsConf().copyFrom(namespaceConf); + return req; + } + + private static DeleteNamespaceRequest deleteNsRequest() { + return new DeleteNamespaceRequest() + .setName(nsName); + } + + private static GetNamespaceRequest getNsRequest() { + return new GetNamespaceRequest() + .setName(nsName); + } + + private static CreateStreamRequest createStreamRequest() { + CreateStreamRequest req = new CreateStreamRequest() + .setNsName(nsName) + .setName(streamName); + req.setStreamConf().copyFrom(DEFAULT_STREAM_CONF); + return req; + } + + private static DeleteStreamRequest deleteStreamRequest() { + return new DeleteStreamRequest() + .setNsName(nsName) + .setName(streamName); + } + + private static GetStreamRequest getStreamRequest() { + GetStreamRequest req = new GetStreamRequest(); + StreamName sn = req.setStreamName(); + sn.setNamespaceName(nsName); + sn.setStreamName(streamName); + return req; + } + // // Test Namespace API // @@ -95,10 +145,9 @@ public class TestGrpcRootRangeService { public void testCreateNamespaceSuccess() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcRootRangeService grpcService = new GrpcRootRangeService(rangeService); - CreateNamespaceResponse createResp = CreateNamespaceResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .setNsProps(namespaceProps) - .build(); + CreateNamespaceResponse createResp = new CreateNamespaceResponse() + .setCode(StatusCode.SUCCESS); + createResp.setNsProps().copyFrom(namespaceProps); CreateNamespaceRequest createReq = createCreateNamespaceRequest(nsName, namespaceConf); when(rangeService.createNamespace(createReq)).thenReturn( CompletableFuture.completedFuture(createResp)); @@ -123,10 +172,7 @@ public void onCompleted() { } }; grpcService.createNamespace( - CreateNamespaceRequest.newBuilder() - .setName(nsName) - .setNsConf(namespaceConf) - .build(), + createNsRequest(), streamObserver); latch.await(); assertNull(exceptionHolder.get()); @@ -139,9 +185,8 @@ public void onCompleted() { public void testCreateNamespaceFailure() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcRootRangeService grpcService = new GrpcRootRangeService(rangeService); - CreateNamespaceResponse createResp = CreateNamespaceResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build(); + CreateNamespaceResponse createResp = new CreateNamespaceResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR); CreateNamespaceRequest createReq = createCreateNamespaceRequest(nsName, namespaceConf); when(rangeService.createNamespace(createReq)).thenReturn( FutureUtils.exception(CAUSE)); @@ -166,10 +211,7 @@ public void onCompleted() { } }; grpcService.createNamespace( - CreateNamespaceRequest.newBuilder() - .setName(nsName) - .setNsConf(namespaceConf) - .build(), + createNsRequest(), streamObserver); latch.await(); assertNull(exceptionHolder.get()); @@ -182,9 +224,8 @@ public void onCompleted() { public void testDeleteNamespaceSuccess() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcRootRangeService grpcService = new GrpcRootRangeService(rangeService); - DeleteNamespaceResponse deleteResp = DeleteNamespaceResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .build(); + DeleteNamespaceResponse deleteResp = new DeleteNamespaceResponse() + .setCode(StatusCode.SUCCESS); DeleteNamespaceRequest deleteReq = createDeleteNamespaceRequest(nsName); when(rangeService.deleteNamespace(deleteReq)).thenReturn( CompletableFuture.completedFuture(deleteResp)); @@ -209,9 +250,7 @@ public void onCompleted() { } }; grpcService.deleteNamespace( - DeleteNamespaceRequest.newBuilder() - .setName(nsName) - .build(), + deleteNsRequest(), streamObserver); latch.await(); assertNull(exceptionHolder.get()); @@ -224,9 +263,8 @@ public void onCompleted() { public void testDeleteNamespaceFailure() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcRootRangeService grpcService = new GrpcRootRangeService(rangeService); - DeleteNamespaceResponse deleteResp = DeleteNamespaceResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build(); + DeleteNamespaceResponse deleteResp = new DeleteNamespaceResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR); DeleteNamespaceRequest deleteReq = createDeleteNamespaceRequest(nsName); when(rangeService.deleteNamespace(deleteReq)).thenReturn( FutureUtils.exception(CAUSE)); @@ -251,9 +289,7 @@ public void onCompleted() { } }; grpcService.deleteNamespace( - DeleteNamespaceRequest.newBuilder() - .setName(nsName) - .build(), + deleteNsRequest(), streamObserver); latch.await(); assertNull(exceptionHolder.get()); @@ -266,10 +302,9 @@ public void onCompleted() { public void testGetNamespaceSuccess() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcRootRangeService grpcService = new GrpcRootRangeService(rangeService); - GetNamespaceResponse getResp = GetNamespaceResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .setNsProps(namespaceProps) - .build(); + GetNamespaceResponse getResp = new GetNamespaceResponse() + .setCode(StatusCode.SUCCESS); + getResp.setNsProps().copyFrom(namespaceProps); GetNamespaceRequest getReq = createGetNamespaceRequest(nsName); when(rangeService.getNamespace(getReq)).thenReturn( CompletableFuture.completedFuture(getResp)); @@ -294,9 +329,7 @@ public void onCompleted() { } }; grpcService.getNamespace( - GetNamespaceRequest.newBuilder() - .setName(nsName) - .build(), + getNsRequest(), streamObserver); latch.await(); assertNull(exceptionHolder.get()); @@ -309,9 +342,8 @@ public void onCompleted() { public void testGetNamespaceFailure() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcRootRangeService grpcService = new GrpcRootRangeService(rangeService); - GetNamespaceResponse getResp = GetNamespaceResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build(); + GetNamespaceResponse getResp = new GetNamespaceResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR); GetNamespaceRequest getReq = createGetNamespaceRequest(nsName); when(rangeService.getNamespace(getReq)).thenReturn( FutureUtils.exception(CAUSE)); @@ -336,9 +368,7 @@ public void onCompleted() { } }; grpcService.getNamespace( - GetNamespaceRequest.newBuilder() - .setName(nsName) - .build(), + getNsRequest(), streamObserver); latch.await(); assertNull(exceptionHolder.get()); @@ -355,10 +385,9 @@ public void onCompleted() { public void testCreateStreamSuccess() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcRootRangeService grpcService = new GrpcRootRangeService(rangeService); - CreateStreamResponse createResp = CreateStreamResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .setStreamProps(streamProps) - .build(); + CreateStreamResponse createResp = new CreateStreamResponse() + .setCode(StatusCode.SUCCESS); + createResp.setStreamProps().copyFrom(streamProps); CreateStreamRequest createReq = createCreateStreamRequest(nsName, streamName, DEFAULT_STREAM_CONF); when(rangeService.createStream(createReq)).thenReturn( CompletableFuture.completedFuture(createResp)); @@ -383,11 +412,7 @@ public void onCompleted() { } }; grpcService.createStream( - CreateStreamRequest.newBuilder() - .setNsName(nsName) - .setName(streamName) - .setStreamConf(DEFAULT_STREAM_CONF) - .build(), + createStreamRequest(), streamObserver); latch.await(); assertNull(exceptionHolder.get()); @@ -400,9 +425,8 @@ public void onCompleted() { public void testCreateStreamFailure() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcRootRangeService grpcService = new GrpcRootRangeService(rangeService); - CreateStreamResponse createResp = CreateStreamResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build(); + CreateStreamResponse createResp = new CreateStreamResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR); CreateStreamRequest createReq = createCreateStreamRequest(nsName, streamName, DEFAULT_STREAM_CONF); when(rangeService.createStream(createReq)).thenReturn( FutureUtils.exception(CAUSE)); @@ -427,11 +451,7 @@ public void onCompleted() { } }; grpcService.createStream( - CreateStreamRequest.newBuilder() - .setNsName(nsName) - .setName(streamName) - .setStreamConf(DEFAULT_STREAM_CONF) - .build(), + createStreamRequest(), streamObserver); latch.await(); assertNull(exceptionHolder.get()); @@ -444,9 +464,8 @@ public void onCompleted() { public void testDeleteStreamSuccess() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcRootRangeService grpcService = new GrpcRootRangeService(rangeService); - DeleteStreamResponse deleteResp = DeleteStreamResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .build(); + DeleteStreamResponse deleteResp = new DeleteStreamResponse() + .setCode(StatusCode.SUCCESS); DeleteStreamRequest deleteReq = createDeleteStreamRequest(nsName, streamName); when(rangeService.deleteStream(deleteReq)).thenReturn( CompletableFuture.completedFuture(deleteResp)); @@ -471,10 +490,7 @@ public void onCompleted() { } }; grpcService.deleteStream( - DeleteStreamRequest.newBuilder() - .setNsName(nsName) - .setName(streamName) - .build(), + deleteStreamRequest(), streamObserver); latch.await(); assertNull(exceptionHolder.get()); @@ -487,9 +503,8 @@ public void onCompleted() { public void testDeleteStreamFailure() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcRootRangeService grpcService = new GrpcRootRangeService(rangeService); - DeleteStreamResponse deleteResp = DeleteStreamResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build(); + DeleteStreamResponse deleteResp = new DeleteStreamResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR); DeleteStreamRequest deleteReq = createDeleteStreamRequest(nsName, streamName); when(rangeService.deleteStream(deleteReq)).thenReturn( FutureUtils.exception(CAUSE)); @@ -514,10 +529,7 @@ public void onCompleted() { } }; grpcService.deleteStream( - DeleteStreamRequest.newBuilder() - .setNsName(nsName) - .setName(streamName) - .build(), + deleteStreamRequest(), streamObserver); latch.await(); assertNull(exceptionHolder.get()); @@ -530,10 +542,9 @@ public void onCompleted() { public void testGetStreamSuccess() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcRootRangeService grpcService = new GrpcRootRangeService(rangeService); - GetStreamResponse getResp = GetStreamResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .setStreamProps(streamProps) - .build(); + GetStreamResponse getResp = new GetStreamResponse() + .setCode(StatusCode.SUCCESS); + getResp.setStreamProps().copyFrom(streamProps); GetStreamRequest getReq = createGetStreamRequest(nsName, streamName); when(rangeService.getStream(getReq)).thenReturn( CompletableFuture.completedFuture(getResp)); @@ -558,11 +569,7 @@ public void onCompleted() { } }; grpcService.getStream( - GetStreamRequest.newBuilder() - .setStreamName(StreamName.newBuilder() - .setNamespaceName(nsName) - .setStreamName(streamName)) - .build(), + getStreamRequest(), streamObserver); latch.await(); assertNull(exceptionHolder.get()); @@ -575,9 +582,8 @@ public void onCompleted() { public void testGetStreamFailure() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcRootRangeService grpcService = new GrpcRootRangeService(rangeService); - GetStreamResponse getResp = GetStreamResponse.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .build(); + GetStreamResponse getResp = new GetStreamResponse() + .setCode(StatusCode.INTERNAL_SERVER_ERROR); GetStreamRequest getReq = createGetStreamRequest(nsName, streamName); when(rangeService.getStream(getReq)).thenReturn( FutureUtils.exception(CAUSE)); @@ -602,11 +608,7 @@ public void onCompleted() { } }; grpcService.getStream( - GetStreamRequest.newBuilder() - .setStreamName(StreamName.newBuilder() - .setNamespaceName(nsName) - .setStreamName(streamName)) - .build(), + getStreamRequest(), streamObserver); latch.await(); assertNull(exceptionHolder.get()); diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/grpc/TestGrpcTableService.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/grpc/TestGrpcTableService.java index 874e69ba376..990104e8d4c 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/grpc/TestGrpcTableService.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/grpc/TestGrpcTableService.java @@ -23,9 +23,9 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import com.google.protobuf.ByteString; import io.grpc.Status; import io.grpc.StatusRuntimeException; +import java.nio.charset.StandardCharsets; import java.util.concurrent.CompletableFuture; import org.apache.bookkeeper.common.concurrent.FutureUtils; import org.apache.bookkeeper.stream.proto.kv.rpc.DeleteRangeRequest; @@ -47,14 +47,62 @@ public class TestGrpcTableService { private static final Throwable CAUSE = new Exception("test-exception"); - private static final ByteString TEST_ROUTING_KEY = ByteString.copyFromUtf8("test-routing-key"); - private static final RoutingHeader ROUTING_HEADER = RoutingHeader.newBuilder() - .setStreamId(1234L) - .setRangeId(1234L) - .setRKey(TEST_ROUTING_KEY) - .build(); - private static final ByteString TEST_KEY = ByteString.copyFromUtf8("test-key"); - private static final ByteString TEST_VAL = ByteString.copyFromUtf8("test-val"); + private static final byte[] TEST_ROUTING_KEY = "test-routing-key".getBytes(StandardCharsets.UTF_8); + private static final byte[] TEST_KEY = "test-key".getBytes(StandardCharsets.UTF_8); + private static final byte[] TEST_VAL = "test-val".getBytes(StandardCharsets.UTF_8); + + private static RoutingHeader newRoutingHeader() { + return new RoutingHeader() + .setStreamId(1234L) + .setRangeId(1234L) + .setRKey(TEST_ROUTING_KEY); + } + + private static PutRequest newPutRequest() { + PutRequest req = new PutRequest() + .setKey(TEST_KEY) + .setValue(TEST_VAL); + req.setHeader().copyFrom(newRoutingHeader()); + return req; + } + + private static PutResponse newPutResponse(StatusCode code) { + PutResponse resp = new PutResponse(); + ResponseHeader header = resp.setHeader(); + header.setCode(code); + header.setRoutingHeader().copyFrom(newRoutingHeader()); + return resp; + } + + private static RangeRequest newRangeRequest() { + RangeRequest req = new RangeRequest() + .setKey(TEST_KEY); + req.setHeader().copyFrom(newRoutingHeader()); + return req; + } + + private static RangeResponse newRangeResponse(StatusCode code) { + RangeResponse resp = new RangeResponse(); + ResponseHeader header = resp.setHeader(); + header.setCode(code); + header.setRoutingHeader().copyFrom(newRoutingHeader()); + return resp; + } + + private static DeleteRangeRequest newDeleteRangeRequest() { + DeleteRangeRequest req = new DeleteRangeRequest() + .setKey(TEST_KEY); + req.setHeader().copyFrom(newRoutingHeader()); + return req; + } + + private static DeleteRangeResponse newDeleteRangeResponse(StatusCode code) { + DeleteRangeResponse resp = new DeleteRangeResponse(); + ResponseHeader header = resp.setHeader(); + header.setCode(code); + header.setRoutingHeader().copyFrom(newRoutingHeader()); + return resp; + } // // Meta KeyRange Server Requests tests @@ -65,19 +113,9 @@ public void testPutSuccess() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcTableService grpcService = new GrpcTableService(rangeService); - PutRequest request = PutRequest - .newBuilder() - .setKey(TEST_KEY) - .setValue(TEST_VAL) - .setHeader(ROUTING_HEADER) - .build(); + PutRequest request = newPutRequest(); - PutResponse response = PutResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.SUCCESS) - .setRoutingHeader(ROUTING_HEADER) - .build()) - .build(); + PutResponse response = newPutResponse(StatusCode.SUCCESS); when(rangeService.put(request)).thenReturn( CompletableFuture.completedFuture(response)); @@ -97,19 +135,9 @@ public void testPutFailure() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcTableService grpcService = new GrpcTableService(rangeService); - PutRequest request = PutRequest - .newBuilder() - .setKey(TEST_KEY) - .setValue(TEST_VAL) - .setHeader(ROUTING_HEADER) - .build(); + PutRequest request = newPutRequest(); - PutResponse response = PutResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .setRoutingHeader(ROUTING_HEADER) - .build()) - .build(); + PutResponse response = newPutResponse(StatusCode.INTERNAL_SERVER_ERROR); when(rangeService.put(request)).thenReturn( FutureUtils.exception(CAUSE)); @@ -129,12 +157,7 @@ public void testPutException() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcTableService grpcService = new GrpcTableService(rangeService); - PutRequest request = PutRequest - .newBuilder() - .setKey(TEST_KEY) - .setValue(TEST_VAL) - .setHeader(ROUTING_HEADER) - .build(); + PutRequest request = newPutRequest(); when(rangeService.put(request)).thenReturn( FutureUtils.exception(new StatusRuntimeException(Status.NOT_FOUND))); @@ -154,18 +177,9 @@ public void testRangeSuccess() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcTableService grpcService = new GrpcTableService(rangeService); - RangeRequest request = RangeRequest - .newBuilder() - .setKey(TEST_KEY) - .setHeader(ROUTING_HEADER) - .build(); + RangeRequest request = newRangeRequest(); - RangeResponse response = RangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.SUCCESS) - .setRoutingHeader(ROUTING_HEADER) - .build()) - .build(); + RangeResponse response = newRangeResponse(StatusCode.SUCCESS); when(rangeService.range(request)).thenReturn( CompletableFuture.completedFuture(response)); @@ -185,18 +199,9 @@ public void testRangeActiveRangesFailure() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcTableService grpcService = new GrpcTableService(rangeService); - RangeRequest request = RangeRequest - .newBuilder() - .setKey(TEST_KEY) - .setHeader(ROUTING_HEADER) - .build(); + RangeRequest request = newRangeRequest(); - RangeResponse response = RangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .setRoutingHeader(ROUTING_HEADER) - .build()) - .build(); + RangeResponse response = newRangeResponse(StatusCode.INTERNAL_SERVER_ERROR); when(rangeService.range(request)).thenReturn( FutureUtils.exception(CAUSE)); @@ -216,11 +221,7 @@ public void testRangeActiveRangesException() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcTableService grpcService = new GrpcTableService(rangeService); - RangeRequest request = RangeRequest - .newBuilder() - .setKey(TEST_KEY) - .setHeader(ROUTING_HEADER) - .build(); + RangeRequest request = newRangeRequest(); when(rangeService.range(request)).thenReturn( FutureUtils.exception(new StatusRuntimeException(Status.NOT_FOUND))); @@ -240,18 +241,9 @@ public void testDeleteSuccess() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcTableService grpcService = new GrpcTableService(rangeService); - DeleteRangeRequest request = DeleteRangeRequest - .newBuilder() - .setKey(TEST_KEY) - .setHeader(ROUTING_HEADER) - .build(); + DeleteRangeRequest request = newDeleteRangeRequest(); - DeleteRangeResponse response = DeleteRangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.SUCCESS) - .setRoutingHeader(ROUTING_HEADER) - .build()) - .build(); + DeleteRangeResponse response = newDeleteRangeResponse(StatusCode.SUCCESS); when(rangeService.delete(request)).thenReturn( CompletableFuture.completedFuture(response)); @@ -271,18 +263,9 @@ public void testDeleteFailure() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcTableService grpcService = new GrpcTableService(rangeService); - DeleteRangeRequest request = DeleteRangeRequest - .newBuilder() - .setKey(TEST_KEY) - .setHeader(ROUTING_HEADER) - .build(); + DeleteRangeRequest request = newDeleteRangeRequest(); - DeleteRangeResponse response = DeleteRangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.INTERNAL_SERVER_ERROR) - .setRoutingHeader(ROUTING_HEADER) - .build()) - .build(); + DeleteRangeResponse response = newDeleteRangeResponse(StatusCode.INTERNAL_SERVER_ERROR); when(rangeService.delete(request)).thenReturn( FutureUtils.exception(CAUSE)); @@ -302,11 +285,7 @@ public void testDeleteException() throws Exception { RangeStoreService rangeService = mock(RangeStoreService.class); GrpcTableService grpcService = new GrpcTableService(rangeService); - DeleteRangeRequest request = DeleteRangeRequest - .newBuilder() - .setKey(TEST_KEY) - .setHeader(ROUTING_HEADER) - .build(); + DeleteRangeRequest request = newDeleteRangeRequest(); when(rangeService.delete(request)).thenReturn( FutureUtils.exception(new StatusRuntimeException(Status.NOT_FOUND))); diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImplTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImplTest.java index 0dd25aaf837..127e2f4720e 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImplTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreImplTest.java @@ -18,6 +18,7 @@ package org.apache.bookkeeper.stream.storage.impl.kv; import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -25,7 +26,8 @@ import static org.junit.Assert.assertTrue; import com.google.common.collect.Lists; -import com.google.protobuf.ByteString; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; import lombok.CustomLog; @@ -40,7 +42,6 @@ import org.apache.bookkeeper.stream.proto.kv.rpc.PutResponse; import org.apache.bookkeeper.stream.proto.kv.rpc.RangeRequest; import org.apache.bookkeeper.stream.proto.kv.rpc.RangeResponse; -import org.apache.bookkeeper.stream.proto.kv.rpc.RequestOp; import org.apache.bookkeeper.stream.proto.kv.rpc.ResponseOp; import org.apache.bookkeeper.stream.proto.kv.rpc.RoutingHeader; import org.apache.bookkeeper.stream.proto.kv.rpc.TxnRequest; @@ -56,12 +57,16 @@ public class TableStoreImplTest extends MVCCAsyncStoreTestBase { private static final long SC_ID = 123L; - private static final ByteString RKEY = ByteString.copyFromUtf8("routing-key"); - private static final RoutingHeader HEADER = RoutingHeader.newBuilder() - .setRangeId(1234L) - .setRKey(RKEY) - .setStreamId(1256L) - .build(); + private static final byte[] RKEY = "routing-key".getBytes(StandardCharsets.UTF_8); + + private static RoutingHeader newHeader() { + return new RoutingHeader() + .setRangeId(1234L) + .setRKey(RKEY) + .setStreamId(1256L); + } + + private static final RoutingHeader HEADER = newHeader(); private TableStoreImpl tableStore; @@ -78,12 +83,12 @@ protected void doTeardown() throws Exception { // Put & Range Ops // - private static ByteString getKey(int i) { - return ByteString.copyFromUtf8(String.format("key-%05d", i)); + private static byte[] getKey(int i) { + return String.format("key-%05d", i).getBytes(StandardCharsets.UTF_8); } - private ByteString getValue(int i) { - return ByteString.copyFromUtf8(String.format("value-%05d", i)); + private byte[] getValue(int i) { + return String.format("value-%05d", i).getBytes(StandardCharsets.UTF_8); } private List writeKVs(int numPairs, boolean prevKv) throws Exception { @@ -92,54 +97,54 @@ private List writeKVs(int numPairs, boolean prevKv) throws Exception { for (int i = 0; i < numPairs; i++) { results.add(writeKV(i, prevKv)); } - return Lists.transform( - result(FutureUtils.collect(results)), putResp -> { - assertEquals(StatusCode.SUCCESS, putResp.getHeader().getCode()); - assertEquals(HEADER, putResp.getHeader().getRoutingHeader()); - if (putResp.hasPrevKv()) { - return putResp.getPrevKv(); - } else { - return null; - } - }); + List responses = result(FutureUtils.collect(results)); + List kvs = new ArrayList<>(responses.size()); + for (PutResponse putResp : responses) { + assertEquals(StatusCode.SUCCESS, putResp.getHeader().getCode()); + assertEquals(HEADER, putResp.getHeader().getRoutingHeader()); + if (putResp.hasPrevKv()) { + kvs.add(putResp.getPrevKv()); + } else { + kvs.add(null); + } + } + return kvs; } private CompletableFuture writeKV(int i, boolean prevKv) { - return tableStore.put(PutRequest.newBuilder() + PutRequest req = new PutRequest() .setKey(getKey(i)) .setValue(getValue(i)) - .setHeader(HEADER) - .setPrevKv(prevKv) - .build()); + .setPrevKv(prevKv); + req.setHeader().copyFrom(HEADER); + return tableStore.put(req); } RangeResponse getKeyFromTableStore(int i) throws Exception { - return result( - tableStore.range(RangeRequest.newBuilder() - .setHeader(HEADER) - .setKey(getKey(i)) - .build())); + RangeRequest req = new RangeRequest() + .setKey(getKey(i)); + req.setHeader().copyFrom(HEADER); + return result(tableStore.range(req)); } KeyValue getKeyValue(int i) throws Exception { RangeResponse rr = getKeyFromTableStore(i); assertEquals(StatusCode.SUCCESS, rr.getHeader().getCode()); assertEquals(HEADER, rr.getHeader().getRoutingHeader()); - assertFalse(rr.getMore()); + assertFalse(rr.isMore()); if (0 == rr.getCount()) { return null; } else { - return rr.getKvs(0); + return rr.getKvAt(0); } } void putKeyToTableStore(int key, int value) throws Exception { - PutResponse putResp = result( - tableStore.put(PutRequest.newBuilder() - .setHeader(HEADER) - .setKey(getKey(key)) - .setValue(getValue(value)) - .build())); + PutRequest req = new PutRequest() + .setKey(getKey(key)) + .setValue(getValue(value)); + req.setHeader().copyFrom(HEADER); + PutResponse putResp = result(tableStore.put(req)); assertEquals(StatusCode.SUCCESS, putResp.getHeader().getCode()); assertEquals(HEADER, putResp.getHeader().getRoutingHeader()); @@ -147,34 +152,35 @@ void putKeyToTableStore(int key, int value) throws Exception { } KeyValue putIfAbsentToTableStore(int key, int value, boolean expectedSuccess) throws Exception { - TxnResponse txnResp = result( - tableStore.txn(TxnRequest.newBuilder() - .setHeader(HEADER) - .addCompare(Compare.newBuilder() - .setResult(CompareResult.EQUAL) - .setTarget(CompareTarget.VALUE) - .setKey(getKey(key)) - .setValue(ByteString.copyFrom(new byte[0]))) - .addSuccess(RequestOp.newBuilder() - .setRequestPut(PutRequest.newBuilder() - .setHeader(HEADER) - .setKey(getKey(key)) - .setValue(getValue(value)) - .setPrevKv(true) - .build())) - .addFailure(RequestOp.newBuilder() - .setRequestRange(RangeRequest.newBuilder() - .setHeader(HEADER) - .setKey(getKey(key)) - .build())) - .build())); + TxnRequest req = new TxnRequest(); + req.setHeader().copyFrom(HEADER); + + Compare compare = req.addCompare(); + compare.setResult(CompareResult.EQUAL); + compare.setTarget(CompareTarget.VALUE); + compare.setKey(getKey(key)); + compare.setValue(new byte[0]); + + PutRequest putReq = new PutRequest() + .setKey(getKey(key)) + .setValue(getValue(value)) + .setPrevKv(true); + putReq.setHeader().copyFrom(HEADER); + req.addSuccess().setRequestPut().copyFrom(putReq); + + RangeRequest rangeReq = new RangeRequest() + .setKey(getKey(key)); + rangeReq.setHeader().copyFrom(HEADER); + req.addFailure().setRequestRange().copyFrom(rangeReq); + + TxnResponse txnResp = result(tableStore.txn(req)); assertEquals(HEADER, txnResp.getHeader().getRoutingHeader()); assertEquals(StatusCode.SUCCESS, txnResp.getHeader().getCode()); - ResponseOp respOp = txnResp.getResponses(0); + ResponseOp respOp = txnResp.getResponseAt(0); if (expectedSuccess) { - assertTrue(txnResp.getSucceeded()); + assertTrue(txnResp.isSucceeded()); PutResponse putResp = respOp.getResponsePut(); assertEquals(HEADER, putResp.getHeader().getRoutingHeader()); if (!putResp.hasPrevKv()) { @@ -182,49 +188,50 @@ KeyValue putIfAbsentToTableStore(int key, int value, boolean expectedSuccess) th } return putResp.getPrevKv(); } else { - assertFalse(txnResp.getSucceeded()); + assertFalse(txnResp.isSucceeded()); RangeResponse rangeResp = respOp.getResponseRange(); if (rangeResp.getCount() == 0) { return null; } else { assertEquals(1, rangeResp.getCount()); - return rangeResp.getKvs(0); + return rangeResp.getKvAt(0); } } } TxnResponse vPutToTableStore(int key, int value, long version) throws Exception { - return result( - tableStore.txn(TxnRequest.newBuilder() - .setHeader(HEADER) - .addCompare(Compare.newBuilder() - .setResult(CompareResult.EQUAL) - .setTarget(CompareTarget.VERSION) - .setKey(getKey(key)) - .setVersion(version)) - .addSuccess(RequestOp.newBuilder() - .setRequestPut(PutRequest.newBuilder() - .setHeader(HEADER) - .setKey(getKey(key)) - .setValue(getValue(value)) - .setPrevKv(true) - .build())) - .addFailure(RequestOp.newBuilder() - .setRequestRange(RangeRequest.newBuilder() - .setHeader(HEADER) - .setKey(getKey(key)) - .build())) - .build())); + TxnRequest req = new TxnRequest(); + req.setHeader().copyFrom(HEADER); + + Compare compare = req.addCompare(); + compare.setResult(CompareResult.EQUAL); + compare.setTarget(CompareTarget.VERSION); + compare.setKey(getKey(key)); + compare.setVersion(version); + + PutRequest putReq = new PutRequest() + .setKey(getKey(key)) + .setValue(getValue(value)) + .setPrevKv(true); + putReq.setHeader().copyFrom(HEADER); + req.addSuccess().setRequestPut().copyFrom(putReq); + + RangeRequest rangeReq = new RangeRequest() + .setKey(getKey(key)); + rangeReq.setHeader().copyFrom(HEADER); + req.addFailure().setRequestRange().copyFrom(rangeReq); + + return result(tableStore.txn(req)); } KeyValue verifyVPutResponse(TxnResponse txnResp, boolean expectedSuccess) throws Exception { assertEquals(HEADER, txnResp.getHeader().getRoutingHeader()); assertEquals(StatusCode.SUCCESS, txnResp.getHeader().getCode()); - ResponseOp respOp = txnResp.getResponses(0); + ResponseOp respOp = txnResp.getResponseAt(0); if (expectedSuccess) { - assertTrue(txnResp.getSucceeded()); + assertTrue(txnResp.isSucceeded()); PutResponse putResp = respOp.getResponsePut(); assertEquals(HEADER, putResp.getHeader().getRoutingHeader()); if (!putResp.hasPrevKv()) { @@ -232,67 +239,66 @@ KeyValue verifyVPutResponse(TxnResponse txnResp, boolean expectedSuccess) throws } return putResp.getPrevKv(); } else { - assertFalse(txnResp.getSucceeded()); + assertFalse(txnResp.isSucceeded()); RangeResponse rangeResp = respOp.getResponseRange(); if (rangeResp.getCount() == 0) { return null; } else { assertEquals(1, rangeResp.getCount()); - return rangeResp.getKvs(0); + return rangeResp.getKvAt(0); } } } TxnResponse rPutToTableStore(int key, int value, long revision) throws Exception { - return result( - tableStore.txn(TxnRequest.newBuilder() - .setHeader(HEADER) - .addCompare(Compare.newBuilder() - .setResult(CompareResult.EQUAL) - .setTarget(CompareTarget.MOD) - .setKey(getKey(key)) - .setModRevision(revision)) - .addSuccess(RequestOp.newBuilder() - .setRequestPut(PutRequest.newBuilder() - .setHeader(HEADER) - .setKey(getKey(key)) - .setValue(getValue(value)) - .setPrevKv(true) - .build())) - .addFailure(RequestOp.newBuilder() - .setRequestRange(RangeRequest.newBuilder() - .setHeader(HEADER) - .setKey(getKey(key)) - .build())) - .build())); + TxnRequest req = new TxnRequest(); + req.setHeader().copyFrom(HEADER); + + Compare compare = req.addCompare(); + compare.setResult(CompareResult.EQUAL); + compare.setTarget(CompareTarget.MOD); + compare.setKey(getKey(key)); + compare.setModRevision(revision); + + PutRequest putReq = new PutRequest() + .setKey(getKey(key)) + .setValue(getValue(value)) + .setPrevKv(true); + putReq.setHeader().copyFrom(HEADER); + req.addSuccess().setRequestPut().copyFrom(putReq); + + RangeRequest rangeReq = new RangeRequest() + .setKey(getKey(key)); + rangeReq.setHeader().copyFrom(HEADER); + req.addFailure().setRequestRange().copyFrom(rangeReq); + + return result(tableStore.txn(req)); } KeyValue deleteKeyFromTableStore(int key) throws Exception { - DeleteRangeResponse response = result( - tableStore.delete(DeleteRangeRequest.newBuilder() - .setHeader(HEADER) - .setKey(getKey(key)) - .setPrevKv(true) - .build())); + DeleteRangeRequest req = new DeleteRangeRequest() + .setKey(getKey(key)) + .setPrevKv(true); + req.setHeader().copyFrom(HEADER); + DeleteRangeResponse response = result(tableStore.delete(req)); assertEquals(StatusCode.SUCCESS, response.getHeader().getCode()); assertEquals(HEADER, response.getHeader().getRoutingHeader()); if (0 == response.getPrevKvsCount()) { return null; } else { - return response.getPrevKvs(0); + return response.getPrevKvAt(0); } } List deleteRange(int startKey, int endKey) throws Exception { - DeleteRangeResponse delResp = result( - tableStore.delete(DeleteRangeRequest.newBuilder() - .setHeader(HEADER) - .setKey(getKey(startKey)) - .setRangeEnd(getKey(endKey)) - .setPrevKv(true) - .build())); + DeleteRangeRequest req = new DeleteRangeRequest() + .setKey(getKey(startKey)) + .setRangeEnd(getKey(endKey)) + .setPrevKv(true); + req.setHeader().copyFrom(HEADER); + DeleteRangeResponse delResp = result(tableStore.delete(req)); assertEquals(StatusCode.SUCCESS, delResp.getHeader().getCode()); assertEquals(HEADER, delResp.getHeader().getRoutingHeader()); @@ -300,12 +306,11 @@ List deleteRange(int startKey, int endKey) throws Exception { } List range(int startKey, int endKey) throws Exception { - RangeResponse rangeResp = result( - tableStore.range(RangeRequest.newBuilder() - .setHeader(HEADER) - .setKey(getKey(startKey)) - .setRangeEnd(getKey(endKey)) - .build())); + RangeRequest req = new RangeRequest() + .setKey(getKey(startKey)) + .setRangeEnd(getKey(endKey)); + req.setHeader().copyFrom(HEADER); + RangeResponse rangeResp = result(tableStore.range(req)); assertEquals(StatusCode.SUCCESS, rangeResp.getHeader().getCode()); assertEquals(HEADER, rangeResp.getHeader().getRoutingHeader()); @@ -323,8 +328,8 @@ public void testBasicOps() throws Exception { putKeyToTableStore(0, 0); // get key(0) again kv = getKeyValue(0); - assertEquals(getKey(0), kv.getKey()); - assertEquals(getValue(0), kv.getValue()); + assertArrayEquals(getKey(0), kv.getKey()); + assertArrayEquals(getValue(0), kv.getValue()); } // putIfAbsent @@ -332,19 +337,19 @@ public void testBasicOps() throws Exception { // failure case KeyValue prevKV = putIfAbsentToTableStore(0, 99, false); assertNotNull(prevKV); - assertEquals(getKey(0), prevKV.getKey()); - assertEquals(getValue(0), prevKV.getValue()); + assertArrayEquals(getKey(0), prevKV.getKey()); + assertArrayEquals(getValue(0), prevKV.getValue()); // get key(0) KeyValue kv = getKeyValue(0); - assertEquals(getKey(0), kv.getKey()); - assertEquals(getValue(0), kv.getValue()); + assertArrayEquals(getKey(0), kv.getKey()); + assertArrayEquals(getValue(0), kv.getValue()); // success case prevKV = putIfAbsentToTableStore(1, 1, true); assertNull(prevKV); // get key(1) kv = getKeyValue(1); - assertEquals(getKey(1), kv.getKey()); - assertEquals(getValue(1), kv.getValue()); + assertArrayEquals(getKey(1), kv.getKey()); + assertArrayEquals(getValue(1), kv.getValue()); } // vPut @@ -367,13 +372,13 @@ public void testBasicOps() throws Exception { assertEquals(StatusCode.SUCCESS, response.getHeader().getCode()); prevKV = verifyVPutResponse(response, true); assertNotNull(prevKV); - assertEquals(getKey(key), prevKV.getKey()); - assertEquals(getValue(initialVal), prevKV.getValue()); + assertArrayEquals(getKey(key), prevKV.getKey()); + assertArrayEquals(getValue(initialVal), prevKV.getValue()); assertEquals(0, prevKV.getVersion()); KeyValue kv = getKeyValue(key); - assertEquals(getKey(key), kv.getKey()); - assertEquals(getValue(casVal), kv.getValue()); + assertArrayEquals(getKey(key), kv.getKey()); + assertArrayEquals(getValue(casVal), kv.getValue()); } // rPut @@ -396,7 +401,7 @@ public void testBasicOps() throws Exception { KeyValue kv = getKeyValue(key); long revision = kv.getModRevision(); - assertEquals(getValue(initialVal), kv.getValue()); + assertArrayEquals(getValue(initialVal), kv.getValue()); // rPut(key2, v, 0) response = rPutToTableStore(key, casVal, revision); @@ -404,7 +409,7 @@ public void testBasicOps() throws Exception { kv = getKeyValue(key); assertEquals(revision + 1, kv.getModRevision()); - assertEquals(getValue(casVal), kv.getValue()); + assertArrayEquals(getValue(casVal), kv.getValue()); } // delete(k) @@ -415,12 +420,12 @@ public void testBasicOps() throws Exception { // key exists int key = 0; kv = getKeyValue(key); - assertEquals(getKey(key), kv.getKey()); - assertEquals(getValue(key), kv.getValue()); + assertArrayEquals(getKey(key), kv.getKey()); + assertArrayEquals(getValue(key), kv.getValue()); kv = deleteKeyFromTableStore(key); assertNotNull(kv); - assertEquals(getKey(key), kv.getKey()); - assertEquals(getValue(key), kv.getValue()); + assertArrayEquals(getKey(key), kv.getKey()); + assertArrayEquals(getValue(key), kv.getValue()); // the key/value pair should not exist after deletion. kv = getKeyValue(key); assertNull(kv); @@ -470,8 +475,8 @@ private void verifyRecords(List kvs, int expectedVersion) { int idx = startKey; for (KeyValue kv : kvs) { - assertEquals(getKey(idx), kv.getKey()); - assertEquals(getValue(idx), kv.getValue()); + assertArrayEquals(getKey(idx), kv.getKey()); + assertArrayEquals(getValue(idx), kv.getValue()); // revision - starts from 1, but the first revision is used for nop barrier record. assertEquals(idx + startCreateRevision, kv.getCreateRevision()); assertEquals(idx + startModRevision, kv.getModRevision()); diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreUtilsTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreUtilsTest.java index dfdd69e50d1..e4bf5369a38 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreUtilsTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/kv/TableStoreUtilsTest.java @@ -32,7 +32,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import com.google.protobuf.ByteString; +import java.nio.charset.StandardCharsets; import java.util.concurrent.ExecutionException; import org.apache.bookkeeper.api.kv.result.Code; import org.apache.bookkeeper.api.kv.result.KeyValue; @@ -62,34 +62,30 @@ public class TableStoreUtilsTest { @Test public void testHasRKey() { assertFalse(hasRKey(null)); - assertFalse(hasRKey(ByteString.copyFrom(new byte[0]))); - assertTrue(hasRKey(ByteString.copyFromUtf8("test"))); + assertFalse(hasRKey(new byte[0])); + assertTrue(hasRKey("test".getBytes(StandardCharsets.UTF_8))); } @Test public void testNewStoreKey() { assertArrayEquals( storeKeyWithRKey, - newStoreKey( - ByteString.copyFrom(rKey), - ByteString.copyFrom(lKey)) + newStoreKey(rKey, lKey) ); assertArrayEquals( storeKeyWithoutRKey, - newStoreKey( - null, - ByteString.copyFrom(lKey)) + newStoreKey(null, lKey) ); } @Test public void testGetLKey() { - assertEquals( - ByteString.copyFrom(lKey), + assertArrayEquals( + lKey, getLKey(storeKeyWithoutRKey, null)); - assertEquals( - ByteString.copyFrom(lKey), - getLKey(storeKeyWithRKey, ByteString.copyFrom(rKey))); + assertArrayEquals( + lKey, + getLKey(storeKeyWithRKey, rKey)); } @Test @@ -153,10 +149,10 @@ public void testNewKeyValue() { when(kv.version()).thenReturn(version); org.apache.bookkeeper.stream.proto.kv.KeyValue keyValue = - newKeyValue(ByteString.copyFrom(rKey), kv); + newKeyValue(rKey, kv); - assertEquals(ByteString.copyFrom(lKey), keyValue.getKey()); - assertEquals(ByteString.copyFrom(value), keyValue.getValue()); + assertArrayEquals(lKey, keyValue.getKey()); + assertArrayEquals(value, keyValue.getValue()); assertEquals(createRev, keyValue.getCreateRevision()); assertEquals(modRev, keyValue.getModRevision()); assertEquals(version, keyValue.getVersion()); diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImplTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImplTest.java index 5e5f4b3d7e8..49c57b903cb 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImplTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/MetaRangeStoreImplTest.java @@ -25,7 +25,6 @@ import static org.mockito.Mockito.when; import com.google.common.collect.Lists; -import java.util.Collections; import java.util.List; import java.util.NavigableMap; import java.util.stream.LongStream; @@ -56,12 +55,11 @@ public class MetaRangeStoreImplTest extends MVCCAsyncStoreTestBase { @Override protected void doSetup() throws Exception { - this.streamProps = StreamProperties.newBuilder() + this.streamProps = new StreamProperties() .setStorageContainerId(1234L) - .setStreamConf(DEFAULT_STREAM_CONF) .setStreamName(name.getMethodName() + "_stream") - .setStreamId(System.currentTimeMillis()) - .build(); + .setStreamId(System.currentTimeMillis()); + this.streamProps.setStreamConf().copyFrom(DEFAULT_STREAM_CONF); this.clientManager = mock(StorageServerClientManager.class); this.mrStoreImpl = new MetaRangeStoreImpl( this.store, @@ -77,9 +75,8 @@ protected void doTeardown() throws Exception { GetActiveRangesRequest createRequest(StreamProperties streamProperties) { when(clientManager.getStreamProperties(eq(this.streamProps.getStreamId()))) .thenReturn(FutureUtils.value(streamProperties)); - GetActiveRangesRequest.Builder reqBuilder = GetActiveRangesRequest.newBuilder() + return new GetActiveRangesRequest() .setStreamId(this.streamProps.getStreamId()); - return reqBuilder.build(); } @Test @@ -118,7 +115,7 @@ private void verifyGetResponse(GetActiveRangesResponse getResp) throws Exception metaRange.unsafeGetRanges().get(expectedRid); assertNotNull(expectedRangeMetadata); - assertEquals(Collections.emptyList(), actualRR.getRelatedRangesList()); + assertEquals(0, actualRR.getRelatedRangesCount()); assertEquals(expectedRangeMetadata.getProps(), actualRR.getProps()); } @@ -192,7 +189,8 @@ private void readRangeMetadataAndVerify(long streamId, RangeState expectedRangeState) throws Exception { byte[] rangeKey = MetaRangeImpl.getStreamRangeKey(streamId, rangeId); byte[] rangeMetadataBytes = FutureUtils.result(store.get(rangeKey)); - RangeMetadata rangeMetadata = RangeMetadata.parseFrom(rangeMetadataBytes); + RangeMetadata rangeMetadata = new RangeMetadata(); + rangeMetadata.parseFrom(rangeMetadataBytes); verifyRangeMetadata( rangeMetadata, diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/TestRootRangeStoreImpl.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/TestRootRangeStoreImpl.java index ed304fa767c..dbf97cae992 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/TestRootRangeStoreImpl.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/TestRootRangeStoreImpl.java @@ -56,14 +56,21 @@ @CustomLog public class TestRootRangeStoreImpl extends MVCCAsyncStoreTestBase { - private final NamespaceConfiguration namespaceConf = - NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build(); - - private final StreamConfiguration streamConf = - StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) - .build(); + private final NamespaceConfiguration namespaceConf = newNamespaceConf(); + + private static NamespaceConfiguration newNamespaceConf() { + NamespaceConfiguration conf = new NamespaceConfiguration(); + conf.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); + return conf; + } + + private final StreamConfiguration streamConf = newStreamConf(); + + private static StreamConfiguration newStreamConf() { + StreamConfiguration conf = new StreamConfiguration(); + conf.copyFrom(DEFAULT_STREAM_CONF); + return conf; + } private RootRangeStoreImpl rootRangeStore; diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/TestMetaRangeImpl.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/TestMetaRangeImpl.java index 6cee1aba839..e33a9e063ce 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/TestMetaRangeImpl.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/metadata/stream/TestMetaRangeImpl.java @@ -52,12 +52,11 @@ public class TestMetaRangeImpl extends MVCCAsyncStoreTestBase { @Override protected void doSetup() { - this.streamProps = StreamProperties.newBuilder() + this.streamProps = new StreamProperties() .setStorageContainerId(1234L) - .setStreamConf(DEFAULT_STREAM_CONF) .setStreamName(name.getMethodName() + "_stream") - .setStreamId(System.currentTimeMillis()) - .build(); + .setStreamId(System.currentTimeMillis()); + this.streamProps.setStreamConf().copyFrom(DEFAULT_STREAM_CONF); this.metaRange = new MetaRangeImpl( this.store, this.scheduler.chooseThread(), @@ -166,7 +165,8 @@ private void readRangeMetadataAndVerify(long streamId, RangeState expectedRangeState) throws Exception { byte[] rangeKey = MetaRangeImpl.getStreamRangeKey(streamId, rangeId); byte[] rangeMetadataBytes = FutureUtils.result(store.get(rangeKey)); - RangeMetadata rangeMetadata = RangeMetadata.parseFrom(rangeMetadataBytes); + RangeMetadata rangeMetadata = new RangeMetadata(); + rangeMetadata.parseFrom(rangeMetadataBytes); verifyRangeMetadata( rangeMetadata, diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/RangeRoutingTableImplTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/RangeRoutingTableImplTest.java index 8201ba714d7..13eda2b16af 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/RangeRoutingTableImplTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/RangeRoutingTableImplTest.java @@ -27,7 +27,6 @@ import static org.junit.Assert.fail; import io.grpc.stub.StreamObserver; -import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -68,37 +67,33 @@ public class RangeRoutingTableImplTest extends GrpcClientTestBase { @Override protected void doSetup() throws Exception { - this.props = StreamProperties.newBuilder() + this.props = new StreamProperties() .setStorageContainerId(scId) .setStreamId(streamId) - .setStreamName("metaclient-stream") - .setStreamConf(StreamConfiguration.newBuilder().build()) - .build(); + .setStreamName("metaclient-stream"); + this.props.setStreamConf().copyFrom(new StreamConfiguration()); this.rangeProps = ProtoUtils.split( streamId, 24, 23456L, StorageContainerPlacementPolicyImpl.of(4) ); - final GetActiveRangesResponse.Builder getActiveRangesResponseBuilder = GetActiveRangesResponse.newBuilder(); + final GetActiveRangesResponse getActiveRangesResp = new GetActiveRangesResponse(); for (RangeProperties range : rangeProps) { - RelatedRanges.Builder rrBuilder = RelatedRanges.newBuilder() - .setProps(range) - .setType(RelationType.PARENTS) - .addAllRelatedRanges(Collections.emptyList()); - getActiveRangesResponseBuilder.addRanges(rrBuilder); + RelatedRanges rr = getActiveRangesResp.addRange(); + rr.setProps().copyFrom(range); + rr.setType(RelationType.PARENTS); } - this.getActiveRangesResponse = getActiveRangesResponseBuilder - .setCode(StatusCode.SUCCESS) - .build(); + getActiveRangesResp.setCode(StatusCode.SUCCESS); + this.getActiveRangesResponse = getActiveRangesResp; RootRangeServiceImplBase rootRangeService = new RootRangeServiceImplBase() { @Override public void getStream(GetStreamRequest request, StreamObserver responseObserver) { - responseObserver.onNext(GetStreamResponse.newBuilder() - .setCode(StatusCode.SUCCESS) - .setStreamProps(props) - .build()); + GetStreamResponse resp = new GetStreamResponse() + .setCode(StatusCode.SUCCESS); + resp.setStreamProps().copyFrom(props); + responseObserver.onNext(resp); responseObserver.onCompleted(); } }; diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptorTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptorTest.java index 0442d6a89f8..17f2ce93195 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptorTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/RoutingHeaderProxyInterceptorTest.java @@ -25,7 +25,6 @@ import static org.apache.bookkeeper.stream.protocol.ProtocolConstants.SID_METADATA_KEY; import static org.junit.Assert.assertEquals; -import com.google.protobuf.ByteString; import io.grpc.CallOptions; import io.grpc.Channel; import io.grpc.ClientCall; @@ -68,6 +67,46 @@ public class RoutingHeaderProxyInterceptorTest extends GrpcClientTestBase { private final AtomicReference receivedRequest = new AtomicReference<>(); private StorageServerChannel channel; + private static RangeResponse newRangeResponse(RoutingHeader header) { + RangeResponse resp = new RangeResponse(); + ResponseHeader rh = resp.setHeader(); + rh.setCode(StatusCode.SUCCESS); + rh.setRoutingHeader().copyFrom(header); + return resp; + } + + private static DeleteRangeResponse newDeleteRangeResponse(RoutingHeader header) { + DeleteRangeResponse resp = new DeleteRangeResponse(); + ResponseHeader rh = resp.setHeader(); + rh.setCode(StatusCode.SUCCESS); + rh.setRoutingHeader().copyFrom(header); + return resp; + } + + private static TxnResponse newTxnResponse(RoutingHeader header) { + TxnResponse resp = new TxnResponse(); + ResponseHeader rh = resp.setHeader(); + rh.setCode(StatusCode.SUCCESS); + rh.setRoutingHeader().copyFrom(header); + return resp; + } + + private static IncrementResponse newIncrementResponse(RoutingHeader header) { + IncrementResponse resp = new IncrementResponse(); + ResponseHeader rh = resp.setHeader(); + rh.setCode(StatusCode.SUCCESS); + rh.setRoutingHeader().copyFrom(header); + return resp; + } + + private static PutResponse newPutResponse(RoutingHeader header) { + PutResponse resp = new PutResponse(); + ResponseHeader rh = resp.setHeader(); + rh.setCode(StatusCode.SUCCESS); + rh.setRoutingHeader().copyFrom(header); + return resp; + } + @Override protected void doSetup() { TableServiceImplBase tableService = new TableServiceImplBase() { @@ -76,12 +115,7 @@ protected void doSetup() { public void range(RangeRequest request, StreamObserver responseObserver) { log.info().attr("request", request).log("Received range request"); receivedRequest.set(request); - responseObserver.onNext(RangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.SUCCESS) - .setRoutingHeader(request.getHeader()) - .build()) - .build()); + responseObserver.onNext(newRangeResponse(request.getHeader())); responseObserver.onCompleted(); } @@ -89,12 +123,7 @@ public void range(RangeRequest request, StreamObserver responseOb public void delete(DeleteRangeRequest request, StreamObserver responseObserver) { log.info().attr("request", request).log("Received delete range request"); receivedRequest.set(request); - responseObserver.onNext(DeleteRangeResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.SUCCESS) - .setRoutingHeader(request.getHeader()) - .build()) - .build()); + responseObserver.onNext(newDeleteRangeResponse(request.getHeader())); responseObserver.onCompleted(); } @@ -102,12 +131,7 @@ public void delete(DeleteRangeRequest request, StreamObserver responseObserver) { log.info().attr("request", request).log("Received txn request"); receivedRequest.set(request); - responseObserver.onNext(TxnResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.SUCCESS) - .setRoutingHeader(request.getHeader()) - .build()) - .build()); + responseObserver.onNext(newTxnResponse(request.getHeader())); responseObserver.onCompleted(); } @@ -115,12 +139,7 @@ public void txn(TxnRequest request, StreamObserver responseObserver public void increment(IncrementRequest request, StreamObserver responseObserver) { log.info().attr("request", request).log("Received incr request"); receivedRequest.set(request); - responseObserver.onNext(IncrementResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.SUCCESS) - .setRoutingHeader(request.getHeader()) - .build()) - .build()); + responseObserver.onNext(newIncrementResponse(request.getHeader())); responseObserver.onCompleted(); } @@ -128,12 +147,7 @@ public void increment(IncrementRequest request, StreamObserver responseObserver) { log.info().attr("request", request).log("Received put request"); receivedRequest.set(request); - responseObserver.onNext(PutResponse.newBuilder() - .setHeader(ResponseHeader.newBuilder() - .setCode(StatusCode.SUCCESS) - .setRoutingHeader(request.getHeader()) - .build()) - .build()); + responseObserver.onNext(newPutResponse(request.getHeader())); responseObserver.onCompleted(); } }; @@ -183,18 +197,20 @@ protected void doTeardown() { channel.close(); } + private static RoutingHeader newRoutingHeader(long streamId, long rangeId, byte[] rk) { + return new RoutingHeader() + .setStreamId(streamId) + .setRangeId(rangeId) + .setRKey(rk); + } + @Test public void testPutRequest() throws Exception { - PutRequest request = PutRequest.newBuilder() - .setKey(ByteString.copyFromUtf8("test-key")) - .build(); - PutRequest expectedRequest = PutRequest.newBuilder(request) - .setHeader(RoutingHeader.newBuilder(request.getHeader()) - .setStreamId(streamId) - .setRangeId(rangeId) - .setRKey(ByteString.copyFrom(routingKey)) - .build()) - .build(); + PutRequest request = new PutRequest() + .setKey("test-key".getBytes(UTF_8)); + PutRequest expectedRequest = new PutRequest(); + expectedRequest.copyFrom(request); + expectedRequest.setHeader().copyFrom(newRoutingHeader(streamId, rangeId, routingKey)); PutResponse response = this.channel.getTableService().put(request).get(); assertEquals(expectedRequest, receivedRequest.get()); @@ -203,16 +219,11 @@ public void testPutRequest() throws Exception { @Test public void testRangeRequest() throws Exception { - RangeRequest request = RangeRequest.newBuilder() - .setKey(ByteString.copyFromUtf8("test-key")) - .build(); - RangeRequest expectedRequest = RangeRequest.newBuilder(request) - .setHeader(RoutingHeader.newBuilder(request.getHeader()) - .setStreamId(streamId) - .setRangeId(rangeId) - .setRKey(ByteString.copyFrom(routingKey)) - .build()) - .build(); + RangeRequest request = new RangeRequest() + .setKey("test-key".getBytes(UTF_8)); + RangeRequest expectedRequest = new RangeRequest(); + expectedRequest.copyFrom(request); + expectedRequest.setHeader().copyFrom(newRoutingHeader(streamId, rangeId, routingKey)); RangeResponse response = this.channel.getTableService() .range(request).get(); @@ -222,16 +233,11 @@ public void testRangeRequest() throws Exception { @Test public void testDeleteRangeRequest() throws Exception { - DeleteRangeRequest request = DeleteRangeRequest.newBuilder() - .setKey(ByteString.copyFromUtf8("test-key")) - .build(); - DeleteRangeRequest expectedRequest = DeleteRangeRequest.newBuilder(request) - .setHeader(RoutingHeader.newBuilder(request.getHeader()) - .setStreamId(streamId) - .setRangeId(rangeId) - .setRKey(ByteString.copyFrom(routingKey)) - .build()) - .build(); + DeleteRangeRequest request = new DeleteRangeRequest() + .setKey("test-key".getBytes(UTF_8)); + DeleteRangeRequest expectedRequest = new DeleteRangeRequest(); + expectedRequest.copyFrom(request); + expectedRequest.setHeader().copyFrom(newRoutingHeader(streamId, rangeId, routingKey)); DeleteRangeResponse response = this.channel.getTableService() .delete(request).get(); @@ -241,16 +247,11 @@ public void testDeleteRangeRequest() throws Exception { @Test public void testIncrementRequest() throws Exception { - IncrementRequest request = IncrementRequest.newBuilder() - .setKey(ByteString.copyFromUtf8("test-key")) - .build(); - IncrementRequest expectedRequest = IncrementRequest.newBuilder(request) - .setHeader(RoutingHeader.newBuilder(request.getHeader()) - .setStreamId(streamId) - .setRangeId(rangeId) - .setRKey(ByteString.copyFrom(routingKey)) - .build()) - .build(); + IncrementRequest request = new IncrementRequest() + .setKey("test-key".getBytes(UTF_8)); + IncrementRequest expectedRequest = new IncrementRequest(); + expectedRequest.copyFrom(request); + expectedRequest.setHeader().copyFrom(newRoutingHeader(streamId, rangeId, routingKey)); IncrementResponse response = this.channel.getTableService() .increment(request).get(); @@ -260,15 +261,10 @@ public void testIncrementRequest() throws Exception { @Test public void testTxnRequest() throws Exception { - TxnRequest request = TxnRequest.newBuilder() - .build(); - TxnRequest expectedRequest = TxnRequest.newBuilder(request) - .setHeader(RoutingHeader.newBuilder(request.getHeader()) - .setStreamId(streamId) - .setRangeId(rangeId) - .setRKey(ByteString.copyFrom(routingKey)) - .build()) - .build(); + TxnRequest request = new TxnRequest(); + TxnRequest expectedRequest = new TxnRequest(); + expectedRequest.copyFrom(request); + expectedRequest.setHeader().copyFrom(newRoutingHeader(streamId, rangeId, routingKey)); TxnResponse response = this.channel.getTableService().txn(request).get(); assertEquals(expectedRequest, receivedRequest.get()); diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/StorageContainerProxyChannelManagerImplTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/StorageContainerProxyChannelManagerImplTest.java index bb94843fe06..7181d2db0ad 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/StorageContainerProxyChannelManagerImplTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/routing/StorageContainerProxyChannelManagerImplTest.java @@ -91,19 +91,16 @@ public void getStorageContainerEndpoint( } private static GetStorageContainerEndpointResponse getResponse(GetStorageContainerEndpointRequest request) { - GetStorageContainerEndpointResponse.Builder respBuilder = - GetStorageContainerEndpointResponse.newBuilder(); - respBuilder.setStatusCode(StatusCode.SUCCESS); + GetStorageContainerEndpointResponse resp = new GetStorageContainerEndpointResponse(); + resp.setStatusCode(StatusCode.SUCCESS); for (OneStorageContainerEndpointRequest oneReq : request.getRequestsList()) { - OneStorageContainerEndpointResponse oneResp = OneStorageContainerEndpointResponse.newBuilder() - .setEndpoint(StorageContainerEndpoint.newBuilder() - .setStorageContainerId(oneReq.getStorageContainer()) - .setRevision(oneReq.getRevision() + 1) - .setRwEndpoint(ENDPOINT)) - .build(); - respBuilder.addResponses(oneResp); + OneStorageContainerEndpointResponse oneResp = resp.addResponse(); + StorageContainerEndpoint endpoint = oneResp.setEndpoint(); + endpoint.setStorageContainerId(oneReq.getStorageContainer()); + endpoint.setRevision(oneReq.getRevision() + 1); + endpoint.setRwEndpoint().copyFrom(ENDPOINT); } - return respBuilder.build(); + return resp; } diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/sc/ZkStorageContainerManagerTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/sc/ZkStorageContainerManagerTest.java index 69cdf762576..ee185e519fe 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/sc/ZkStorageContainerManagerTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/sc/ZkStorageContainerManagerTest.java @@ -64,10 +64,9 @@ public class ZkStorageContainerManagerTest extends ZooKeeperClusterTestCase { @Rule public final TestName runtime = new TestName(); - private final Endpoint myEndpoint = Endpoint.newBuilder() + private final Endpoint myEndpoint = new Endpoint() .setHostname("127.0.0.1") - .setPort(4181) - .build(); + .setPort(4181); private CuratorFramework curatorClient; private StorageContainerFactory mockScFactory; diff --git a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImplTest.java b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImplTest.java index 6b678393605..fafa4c5f7ca 100644 --- a/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImplTest.java +++ b/stream/storage/impl/src/test/java/org/apache/bookkeeper/stream/storage/impl/service/RangeStoreServiceImplTest.java @@ -165,7 +165,7 @@ private void mockStorageContainer(long scId) { when(tableStoreFactory.createStore(eq(trMvccStore))) .thenReturn(trStore); when(clientManager.getStreamProperties(eq(STREAM_ID))) - .thenReturn(FutureUtils.value(StreamProperties.getDefaultInstance())); + .thenReturn(FutureUtils.value(new StreamProperties())); } @Test @@ -235,11 +235,11 @@ public void testClose() throws Exception { public void testCreateNamespace() throws Exception { mockStorageContainer(SCID); - CreateNamespaceResponse expectedResp = CreateNamespaceResponse.getDefaultInstance(); + CreateNamespaceResponse expectedResp = new CreateNamespaceResponse(); when(rrStore.createNamespace(any(CreateNamespaceRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); - CreateNamespaceRequest expectedReq = CreateNamespaceRequest.getDefaultInstance(); + CreateNamespaceRequest expectedReq = new CreateNamespaceRequest(); assertSame( expectedResp, FutureUtils.result(rrStore.createNamespace(expectedReq))); @@ -251,11 +251,11 @@ public void testCreateNamespace() throws Exception { public void testDeleteNamespace() throws Exception { mockStorageContainer(SCID); - DeleteNamespaceResponse expectedResp = DeleteNamespaceResponse.getDefaultInstance(); + DeleteNamespaceResponse expectedResp = new DeleteNamespaceResponse(); when(rrStore.deleteNamespace(any(DeleteNamespaceRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); - DeleteNamespaceRequest expectedReq = DeleteNamespaceRequest.getDefaultInstance(); + DeleteNamespaceRequest expectedReq = new DeleteNamespaceRequest(); assertSame( expectedResp, FutureUtils.result(rrStore.deleteNamespace(expectedReq))); @@ -267,11 +267,11 @@ public void testDeleteNamespace() throws Exception { public void testGetNamespace() throws Exception { mockStorageContainer(SCID); - GetNamespaceResponse expectedResp = GetNamespaceResponse.getDefaultInstance(); + GetNamespaceResponse expectedResp = new GetNamespaceResponse(); when(rrStore.getNamespace(any(GetNamespaceRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); - GetNamespaceRequest expectedReq = GetNamespaceRequest.getDefaultInstance(); + GetNamespaceRequest expectedReq = new GetNamespaceRequest(); assertSame( expectedResp, FutureUtils.result(rrStore.getNamespace(expectedReq))); @@ -283,11 +283,11 @@ public void testGetNamespace() throws Exception { public void testCreateStream() throws Exception { mockStorageContainer(SCID); - CreateStreamResponse expectedResp = CreateStreamResponse.getDefaultInstance(); + CreateStreamResponse expectedResp = new CreateStreamResponse(); when(rrStore.createStream(any(CreateStreamRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); - CreateStreamRequest expectedReq = CreateStreamRequest.getDefaultInstance(); + CreateStreamRequest expectedReq = new CreateStreamRequest(); assertSame( expectedResp, FutureUtils.result(rrStore.createStream(expectedReq))); @@ -299,11 +299,11 @@ public void testCreateStream() throws Exception { public void testDeleteStream() throws Exception { mockStorageContainer(SCID); - DeleteStreamResponse expectedResp = DeleteStreamResponse.getDefaultInstance(); + DeleteStreamResponse expectedResp = new DeleteStreamResponse(); when(rrStore.deleteStream(any(DeleteStreamRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); - DeleteStreamRequest expectedReq = DeleteStreamRequest.getDefaultInstance(); + DeleteStreamRequest expectedReq = new DeleteStreamRequest(); assertSame( expectedResp, FutureUtils.result(rrStore.deleteStream(expectedReq))); @@ -315,11 +315,11 @@ public void testDeleteStream() throws Exception { public void testGetStream() throws Exception { mockStorageContainer(SCID); - GetStreamResponse expectedResp = GetStreamResponse.getDefaultInstance(); + GetStreamResponse expectedResp = new GetStreamResponse(); when(rrStore.getStream(any(GetStreamRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); - GetStreamRequest expectedReq = GetStreamRequest.getDefaultInstance(); + GetStreamRequest expectedReq = new GetStreamRequest(); assertSame( expectedResp, FutureUtils.result(rrStore.getStream(expectedReq))); @@ -335,11 +335,11 @@ public void testGetStream() throws Exception { public void testGetActiveRanges() throws Exception { mockStorageContainer(SCID); - GetActiveRangesResponse expectedResp = GetActiveRangesResponse.getDefaultInstance(); + GetActiveRangesResponse expectedResp = new GetActiveRangesResponse(); when(mrStore.getActiveRanges(any(GetActiveRangesRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); - GetActiveRangesRequest expectedReq = GetActiveRangesRequest.getDefaultInstance(); + GetActiveRangesRequest expectedReq = new GetActiveRangesRequest(); assertSame( expectedResp, FutureUtils.result(mrStore.getActiveRanges(expectedReq))); @@ -351,61 +351,47 @@ public void testGetActiveRanges() throws Exception { // Table API // - private PutRequest newPutRequest() { - RoutingHeader header = RoutingHeader.newBuilder() + private static RoutingHeader newRoutingHeader() { + return new RoutingHeader() .setStreamId(STREAM_ID) - .setRangeId(RANGE_ID) - .build(); - return PutRequest.newBuilder() - .setHeader(header) - .build(); + .setRangeId(RANGE_ID); + } + + private PutRequest newPutRequest() { + PutRequest req = new PutRequest(); + req.setHeader().copyFrom(newRoutingHeader()); + return req; } private DeleteRangeRequest newDeleteRequest() { - RoutingHeader header = RoutingHeader.newBuilder() - .setStreamId(STREAM_ID) - .setRangeId(RANGE_ID) - .build(); - return DeleteRangeRequest.newBuilder() - .setHeader(header) - .build(); + DeleteRangeRequest req = new DeleteRangeRequest(); + req.setHeader().copyFrom(newRoutingHeader()); + return req; } private RangeRequest newRangeRequest() { - RoutingHeader header = RoutingHeader.newBuilder() - .setStreamId(STREAM_ID) - .setRangeId(RANGE_ID) - .build(); - return RangeRequest.newBuilder() - .setHeader(header) - .build(); + RangeRequest req = new RangeRequest(); + req.setHeader().copyFrom(newRoutingHeader()); + return req; } private IncrementRequest newIncrRequest() { - RoutingHeader header = RoutingHeader.newBuilder() - .setStreamId(STREAM_ID) - .setRangeId(RANGE_ID) - .build(); - return IncrementRequest.newBuilder() - .setHeader(header) - .build(); + IncrementRequest req = new IncrementRequest(); + req.setHeader().copyFrom(newRoutingHeader()); + return req; } private TxnRequest newTxnRequest() { - RoutingHeader header = RoutingHeader.newBuilder() - .setStreamId(STREAM_ID) - .setRangeId(RANGE_ID) - .build(); - return TxnRequest.newBuilder() - .setHeader(header) - .build(); + TxnRequest req = new TxnRequest(); + req.setHeader().copyFrom(newRoutingHeader()); + return req; } @Test public void testRangeWhenTableStoreNotCached() throws Exception { mockStorageContainer(SCID); - RangeResponse expectedResp = RangeResponse.getDefaultInstance(); + RangeResponse expectedResp = new RangeResponse(); when(trStore.range(any(RangeRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); @@ -419,7 +405,7 @@ public void testRangeWhenTableStoreNotCached() throws Exception { public void testRangeWhenTableStoreCached() throws Exception { mockStorageContainer(SCID); - RangeResponse expectedResp = RangeResponse.getDefaultInstance(); + RangeResponse expectedResp = new RangeResponse(); when(trStore.range(any(RangeRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); container.getTableStoreCache().getTableStores().put(RID, trStore); @@ -433,7 +419,7 @@ public void testRangeWhenTableStoreCached() throws Exception { public void testPutWhenTableStoreNotCached() throws Exception { mockStorageContainer(SCID); - PutResponse expectedResp = PutResponse.getDefaultInstance(); + PutResponse expectedResp = new PutResponse(); when(trStore.put(any(PutRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); @@ -447,7 +433,7 @@ public void testPutWhenTableStoreNotCached() throws Exception { public void testPutWhenTableStoreCached() throws Exception { mockStorageContainer(SCID); - PutResponse expectedResp = PutResponse.getDefaultInstance(); + PutResponse expectedResp = new PutResponse(); when(trStore.put(any(PutRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); container.getTableStoreCache().getTableStores().put(RID, trStore); @@ -461,7 +447,7 @@ public void testPutWhenTableStoreCached() throws Exception { public void testDeleteWhenTableStoreNotCached() throws Exception { mockStorageContainer(SCID); - DeleteRangeResponse expectedResp = DeleteRangeResponse.getDefaultInstance(); + DeleteRangeResponse expectedResp = new DeleteRangeResponse(); when(trStore.delete(any(DeleteRangeRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); @@ -475,7 +461,7 @@ public void testDeleteWhenTableStoreNotCached() throws Exception { public void testDeleteWhenTableStoreCached() throws Exception { mockStorageContainer(SCID); - DeleteRangeResponse expectedResp = DeleteRangeResponse.getDefaultInstance(); + DeleteRangeResponse expectedResp = new DeleteRangeResponse(); when(trStore.delete(any(DeleteRangeRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); container.getTableStoreCache().getTableStores().put(RID, trStore); @@ -489,7 +475,7 @@ public void testDeleteWhenTableStoreCached() throws Exception { public void testTxnWhenTableStoreNotCached() throws Exception { mockStorageContainer(SCID); - TxnResponse expectedResp = TxnResponse.getDefaultInstance(); + TxnResponse expectedResp = new TxnResponse(); when(trStore.txn(any(TxnRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); @@ -503,7 +489,7 @@ public void testTxnWhenTableStoreNotCached() throws Exception { public void testTxnWhenTableStoreCached() throws Exception { mockStorageContainer(SCID); - TxnResponse expectedResp = TxnResponse.getDefaultInstance(); + TxnResponse expectedResp = new TxnResponse(); when(trStore.txn(any(TxnRequest.class))) .thenReturn(FutureUtils.value(expectedResp)); container.getTableStoreCache().getTableStores().put(RID, trStore); diff --git a/stream/tests-common/pom.xml b/stream/tests-common/pom.xml index bdd3b8ab2d4..5b5c9a5cd9b 100644 --- a/stream/tests-common/pom.xml +++ b/stream/tests-common/pom.xml @@ -41,8 +41,8 @@ grpc-testing - com.google.protobuf - protobuf-java + io.netty + netty-buffer javax.annotation @@ -53,14 +53,16 @@ - - - kr.motd.maven - os-maven-plugin - ${os-maven-plugin.version} - - + + org.apache.rat + apache-rat-plugin + + + target/generated-sources/lightproto/** + + + org.apache.maven.plugins maven-compiler-plugin @@ -70,20 +72,16 @@ - org.xolstice.maven.plugins - protobuf-maven-plugin - ${protobuf-maven-plugin.version} + io.streamnative.lightproto + lightproto-maven-plugin + ${lightproto-maven-plugin.version} - com.google.protobuf:protoc:${protoc.version}:exe:${os.detected.classifier} - grpc-java - io.grpc:protoc-gen-grpc-java:${protoc-gen-grpc-java.version}:exe:${os.detected.classifier} - true + generated-sources/lightproto/java - compile - compile-custom + generate diff --git a/stream/tests-common/src/main/java/org/apache/bookkeeper/tests/rpc/PingPongService.java b/stream/tests-common/src/main/java/org/apache/bookkeeper/tests/rpc/PingPongService.java index 1b6d7af448d..077f99859c0 100644 --- a/stream/tests-common/src/main/java/org/apache/bookkeeper/tests/rpc/PingPongService.java +++ b/stream/tests-common/src/main/java/org/apache/bookkeeper/tests/rpc/PingPongService.java @@ -38,11 +38,10 @@ public PingPongService(int streamPongSize) { @Override public void pingPong(PingRequest request, StreamObserver responseObserver) { - responseObserver.onNext(PongResponse.newBuilder() + responseObserver.onNext(new PongResponse() .setLastSequence(request.getSequence()) .setNumPingReceived(1) - .setSlotId(0) - .build()); + .setSlotId(0)); responseObserver.onCompleted(); } @@ -66,11 +65,10 @@ public void onError(Throwable t) { @Override public void onCompleted() { - responseObserver.onNext(PongResponse.newBuilder() + responseObserver.onNext(new PongResponse() .setNumPingReceived(pingCount) .setLastSequence(lastSequence) - .setSlotId(0) - .build()); + .setSlotId(0)); responseObserver.onCompleted(); } }; @@ -80,11 +78,10 @@ public void onCompleted() { public void lotsOfPongs(PingRequest request, StreamObserver responseObserver) { long sequence = request.getSequence(); for (int i = 0; i < streamPongSize; i++) { - responseObserver.onNext(PongResponse.newBuilder() + responseObserver.onNext(new PongResponse() .setLastSequence(sequence) .setNumPingReceived(1) - .setSlotId(i) - .build()); + .setSlotId(i)); } responseObserver.onCompleted(); } @@ -98,11 +95,10 @@ public StreamObserver bidiPingPong(StreamObserver res @Override public void onNext(PingRequest ping) { pingCount++; - responseObserver.onNext(PongResponse.newBuilder() + responseObserver.onNext(new PongResponse() .setLastSequence(ping.getSequence()) .setNumPingReceived(pingCount) - .setSlotId(0) - .build()); + .setSlotId(0)); } @Override diff --git a/stream/tests-common/src/main/proto/proto2_coder_test_messages.proto b/stream/tests-common/src/main/proto/proto2_coder_test_messages.proto deleted file mode 100644 index 4818a683d48..00000000000 --- a/stream/tests-common/src/main/proto/proto2_coder_test_messages.proto +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Copyright (C) 2015 Google Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); you may not - * use this file except in compliance with the License. You may obtain a copy of - * the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT - * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the - * License for the specific language governing permissions and limitations under - * the License. - */ - -/* - * Protocol Buffer messages used for testing Proto2Coder implementation. - * - * Copy from {@link https://github.com/GoogleCloudPlatform/DataflowJavaSDK/blob/master/sdk/src/main/proto/proto2_coder_test_messages.proto} - */ - -syntax = "proto2"; - -package proto2_coder_test_messages; - -option java_package = "org.apache.bookkeeper.stream.coder.protobuf.test"; - -message MessageA { - optional string field1 = 1; - repeated MessageB field2 = 2; -} - -message MessageB { - optional bool field1 = 1; -} - -message MessageC { - extensions 100 to 105; -} - -extend MessageC { - optional MessageA field1 = 101; - optional MessageB field2 = 102; -} - -message MessageWithMap { - map field1 = 1; -} - -message ReferencesMessageWithMap { - repeated MessageWithMap field1 = 1; -} diff --git a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/LocationClientTest.java b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/LocationClientTest.java index d231ce9d83a..4031d555f8d 100644 --- a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/LocationClientTest.java +++ b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/LocationClientTest.java @@ -81,8 +81,8 @@ public void testLocateStorageContainers() throws Exception { log.info("Response : rw endpoint = {}", endpoint); assertTrue(getInternalStreamEndpoints().contains(endpoint)); - assertEquals(1, oneResponse.getEndpoint().getRoEndpointCount()); - endpoint = oneResponse.getEndpoint().getRoEndpoint(0); + assertEquals(1, oneResponse.getEndpoint().getRoEndpointsCount()); + endpoint = oneResponse.getEndpoint().getRoEndpointAt(0); log.info("Response : ro endpoint = {}", endpoint); assertTrue(getInternalStreamEndpoints().contains(endpoint)); } diff --git a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/StorageAdminClientTest.java b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/StorageAdminClientTest.java index e897564d05c..6d0c361a4f5 100644 --- a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/StorageAdminClientTest.java +++ b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/StorageAdminClientTest.java @@ -67,9 +67,8 @@ private void testNamespaceAPI(boolean enableServerSideRouting) throws Exception private void testNamespaceAPI(StorageAdminClient adminClient) throws Exception { // Create a namespace String nsName = testName.getMethodName(); - NamespaceConfiguration colConf = NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build(); + NamespaceConfiguration colConf = new NamespaceConfiguration(); + colConf.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); NamespaceProperties colProps = FutureUtils.result(adminClient.createNamespace(nsName, colConf)); assertEquals(nsName, colProps.getNamespaceName()); assertEquals(colConf.getDefaultStreamConf(), colProps.getDefaultStreamConf()); @@ -139,17 +138,15 @@ private void testStreamAPI(boolean enableServerSideRouting) throws Exception { private void testStreamAPI(StorageAdminClient adminClient) throws Exception { // Create a namespace String nsName = testName.getMethodName() + "_ns"; - NamespaceConfiguration colConf = NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build(); + NamespaceConfiguration colConf = new NamespaceConfiguration(); + colConf.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); NamespaceProperties colProps = FutureUtils.result(adminClient.createNamespace(nsName, colConf)); assertEquals(nsName, colProps.getNamespaceName()); assertEquals(colConf.getDefaultStreamConf(), colProps.getDefaultStreamConf()); // Create a stream String streamName = testName.getMethodName() + "_stream"; - StreamConfiguration streamConf = StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) - .build(); + StreamConfiguration streamConf = new StreamConfiguration().copyFrom(DEFAULT_STREAM_CONF); StreamProperties streamProps = FutureUtils.result(adminClient.createStream(nsName, streamName, streamConf)); assertEquals(streamName, streamProps.getStreamName()); diff --git a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/TableClientSimpleTest.java b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/TableClientSimpleTest.java index 630c13a5233..e3cca58ade6 100644 --- a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/TableClientSimpleTest.java +++ b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/TableClientSimpleTest.java @@ -93,18 +93,16 @@ private static ByteBuf getValue(int i) { @FlakyTest("https://github.com/apache/bookkeeper/issues/1440") public void testTableSimpleAPI() throws Exception { // Create a namespace - NamespaceConfiguration nsConf = NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build(); + NamespaceConfiguration nsConf = new NamespaceConfiguration(); + nsConf.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); NamespaceProperties nsProps = result(adminClient.createNamespace(namespace, nsConf)); assertEquals(namespace, nsProps.getNamespaceName()); assertEquals(nsConf.getDefaultStreamConf(), nsProps.getDefaultStreamConf()); // Create a stream String streamName = testName.getMethodName() + "_stream"; - StreamConfiguration streamConf = StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) - .setStorageType(StorageType.TABLE) - .build(); + StreamConfiguration streamConf = new StreamConfiguration().copyFrom(DEFAULT_STREAM_CONF) + .setStorageType(StorageType.TABLE); StreamProperties streamProps = result( adminClient.createStream(namespace, streamName, streamConf)); assertEquals(streamName, streamProps.getStreamName()); @@ -247,15 +245,14 @@ public void testTableTtl() throws Exception { final String ns = namespace + "_ttl"; final int ttlSeconds = 5; - result(adminClient.createNamespace(ns, NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build())); + NamespaceConfiguration nsConfTtl = new NamespaceConfiguration(); + nsConfTtl.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); + result(adminClient.createNamespace(ns, nsConfTtl)); final String streamName = testName.getMethodName() + "_stream"; - result(adminClient.createStream(ns, streamName, StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) + result(adminClient.createStream(ns, streamName, new StreamConfiguration().copyFrom(DEFAULT_STREAM_CONF) .setStorageType(StorageType.TABLE) - .setTtlSeconds(ttlSeconds) - .build())); + .setTtlSeconds(ttlSeconds))); final PTable table = result(storageClient.openPTable(streamName)); diff --git a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/TableClientTest.java b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/TableClientTest.java index 81c3c853109..3d0a27c6f0d 100644 --- a/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/TableClientTest.java +++ b/tests/integration/cluster/src/test/java/org/apache/bookkeeper/tests/integration/stream/TableClientTest.java @@ -102,18 +102,16 @@ private void testTableAPI(String namespace, StorageAdminClient adminClient, StorageClient storageClient) throws Exception { // Create a namespace - NamespaceConfiguration nsConf = NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build(); + NamespaceConfiguration nsConf = new NamespaceConfiguration(); + nsConf.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); NamespaceProperties nsProps = FutureUtils.result(adminClient.createNamespace(namespace, nsConf)); assertEquals(namespace, nsProps.getNamespaceName()); assertEquals(nsConf.getDefaultStreamConf(), nsProps.getDefaultStreamConf()); // Create a stream String streamName = testName.getMethodName() + "_stream"; - StreamConfiguration streamConf = StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) - .setStorageType(StorageType.TABLE) - .build(); + StreamConfiguration streamConf = new StreamConfiguration().copyFrom(DEFAULT_STREAM_CONF) + .setStorageType(StorageType.TABLE); StreamProperties streamProps = FutureUtils.result( adminClient.createStream(namespace, streamName, streamConf)); assertEquals(streamName, streamProps.getStreamName()); diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/CreateNamespaceCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/CreateNamespaceCommand.java index 2c572092b25..db54158968f 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/CreateNamespaceCommand.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/namespace/CreateNamespaceCommand.java @@ -65,12 +65,9 @@ protected void run(StorageAdminClient admin, String namespaceName = flags.arguments.get(0); try { - NamespaceProperties nsProps = result( - admin.createNamespace( - namespaceName, - NamespaceConfiguration.newBuilder() - .setDefaultStreamConf(DEFAULT_STREAM_CONF) - .build())); + NamespaceConfiguration nsConf = new NamespaceConfiguration(); + nsConf.setDefaultStreamConf().copyFrom(DEFAULT_STREAM_CONF); + NamespaceProperties nsProps = result(admin.createNamespace(namespaceName, nsConf)); spec.console().println("Successfully created namespace '" + namespaceName + "':"); spec.console().println(nsProps); } catch (NamespaceExistsException nee) { diff --git a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java index 2979b35e561..10fa3d89392 100644 --- a/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java +++ b/tools/stream/src/main/java/org/apache/bookkeeper/stream/cli/commands/table/CreateTableCommand.java @@ -79,17 +79,12 @@ protected void run(StorageAdminClient admin, String streamName = flags.arguments.get(0); try { - StreamConfiguration conf = StreamConfiguration.newBuilder(DEFAULT_STREAM_CONF) + StreamConfiguration conf = new StreamConfiguration().copyFrom(DEFAULT_STREAM_CONF) .setMinNumRanges(flags.numRanges) .setInitialNumRanges(flags.numRanges) - .build(); + .setStorageType(StorageType.TABLE); StreamProperties nsProps = result( - admin.createStream( - globalFlags.namespace, - streamName, - StreamConfiguration.newBuilder(conf) - .setStorageType(StorageType.TABLE) - .build())); + admin.createStream(globalFlags.namespace, streamName, conf)); spec.console().println("Successfully created table '" + streamName + "':"); spec.console().println(nsProps); } catch (NamespaceNotFoundException nfe) {