jsancio commented on a change in pull request #9418: URL: https://github.com/apache/kafka/pull/9418#discussion_r511112281
########## File path: raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java ########## @@ -0,0 +1,294 @@ +/* + * 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.kafka.raft.internals; + +import org.apache.kafka.common.memory.MemoryPool; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; +import org.apache.kafka.raft.RecordSerde; + +import java.io.Closeable; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.locks.ReentrantLock; + +/** + * TODO: Also flush after minimum size limit is reached? + */ +public class BatchAccumulator<T> implements Closeable { + private final int epoch; + private final Time time; + private final Timer lingerTimer; + private final int lingerMs; + private final int maxBatchSize; + private final CompressionType compressionType; + private final MemoryPool memoryPool; + private final ReentrantLock lock; + private final RecordSerde<T> serde; + + private long nextOffset; + private BatchBuilder<T> currentBatch; + private List<CompletedBatch<T>> completed; + + public BatchAccumulator( + int epoch, + long baseOffset, + int lingerMs, + int maxBatchSize, + MemoryPool memoryPool, + Time time, + CompressionType compressionType, + RecordSerde<T> serde + ) { + this.epoch = epoch; + this.lingerMs = lingerMs; + this.maxBatchSize = maxBatchSize; + this.memoryPool = memoryPool; + this.time = time; + this.lingerTimer = time.timer(lingerMs); + this.compressionType = compressionType; + this.serde = serde; + this.nextOffset = baseOffset; + this.completed = new ArrayList<>(); + this.lock = new ReentrantLock(); + } + + /** + * Append a list of records into an atomic batch. We guarantee all records + * are included in the same underlying record batch so that either all of + * the records become committed or none of them do. + * + * @param epoch the expected leader epoch + * @param records the list of records to include in a batch + * @return the offset of the last message or {@link Long#MAX_VALUE} if the epoch + * does not match + */ + public Long append(int epoch, List<T> records) { + if (epoch != this.epoch) { + // If the epoch does not match, then the state machine probably + // has not gotten the notification about the latest epoch change. + // In this case, ignore the append and return a large offset value + // which will never be committed. + return Long.MAX_VALUE; + } + + Object serdeContext = serde.newWriteContext(); + int batchSize = 0; + for (T record : records) { + batchSize += serde.recordSize(record, serdeContext); + } + + if (batchSize > maxBatchSize) { + throw new IllegalArgumentException("The total size of " + records + " is " + batchSize + + ", which exceeds the maximum allowed batch size of " + maxBatchSize); + } + + lock.lock(); + try { + BatchBuilder<T> batch = maybeAllocateBatch(batchSize); + if (batch == null) { + return null; + } + + if (isEmpty()) { + lingerTimer.update(); + lingerTimer.reset(lingerMs); + } + + for (T record : records) { + batch.appendRecord(record, serdeContext); + nextOffset += 1; + } + + return nextOffset - 1; + } finally { + lock.unlock(); + } + } + + private BatchBuilder<T> maybeAllocateBatch(int batchSize) { + if (currentBatch == null) { + startNewBatch(); + } else if (!currentBatch.hasRoomFor(batchSize)) { + completeCurrentBatch(); + } + return currentBatch; + } + + private void completeCurrentBatch() { + MemoryRecords data = currentBatch.build(); + completed.add(new CompletedBatch<>( + currentBatch.baseOffset(), + currentBatch.records(), + data, + memoryPool, + currentBatch.initialBuffer() + )); + currentBatch = null; + startNewBatch(); + } + + private void startNewBatch() { + ByteBuffer buffer = memoryPool.tryAllocate(maxBatchSize); + if (buffer != null) { + currentBatch = new BatchBuilder<>( + buffer, + serde, + compressionType, + nextOffset, + time.milliseconds(), + false, + RecordBatch.NO_PARTITION_LEADER_EPOCH, + maxBatchSize + ); + } + } + + /** + * Check whether there are any batches which need flushing now. Review comment: Throughout this type we use flush to mean "collect the current batch" and start a new one. When I started reading this code I was assuming that it meant flush to disk/IO. Should we use a different verb for this? I think Java tends to use "collect" for this type of operation. What do you think? ########## File path: raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java ########## @@ -0,0 +1,294 @@ +/* + * 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.kafka.raft.internals; + +import org.apache.kafka.common.memory.MemoryPool; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; +import org.apache.kafka.raft.RecordSerde; + +import java.io.Closeable; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.locks.ReentrantLock; + +/** + * TODO: Also flush after minimum size limit is reached? + */ +public class BatchAccumulator<T> implements Closeable { + private final int epoch; + private final Time time; + private final Timer lingerTimer; + private final int lingerMs; + private final int maxBatchSize; + private final CompressionType compressionType; + private final MemoryPool memoryPool; + private final ReentrantLock lock; + private final RecordSerde<T> serde; + + private long nextOffset; + private BatchBuilder<T> currentBatch; + private List<CompletedBatch<T>> completed; + + public BatchAccumulator( + int epoch, + long baseOffset, + int lingerMs, + int maxBatchSize, + MemoryPool memoryPool, + Time time, + CompressionType compressionType, + RecordSerde<T> serde + ) { + this.epoch = epoch; + this.lingerMs = lingerMs; + this.maxBatchSize = maxBatchSize; + this.memoryPool = memoryPool; + this.time = time; + this.lingerTimer = time.timer(lingerMs); + this.compressionType = compressionType; + this.serde = serde; + this.nextOffset = baseOffset; + this.completed = new ArrayList<>(); + this.lock = new ReentrantLock(); + } + + /** + * Append a list of records into an atomic batch. We guarantee all records + * are included in the same underlying record batch so that either all of + * the records become committed or none of them do. + * + * @param epoch the expected leader epoch + * @param records the list of records to include in a batch + * @return the offset of the last message or {@link Long#MAX_VALUE} if the epoch + * does not match + */ + public Long append(int epoch, List<T> records) { + if (epoch != this.epoch) { + // If the epoch does not match, then the state machine probably + // has not gotten the notification about the latest epoch change. + // In this case, ignore the append and return a large offset value + // which will never be committed. Review comment: What are you trying to protect with this check? For example, the signature could be `public long append(List<T> records)` with the accumulator writing the correct epoch. In https://github.com/apache/kafka/pull/9482 you implemented `handleClaim` in to only fire when the `Listener`'s "acknowledged" offset + 1 is >= to the leader's epoch start offset. Thinking through the code's behaviour, I see this check catching the case the the raft replica lost leadership and won leadership before the `Listener` was able to asynchronously process `handleResign` and `handleClaim`. ########## File path: core/src/main/scala/kafka/raft/KafkaNetworkChannel.scala ########## @@ -216,11 +216,9 @@ class KafkaNetworkChannel(time: Time, endpoints.put(id, node) } - def postInboundRequest(header: RequestHeader, - request: AbstractRequest, - onResponseReceived: ResponseHandler): Unit = { + def postInboundRequest(request: AbstractRequest, onResponseReceived: ResponseHandler): Unit = { Review comment: Currently, `postInboundRequest` in only used by `TestRaftRequestHandler` and `KafkaNetworkChannelTest`. Are you thinking that we will use the same or similar method when integrating with the broker code? ########## File path: raft/src/main/java/org/apache/kafka/raft/internals/BatchMemoryPool.java ########## @@ -0,0 +1,105 @@ +/* + * 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.kafka.raft.internals; + +import org.apache.kafka.common.memory.MemoryPool; + +import java.nio.ByteBuffer; +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Simple memory pool which maintains a limited number of fixed-size buffers. + */ +public class BatchMemoryPool implements MemoryPool { + private final ReentrantLock lock; + private final Deque<ByteBuffer> free; + private final int maxBatches; + private final int batchSize; + + private int numAllocatedBatches = 0; + + public BatchMemoryPool(int maxBatches, int batchSize) { + this.maxBatches = maxBatches; + this.batchSize = batchSize; + this.free = new ArrayDeque<>(maxBatches); + this.lock = new ReentrantLock(); + } + + @Override + public ByteBuffer tryAllocate(int sizeBytes) { + if (sizeBytes > batchSize) { + throw new IllegalArgumentException("Cannot allocate buffers larger than max " + + "batch size of " + batchSize); + } + + lock.lock(); + try { + ByteBuffer buffer = free.poll(); + if (buffer == null && numAllocatedBatches < maxBatches) { + buffer = ByteBuffer.allocate(batchSize); + numAllocatedBatches += 1; + } + + if (buffer != null) { + buffer.clear(); + buffer.limit(sizeBytes); + } + return buffer; + } finally { + lock.unlock(); + } + } + + @Override + public void release(ByteBuffer previouslyAllocated) { Review comment: Let's check that `previouslyAllocated`'s capacity is `batchSize` else `buffer.limit(sizeBytes)` is going to throw with a less useful stacktrace. ########## File path: raft/src/main/java/org/apache/kafka/raft/internals/BatchBuilder.java ########## @@ -0,0 +1,231 @@ +/* + * 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.kafka.raft.internals; + +import org.apache.kafka.common.protocol.DataOutputStreamWritable; +import org.apache.kafka.common.record.AbstractRecords; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.DefaultRecord; +import org.apache.kafka.common.record.DefaultRecordBatch; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.common.utils.ByteBufferOutputStream; +import org.apache.kafka.common.utils.ByteUtils; +import org.apache.kafka.raft.RecordSerde; + +import java.io.DataOutputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; + +public class BatchBuilder<T> { Review comment: I think it would be good to have a description of what this type is doing and how to use it. Reading the code and seeing how the buffer is shared, I am under the impression that this type should not be reused after the `build` method is called. Is that correct? If this is correct, I think all of the public methods should check that they are not being called after `build` is called. ########## File path: raft/src/main/java/org/apache/kafka/raft/internals/BatchAccumulator.java ########## @@ -0,0 +1,294 @@ +/* + * 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.kafka.raft.internals; + +import org.apache.kafka.common.memory.MemoryPool; +import org.apache.kafka.common.record.CompressionType; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.RecordBatch; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; +import org.apache.kafka.raft.RecordSerde; + +import java.io.Closeable; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.locks.ReentrantLock; + +/** + * TODO: Also flush after minimum size limit is reached? + */ +public class BatchAccumulator<T> implements Closeable { + private final int epoch; + private final Time time; + private final Timer lingerTimer; + private final int lingerMs; + private final int maxBatchSize; + private final CompressionType compressionType; + private final MemoryPool memoryPool; + private final ReentrantLock lock; + private final RecordSerde<T> serde; + + private long nextOffset; + private BatchBuilder<T> currentBatch; + private List<CompletedBatch<T>> completed; + + public BatchAccumulator( + int epoch, + long baseOffset, + int lingerMs, + int maxBatchSize, + MemoryPool memoryPool, + Time time, + CompressionType compressionType, + RecordSerde<T> serde + ) { + this.epoch = epoch; + this.lingerMs = lingerMs; + this.maxBatchSize = maxBatchSize; + this.memoryPool = memoryPool; + this.time = time; + this.lingerTimer = time.timer(lingerMs); + this.compressionType = compressionType; + this.serde = serde; + this.nextOffset = baseOffset; + this.completed = new ArrayList<>(); + this.lock = new ReentrantLock(); + } + + /** + * Append a list of records into an atomic batch. We guarantee all records + * are included in the same underlying record batch so that either all of + * the records become committed or none of them do. + * + * @param epoch the expected leader epoch + * @param records the list of records to include in a batch + * @return the offset of the last message or {@link Long#MAX_VALUE} if the epoch + * does not match Review comment: We should document this returns `null` and what it means when `null` is returned. ########## File path: core/src/main/scala/kafka/raft/KafkaNetworkChannel.scala ########## @@ -216,11 +216,9 @@ class KafkaNetworkChannel(time: Time, endpoints.put(id, node) } - def postInboundRequest(header: RequestHeader, - request: AbstractRequest, - onResponseReceived: ResponseHandler): Unit = { + def postInboundRequest(request: AbstractRequest, onResponseReceived: ResponseHandler): Unit = { Review comment: Unrelated with this change but when reading the code. I was confused by the name `pollInboundResponses`. That function returns both pending request and responses. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org