You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/05/12 18:26:26 UTC

[GitHub] [nifi] exceptionfactory opened a new pull request, #6040: NIFI-9085 Refactor Distributed Cache Servers using Netty

exceptionfactory opened a new pull request, #6040:
URL: https://github.com/apache/nifi/pull/6040

   # Summary
   
   [NIFI-9805](https://issues.apache.org/jira/browse/NIFI-9805) Refactors the `DistributedMapCacheServer` and `DistributedSetCacheServer` implementations using Netty instead of direct socket handling.
   
   The implementation leverages components from `nifi-event-transport` to provide standard server configuration options and support for TLS. This approach removes a dependency on the custom `SSLSocketChannel` class in favor of the standard Netty `SslHandler` to support mututal TLS authentication.
   
   The `StandardMapCacheServer` and `StandardSetCacheServer` classes incorporate current configuration options and provide the Netty Channel Handler pipeline necessary for processing requests and responses. Current unit tests exercise socket communication and support for TLS, so test changes are limited to upgrading from JUnit 4 to JUnit 5.
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [X] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [X] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [X] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [X] Pull Request based on current revision of the `main` branch
   - [X] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [X] Build completed using `mvn clean install -P contrib-check`
     - [X] JDK 8
     - [ ] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6040: NIFI-9085 Refactor Distributed Cache Servers using Netty

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6040:
URL: https://github.com/apache/nifi/pull/6040#discussion_r873943649


##########
nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/EventCacheServer.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.nifi.distributed.cache.server;
+
+import org.apache.nifi.event.transport.EventServer;
+import org.apache.nifi.event.transport.configuration.ShutdownQuietPeriod;
+import org.apache.nifi.event.transport.configuration.ShutdownTimeout;
+import org.apache.nifi.event.transport.configuration.TransportProtocol;
+import org.apache.nifi.event.transport.netty.NettyEventServerFactory;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.security.util.ClientAuth;
+
+import javax.net.ssl.SSLContext;
+import java.net.InetAddress;
+import java.util.Objects;
+
+/**
+ * Abstract Event Cache Server with standard lifecycle methods
+ */
+public abstract class EventCacheServer implements CacheServer {
+    private static final InetAddress ALL_ADDRESSES = null;
+
+    private final ComponentLog log;
+
+    private final int port;
+
+    private EventServer eventServer;
+
+    public EventCacheServer(
+            final ComponentLog log,
+            final int port
+    ) {
+        this.log = Objects.requireNonNull(log, "Component Log required");
+        this.port = port;
+    }
+
+    /**
+     * Start Server
+     *
+     */
+    @Override
+    public void start() {
+        eventServer = createEventServer();
+        log.info("Started Cache Server Port [{}]", port);
+    }
+
+    /**
+     * Stop Server
+     *
+     */
+    @Override
+    public void stop() {
+        if (eventServer == null) {
+            log.info("Server not running");
+        } else {
+            eventServer.shutdown();
+        }
+
+        log.info("Stopped Cache Server Port [{}]", port);
+    }
+
+    /**
+     * Get Server Port Number
+     *
+     * @return Port Number
+     */
+    @Override
+    public int getPort() {
+        return port;
+    }
+
+    /**
+     * Create Event Server Factory with standard properties
+     *
+     * @param identifier Component Identifier
+     * @param sslContext SSL Context is null when not configured
+     * @return Netty Event Server Factory
+     */
+    protected NettyEventServerFactory createEventServerFactory(final String identifier, final SSLContext sslContext) {
+        final NettyEventServerFactory eventServerFactory = new NettyEventServerFactory(ALL_ADDRESSES, port, TransportProtocol.TCP);
+        eventServerFactory.setSslContext(sslContext);
+        eventServerFactory.setClientAuth(ClientAuth.REQUIRED);

Review Comment:
   Yes, the current implementation requires client authentication when enabling TLS. This could be changed to make client authentication configurable using a new property, but it seems best to address that in a separate issue.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on pull request #6040: NIFI-9805 Refactor Distributed Cache Servers using Netty

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on PR #6040:
URL: https://github.com/apache/nifi/pull/6040#issuecomment-1128071237

   @greyp9 I pushed another commit to add checking for readable bytes prior to all buffer reads.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6040: NIFI-9085 Refactor Distributed Cache Servers using Netty

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6040:
URL: https://github.com/apache/nifi/pull/6040#discussion_r873947923


##########
nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/codec/CacheRequestDecoder.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.nifi.distributed.cache.server.codec;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.ByteToMessageDecoder;
+import org.apache.nifi.distributed.cache.operations.CacheOperation;
+import org.apache.nifi.distributed.cache.server.protocol.CacheRequest;
+import org.apache.nifi.distributed.cache.server.protocol.CacheVersionRequest;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Cache Request Decoder processes bytes and decodes cache version and operation requests
+ */
+public class CacheRequestDecoder extends ByteToMessageDecoder {
+    private static final int DEFAULT_LENGTH = 0;
+
+    private static final int HEADER_LENGTH = 4;
+
+    private final AtomicBoolean headerReceived = new AtomicBoolean();
+
+    private final AtomicInteger protocolVersion = new AtomicInteger();
+
+    private final AtomicInteger valueLength = new AtomicInteger(DEFAULT_LENGTH);
+
+    private final AtomicReference<CacheOperation> currentOperation = new AtomicReference<>();
+
+    private final ComponentLog log;
+
+    private final int maxLength;
+
+    private final CacheOperation[] supportedOperations;
+
+    public CacheRequestDecoder(
+            final ComponentLog log,
+            final int maxLength,
+            final CacheOperation[] supportedOperations
+    ) {
+        this.log = log;
+        this.maxLength = maxLength;
+        this.supportedOperations = supportedOperations;
+    }
+
+    /**
+     * Decode Byte Buffer reading header on initial connection followed by protocol version and cache operations
+     *
+     * @param channelHandlerContext Channel Handler Context
+     * @param byteBuf Byte Buffer
+     * @param objects Decoded Objects
+     */
+    @Override
+    protected void decode(final ChannelHandlerContext channelHandlerContext, final ByteBuf byteBuf, final List<Object> objects) {
+        if (!headerReceived.get()) {
+            byteBuf.readBytes(HEADER_LENGTH);
+            headerReceived.getAndSet(true);
+            log.debug("Header Received [{}]", channelHandlerContext.channel().remoteAddress());
+        }
+
+        if (protocolVersion.get() == 0) {
+            final int clientVersion = byteBuf.readInt();
+            log.debug("Protocol Version [{}] Received [{}]", clientVersion, channelHandlerContext.channel().remoteAddress());
+            final CacheVersionRequest cacheVersionRequest = new CacheVersionRequest(clientVersion);
+            objects.add(cacheVersionRequest);
+        } else {
+            final CacheOperation cacheOperation = readOperation(byteBuf);
+            final Object cacheRequest = readRequest(cacheOperation, byteBuf);
+            if (cacheRequest == null) {
+                log.debug("Cache Operation [{}] request not processed", cacheOperation);
+            } else {
+                objects.add(cacheRequest);
+                // Reset Cache Operation after successful decoding
+                currentOperation.set(null);
+            }
+        }
+    }
+
+    @Override
+    public void exceptionCaught(final ChannelHandlerContext context, final Throwable cause) {
+        log.warn("Request Decoding Failed: Closing Connection [{}]", context.channel().remoteAddress(), cause);
+        context.close();
+    }
+
+    /**
+     * Set Protocol Version based on version negotiated in other handlers
+     *
+     * @param protocolVersion Protocol Version
+     */
+    public void setProtocolVersion(final int protocolVersion) {
+        this.protocolVersion.getAndSet(protocolVersion);
+    }
+
+    /**
+     * Read Request Object based on Cache Operation
+     *
+     * @param cacheOperation Cache Operation
+     * @param byteBuf Byte Buffer
+     * @return Request Object or null when buffer does not contain sufficient bytes
+     */
+    protected Object readRequest(final CacheOperation cacheOperation, final ByteBuf byteBuf) {
+        final int readableBytes = byteBuf.readableBytes();
+        final byte[] body = readableBytes == 0 ? new byte[0] : readBytes(byteBuf);
+        return body == null ? null : new CacheRequest(cacheOperation, body);
+    }
+
+    /**
+     * Read Bytes from buffer based on length indicated
+     *
+     * @param byteBuf Byte Buffer
+     * @return Bytes read or null when buffer does not contain sufficient bytes
+     */
+    protected byte[] readBytes(final ByteBuf byteBuf) {
+        final int length = readBytesLength(byteBuf);

Review Comment:
   Yes, the Netty `BytesToMessageDecoder` class, which this class extends, buffers incoming data in the provided `ByteBuf`. Netty will continue calling this decode with the current contents of the buffer. As implemented, this method checks for a sufficient number of bytes available before attempting to read the buffer contents. With this approach, Netty handles the buffering so this class can support both small and large requests.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] greyp9 closed pull request #6040: NIFI-9805 Refactor Distributed Cache Servers using Netty

Posted by GitBox <gi...@apache.org>.
greyp9 closed pull request #6040: NIFI-9805 Refactor Distributed Cache Servers using Netty
URL: https://github.com/apache/nifi/pull/6040


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] greyp9 commented on a diff in pull request #6040: NIFI-9085 Refactor Distributed Cache Servers using Netty

Posted by GitBox <gi...@apache.org>.
greyp9 commented on code in PR #6040:
URL: https://github.com/apache/nifi/pull/6040#discussion_r872843969


##########
nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/EventCacheServer.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.nifi.distributed.cache.server;
+
+import org.apache.nifi.event.transport.EventServer;
+import org.apache.nifi.event.transport.configuration.ShutdownQuietPeriod;
+import org.apache.nifi.event.transport.configuration.ShutdownTimeout;
+import org.apache.nifi.event.transport.configuration.TransportProtocol;
+import org.apache.nifi.event.transport.netty.NettyEventServerFactory;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.security.util.ClientAuth;
+
+import javax.net.ssl.SSLContext;
+import java.net.InetAddress;
+import java.util.Objects;
+
+/**
+ * Abstract Event Cache Server with standard lifecycle methods
+ */
+public abstract class EventCacheServer implements CacheServer {
+    private static final InetAddress ALL_ADDRESSES = null;
+
+    private final ComponentLog log;
+
+    private final int port;
+
+    private EventServer eventServer;
+
+    public EventCacheServer(
+            final ComponentLog log,
+            final int port
+    ) {
+        this.log = Objects.requireNonNull(log, "Component Log required");
+        this.port = port;
+    }
+
+    /**
+     * Start Server
+     *
+     */
+    @Override
+    public void start() {
+        eventServer = createEventServer();
+        log.info("Started Cache Server Port [{}]", port);
+    }
+
+    /**
+     * Stop Server
+     *
+     */
+    @Override
+    public void stop() {
+        if (eventServer == null) {
+            log.info("Server not running");
+        } else {
+            eventServer.shutdown();
+        }
+
+        log.info("Stopped Cache Server Port [{}]", port);
+    }
+
+    /**
+     * Get Server Port Number
+     *
+     * @return Port Number
+     */
+    @Override
+    public int getPort() {
+        return port;
+    }
+
+    /**
+     * Create Event Server Factory with standard properties
+     *
+     * @param identifier Component Identifier
+     * @param sslContext SSL Context is null when not configured
+     * @return Netty Event Server Factory
+     */
+    protected NettyEventServerFactory createEventServerFactory(final String identifier, final SSLContext sslContext) {
+        final NettyEventServerFactory eventServerFactory = new NettyEventServerFactory(ALL_ADDRESSES, port, TransportProtocol.TCP);
+        eventServerFactory.setSslContext(sslContext);
+        eventServerFactory.setClientAuth(ClientAuth.REQUIRED);

Review Comment:
   REQUIRED?



##########
nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/protocol/MapCacheRequest.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.nifi.distributed.cache.server.protocol;
+
+import org.apache.nifi.distributed.cache.operations.CacheOperation;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Map Cache Request with operation and other optional properties
+ */
+public class MapCacheRequest {
+    private final CacheOperation cacheOperation;
+
+    private byte[] key;
+
+    private byte[] value;
+
+    private String pattern;
+
+    private long revision;
+
+    private List<byte[]> keys = Collections.emptyList();
+
+    public MapCacheRequest(

Review Comment:
   What do you think about chaining these constructors where possible? (cut down on duplicate code)



##########
nifi-commons/nifi-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java:
##########
@@ -1,188 +0,0 @@
-/*
- * 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.nifi.remote.io.socket;
-
-import org.apache.nifi.remote.exception.TransmissionDisabledException;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.SocketTimeoutException;
-import java.nio.ByteBuffer;
-import java.nio.channels.SelectionKey;
-import java.nio.channels.Selector;
-import java.nio.channels.SocketChannel;
-import java.util.Set;
-
-public class SocketChannelInputStream extends InputStream {

Review Comment:
   Finally!



##########
nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/protocol/MapValueResponse.java:
##########
@@ -14,39 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.remote.io.socket.ssl;
+package org.apache.nifi.distributed.cache.server.protocol;
 
-import java.io.IOException;
-import java.io.OutputStream;
+/**
+ * Map Value Response
+ */
+public class MapValueResponse {
+    private final int length;
 
-public class SSLSocketChannelOutputStream extends OutputStream {
+    private final byte[] value;
 
-    private final SSLSocketChannel channel;
+    private Long revision;
 
-    public SSLSocketChannelOutputStream(final SSLSocketChannel channel) {
-        this.channel = channel;
+    public MapValueResponse(
+            final int length,
+            final byte[] value
+    ) {
+        this.length = length;
+        this.value = value;
     }
 
-    @Override
-    public void write(final int b) throws IOException {
-        channel.write(b);
+    public int getLength() {
+        return length;
     }
 
-    @Override
-    public void write(byte[] b) throws IOException {
-        channel.write(b);
+    public byte[] getValue() {
+        return value;
     }
 
-    @Override
-    public void write(byte[] b, int off, int len) throws IOException {
-        channel.write(b, off, len);
+    public void setRevision(final Long revision) {

Review Comment:
   Looks like only one usage.  Maybe this could be included in the constructor?



##########
nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/codec/CacheRequestDecoder.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.nifi.distributed.cache.server.codec;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.ByteToMessageDecoder;
+import org.apache.nifi.distributed.cache.operations.CacheOperation;
+import org.apache.nifi.distributed.cache.server.protocol.CacheRequest;
+import org.apache.nifi.distributed.cache.server.protocol.CacheVersionRequest;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Cache Request Decoder processes bytes and decodes cache version and operation requests
+ */
+public class CacheRequestDecoder extends ByteToMessageDecoder {
+    private static final int DEFAULT_LENGTH = 0;
+
+    private static final int HEADER_LENGTH = 4;
+
+    private final AtomicBoolean headerReceived = new AtomicBoolean();
+
+    private final AtomicInteger protocolVersion = new AtomicInteger();
+
+    private final AtomicInteger valueLength = new AtomicInteger(DEFAULT_LENGTH);
+
+    private final AtomicReference<CacheOperation> currentOperation = new AtomicReference<>();
+
+    private final ComponentLog log;
+
+    private final int maxLength;
+
+    private final CacheOperation[] supportedOperations;
+
+    public CacheRequestDecoder(
+            final ComponentLog log,
+            final int maxLength,
+            final CacheOperation[] supportedOperations
+    ) {
+        this.log = log;
+        this.maxLength = maxLength;
+        this.supportedOperations = supportedOperations;
+    }
+
+    /**
+     * Decode Byte Buffer reading header on initial connection followed by protocol version and cache operations
+     *
+     * @param channelHandlerContext Channel Handler Context
+     * @param byteBuf Byte Buffer
+     * @param objects Decoded Objects
+     */
+    @Override
+    protected void decode(final ChannelHandlerContext channelHandlerContext, final ByteBuf byteBuf, final List<Object> objects) {
+        if (!headerReceived.get()) {
+            byteBuf.readBytes(HEADER_LENGTH);
+            headerReceived.getAndSet(true);
+            log.debug("Header Received [{}]", channelHandlerContext.channel().remoteAddress());
+        }
+
+        if (protocolVersion.get() == 0) {
+            final int clientVersion = byteBuf.readInt();
+            log.debug("Protocol Version [{}] Received [{}]", clientVersion, channelHandlerContext.channel().remoteAddress());
+            final CacheVersionRequest cacheVersionRequest = new CacheVersionRequest(clientVersion);
+            objects.add(cacheVersionRequest);
+        } else {
+            final CacheOperation cacheOperation = readOperation(byteBuf);
+            final Object cacheRequest = readRequest(cacheOperation, byteBuf);
+            if (cacheRequest == null) {
+                log.debug("Cache Operation [{}] request not processed", cacheOperation);
+            } else {
+                objects.add(cacheRequest);
+                // Reset Cache Operation after successful decoding
+                currentOperation.set(null);
+            }
+        }
+    }
+
+    @Override
+    public void exceptionCaught(final ChannelHandlerContext context, final Throwable cause) {
+        log.warn("Request Decoding Failed: Closing Connection [{}]", context.channel().remoteAddress(), cause);
+        context.close();
+    }
+
+    /**
+     * Set Protocol Version based on version negotiated in other handlers
+     *
+     * @param protocolVersion Protocol Version
+     */
+    public void setProtocolVersion(final int protocolVersion) {
+        this.protocolVersion.getAndSet(protocolVersion);
+    }
+
+    /**
+     * Read Request Object based on Cache Operation
+     *
+     * @param cacheOperation Cache Operation
+     * @param byteBuf Byte Buffer
+     * @return Request Object or null when buffer does not contain sufficient bytes
+     */
+    protected Object readRequest(final CacheOperation cacheOperation, final ByteBuf byteBuf) {
+        final int readableBytes = byteBuf.readableBytes();
+        final byte[] body = readableBytes == 0 ? new byte[0] : readBytes(byteBuf);
+        return body == null ? null : new CacheRequest(cacheOperation, body);
+    }
+
+    /**
+     * Read Bytes from buffer based on length indicated
+     *
+     * @param byteBuf Byte Buffer
+     * @return Bytes read or null when buffer does not contain sufficient bytes
+     */
+    protected byte[] readBytes(final ByteBuf byteBuf) {
+        final int length = readBytesLength(byteBuf);

Review Comment:
   Is there some sort of buffering implemented, in case the entire request payload is not received in a single netty ByteBuf?  We should be able to handle some small amount of latency, in the case of a heavily loaded network.



##########
nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/StandardSetCacheServer.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * 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.nifi.distributed.cache.server.set;
+
+import org.apache.nifi.distributed.cache.operations.SetOperation;
+import org.apache.nifi.distributed.cache.protocol.ProtocolVersion;
+import org.apache.nifi.distributed.cache.server.EventCacheServer;
+import org.apache.nifi.distributed.cache.server.EvictionPolicy;
+import org.apache.nifi.distributed.cache.server.codec.CacheOperationResultEncoder;
+import org.apache.nifi.distributed.cache.server.codec.CacheRequestDecoder;
+import org.apache.nifi.distributed.cache.server.codec.CacheVersionRequestHandler;
+import org.apache.nifi.distributed.cache.server.codec.CacheVersionResponseEncoder;
+import org.apache.nifi.distributed.cache.server.codec.SetCacheRequestHandler;
+import org.apache.nifi.event.transport.EventServer;
+import org.apache.nifi.event.transport.EventServerFactory;
+import org.apache.nifi.event.transport.netty.NettyEventServerFactory;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.remote.StandardVersionNegotiator;
+import org.apache.nifi.remote.VersionNegotiator;
+
+import javax.net.ssl.SSLContext;
+import java.io.File;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Arrays;
+
+/**
+ * Standard Set Cache Server implementation based on Netty
+ */
+public class StandardSetCacheServer extends EventCacheServer {
+    private final EventServerFactory eventServerFactory;
+
+    private final SetCache cache;
+
+    public StandardSetCacheServer(
+            final ComponentLog log,
+            final String identifier,
+            final SSLContext sslContext,
+            final int port,
+            final int maxCacheEntries,
+            final EvictionPolicy evictionPolicy,
+            final File persistencePath,
+            final int maxReadLength
+    ) throws IOException {
+        super(log, port);
+
+        final SetCache simpleCache = new SimpleSetCache(identifier, maxCacheEntries, evictionPolicy);
+
+        if (persistencePath == null) {
+            this.cache = simpleCache;
+        } else {
+            final PersistentSetCache persistentCache = new PersistentSetCache(identifier, persistencePath, simpleCache);
+            persistentCache.restore();
+            this.cache = persistentCache;
+        }
+
+        final NettyEventServerFactory nettyEventServerFactory = createEventServerFactory(identifier, sslContext);
+
+        // Create Sharable Handlers to avoid unnecessary instantiation for each connection

Review Comment:
   Nice!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6040: NIFI-9085 Refactor Distributed Cache Servers using Netty

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6040:
URL: https://github.com/apache/nifi/pull/6040#discussion_r873945304


##########
nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/protocol/MapCacheRequest.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.nifi.distributed.cache.server.protocol;
+
+import org.apache.nifi.distributed.cache.operations.CacheOperation;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Map Cache Request with operation and other optional properties
+ */
+public class MapCacheRequest {
+    private final CacheOperation cacheOperation;
+
+    private byte[] key;
+
+    private byte[] value;
+
+    private String pattern;
+
+    private long revision;
+
+    private List<byte[]> keys = Collections.emptyList();
+
+    public MapCacheRequest(

Review Comment:
   That's a good point. Different constructors require different parameters, but there is some overlap that could be reduced, so will make some adjustments.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] greyp9 commented on a diff in pull request #6040: NIFI-9805 Refactor Distributed Cache Servers using Netty

Posted by GitBox <gi...@apache.org>.
greyp9 commented on code in PR #6040:
URL: https://github.com/apache/nifi/pull/6040#discussion_r876084246


##########
nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/codec/CacheRequestDecoder.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.nifi.distributed.cache.server.codec;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.ByteToMessageDecoder;
+import org.apache.nifi.distributed.cache.operations.CacheOperation;
+import org.apache.nifi.distributed.cache.operations.StandardCacheOperation;
+import org.apache.nifi.distributed.cache.server.protocol.CacheRequest;
+import org.apache.nifi.distributed.cache.server.protocol.CacheVersionRequest;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.net.SocketAddress;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.OptionalLong;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Cache Request Decoder processes bytes and decodes cache version and operation requests
+ */
+public class CacheRequestDecoder extends ByteToMessageDecoder {
+    private static final int HEADER_LENGTH = 4;
+
+    private static final int LONG_LENGTH = 8;
+
+    private static final int INT_LENGTH = 4;
+
+    private static final int SHORT_LENGTH = 2;
+
+    private final AtomicBoolean headerReceived = new AtomicBoolean();
+
+    private final AtomicInteger protocolVersion = new AtomicInteger();
+
+    private final ComponentLog log;
+
+    private final int maxLength;
+
+    private final CacheOperation[] supportedOperations;
+
+    public CacheRequestDecoder(
+            final ComponentLog log,
+            final int maxLength,
+            final CacheOperation[] supportedOperations
+    ) {
+        this.log = log;
+        this.maxLength = maxLength;
+        this.supportedOperations = supportedOperations;
+    }
+
+    /**
+     * Decode Byte Buffer reading header on initial connection followed by protocol version and cache operations
+     *
+     * @param channelHandlerContext Channel Handler Context
+     * @param byteBuf Byte Buffer
+     * @param objects Decoded Objects
+     */
+    @Override
+    protected void decode(final ChannelHandlerContext channelHandlerContext, final ByteBuf byteBuf, final List<Object> objects) {
+        if (!headerReceived.get()) {
+            readHeader(byteBuf, channelHandlerContext.channel().remoteAddress());
+        }
+
+        if (protocolVersion.get() == 0) {
+            final OptionalInt clientVersion = readInt(byteBuf);
+            if (clientVersion.isPresent()) {
+                final int clientVersionFound = clientVersion.getAsInt();
+                log.debug("Protocol Version [{}] Received [{}]", clientVersionFound, channelHandlerContext.channel().remoteAddress());
+                final CacheVersionRequest cacheVersionRequest = new CacheVersionRequest(clientVersionFound);
+                objects.add(cacheVersionRequest);
+            }
+        } else {
+            // Mark ByteBuf reader index to reset when sufficient bytes are not found
+            byteBuf.markReaderIndex();

Review Comment:
   +1



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6040: NIFI-9805 Refactor Distributed Cache Servers using Netty

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6040:
URL: https://github.com/apache/nifi/pull/6040#discussion_r875221156


##########
nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/codec/MapCacheRequestDecoder.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.nifi.distributed.cache.server.codec;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.nifi.distributed.cache.operations.CacheOperation;
+import org.apache.nifi.distributed.cache.operations.MapOperation;
+import org.apache.nifi.distributed.cache.server.protocol.MapCacheRequest;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.OptionalLong;
+
+/**
+ * Cache Request Decoder processes bytes and decodes cache version and operation requests
+ */
+public class MapCacheRequestDecoder extends CacheRequestDecoder {
+
+    public MapCacheRequestDecoder(
+            final ComponentLog log,
+            final int maxLength,
+            final CacheOperation[] supportedOperations
+    ) {
+        super(log, maxLength, supportedOperations);
+    }
+
+    @Override
+    protected Object readRequest(final CacheOperation cacheOperation, final ByteBuf byteBuf) {
+        final Object request;
+
+        if (MapOperation.CONTAINS_KEY == cacheOperation) {
+            request = readKeyRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.FETCH == cacheOperation) {
+            request = readKeyRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.GET == cacheOperation) {
+            request = readKeyRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.GET_AND_PUT_IF_ABSENT == cacheOperation) {
+            request = readKeyValueRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.KEYSET == cacheOperation) {
+            request = new MapCacheRequest(cacheOperation);
+        } else if (MapOperation.REMOVE == cacheOperation) {
+            request = readKeyRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.REMOVE_BY_PATTERN == cacheOperation) {
+            request = readPatternRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.REMOVE_BY_PATTERN_AND_GET == cacheOperation) {
+            request = readPatternRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.REMOVE_AND_GET == cacheOperation) {
+            request = readKeyRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.REPLACE == cacheOperation) {
+            request = readKeyRevisionValueRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.SUBMAP == cacheOperation) {
+            request = readSubMapRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.PUT == cacheOperation) {
+            request = readKeyValueRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.PUT_IF_ABSENT == cacheOperation) {
+            request = readKeyValueRequest(cacheOperation, byteBuf);
+        } else {
+            request = new MapCacheRequest(cacheOperation);
+        }
+
+        return request;
+    }
+
+    private MapCacheRequest readKeyRequest(final CacheOperation cacheOperation, final ByteBuf byteBuf) {
+        final Optional<byte[]> key = readBytes(byteBuf);
+        return key.map(bytes -> new MapCacheRequest(cacheOperation, bytes)).orElse(null);
+    }
+
+    private MapCacheRequest readKeyValueRequest(final CacheOperation cacheOperation, final ByteBuf byteBuf) {
+        final MapCacheRequest mapCacheRequest;
+
+        final Optional<byte[]> key = readBytes(byteBuf);
+        if (key.isPresent()) {

Review Comment:
   Thanks for catching that issue @greyp9! I pushed an update with a new test method to exercise this condition, and also adjusted `CacheRequestDecoder` to use `ByteBuf.markReaderIndex()` and `ByteBuf.resetReaderIndex()` to avoid partial reads.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on pull request #6040: NIFI-9085 Refactor Distributed Cache Servers using Netty

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on PR #6040:
URL: https://github.com/apache/nifi/pull/6040#issuecomment-1127912717

   Thanks for the feedback @greyp9! I pushed an update reducing the duplication in several classes based on your comments.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] greyp9 commented on a diff in pull request #6040: NIFI-9805 Refactor Distributed Cache Servers using Netty

Posted by GitBox <gi...@apache.org>.
greyp9 commented on code in PR #6040:
URL: https://github.com/apache/nifi/pull/6040#discussion_r875154073


##########
nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/codec/MapCacheRequestDecoder.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.nifi.distributed.cache.server.codec;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.nifi.distributed.cache.operations.CacheOperation;
+import org.apache.nifi.distributed.cache.operations.MapOperation;
+import org.apache.nifi.distributed.cache.server.protocol.MapCacheRequest;
+import org.apache.nifi.logging.ComponentLog;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.OptionalInt;
+import java.util.OptionalLong;
+
+/**
+ * Cache Request Decoder processes bytes and decodes cache version and operation requests
+ */
+public class MapCacheRequestDecoder extends CacheRequestDecoder {
+
+    public MapCacheRequestDecoder(
+            final ComponentLog log,
+            final int maxLength,
+            final CacheOperation[] supportedOperations
+    ) {
+        super(log, maxLength, supportedOperations);
+    }
+
+    @Override
+    protected Object readRequest(final CacheOperation cacheOperation, final ByteBuf byteBuf) {
+        final Object request;
+
+        if (MapOperation.CONTAINS_KEY == cacheOperation) {
+            request = readKeyRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.FETCH == cacheOperation) {
+            request = readKeyRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.GET == cacheOperation) {
+            request = readKeyRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.GET_AND_PUT_IF_ABSENT == cacheOperation) {
+            request = readKeyValueRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.KEYSET == cacheOperation) {
+            request = new MapCacheRequest(cacheOperation);
+        } else if (MapOperation.REMOVE == cacheOperation) {
+            request = readKeyRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.REMOVE_BY_PATTERN == cacheOperation) {
+            request = readPatternRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.REMOVE_BY_PATTERN_AND_GET == cacheOperation) {
+            request = readPatternRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.REMOVE_AND_GET == cacheOperation) {
+            request = readKeyRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.REPLACE == cacheOperation) {
+            request = readKeyRevisionValueRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.SUBMAP == cacheOperation) {
+            request = readSubMapRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.PUT == cacheOperation) {
+            request = readKeyValueRequest(cacheOperation, byteBuf);
+        } else if (MapOperation.PUT_IF_ABSENT == cacheOperation) {
+            request = readKeyValueRequest(cacheOperation, byteBuf);
+        } else {
+            request = new MapCacheRequest(cacheOperation);
+        }
+
+        return request;
+    }
+
+    private MapCacheRequest readKeyRequest(final CacheOperation cacheOperation, final ByteBuf byteBuf) {
+        final Optional<byte[]> key = readBytes(byteBuf);
+        return key.map(bytes -> new MapCacheRequest(cacheOperation, bytes)).orElse(null);
+    }
+
+    private MapCacheRequest readKeyValueRequest(final CacheOperation cacheOperation, final ByteBuf byteBuf) {
+        final MapCacheRequest mapCacheRequest;
+
+        final Optional<byte[]> key = readBytes(byteBuf);
+        if (key.isPresent()) {

Review Comment:
   It looks like, for large values (where the first payload read does not fetch the entire payload), there is the potential that the key data would be lost. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6040: NIFI-9085 Refactor Distributed Cache Servers using Netty

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6040:
URL: https://github.com/apache/nifi/pull/6040#discussion_r873944162


##########
nifi-nar-bundles/nifi-standard-services/nifi-distributed-cache-services-bundle/nifi-distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/protocol/MapValueResponse.java:
##########
@@ -14,39 +14,39 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.remote.io.socket.ssl;
+package org.apache.nifi.distributed.cache.server.protocol;
 
-import java.io.IOException;
-import java.io.OutputStream;
+/**
+ * Map Value Response
+ */
+public class MapValueResponse {
+    private final int length;
 
-public class SSLSocketChannelOutputStream extends OutputStream {
+    private final byte[] value;
 
-    private final SSLSocketChannel channel;
+    private Long revision;
 
-    public SSLSocketChannelOutputStream(final SSLSocketChannel channel) {
-        this.channel = channel;
+    public MapValueResponse(
+            final int length,
+            final byte[] value
+    ) {
+        this.length = length;
+        this.value = value;
     }
 
-    @Override
-    public void write(final int b) throws IOException {
-        channel.write(b);
+    public int getLength() {
+        return length;
     }
 
-    @Override
-    public void write(byte[] b) throws IOException {
-        channel.write(b);
+    public byte[] getValue() {
+        return value;
     }
 
-    @Override
-    public void write(byte[] b, int off, int len) throws IOException {
-        channel.write(b, off, len);
+    public void setRevision(final Long revision) {

Review Comment:
   Thanks, that's a good point, this should be removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org