weisong44 commented on a change in pull request #1031: SAMZA-2191: Batch support for table APIs URL: https://github.com/apache/samza/pull/1031#discussion_r284039492
########## File path: samza-core/src/main/java/org/apache/samza/table/batching/BatchReadWriteTable.java ########## @@ -0,0 +1,217 @@ +/* + * 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.samza.table.batching; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ScheduledExecutorService; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.samza.SamzaException; +import org.apache.samza.storage.kv.Entry; +import org.apache.samza.table.AsyncReadWriteTable; +import org.apache.samza.table.ReadWriteTable; +import org.apache.samza.table.batch.BatchPolicy; + +/** + * A wrapper of a {@link AsyncReadWriteTable} that supports batch operations. + * + * This batching table does not guarantee any ordering of different operation types within the batch. + * For instance, query(Q) and update(u) operations arrives in the following sequences, Q1, U1, Q2, U2, + * it does not mean the the remote data store will receive the messages in the same order. Instead, + * the operations will be grouped by type and sent via micro batches. For this sequence, Q1 and Q2 will + * be grouped to micro batch B1; U1 and U2 will be grouped to micro batch B2, the implementation class + * can decide the order of the micro batches. + * + * Synchronized table operations (get/put/delete) should be used with caution for the batching feature. + * If the table is used by a single thread, there will be at most one operation in the batch, and the + * batch will be performed when the TTL of the batch window expires. Batching does not make sense in this scenario. + * + * @param <K> The type of the key. + * @param <V> The type of the value. + */ +public class BatchReadWriteTable<K, V> implements ReadWriteTable<K, V> { + private final AsyncReadWriteTable<K, V> table; + private final BatchProcessor<K, V> batchProcessor; + + /** + * @param table The target table that serves the batch operations. + * @param batchPolicy Batch configs. + */ + public BatchReadWriteTable(AsyncReadWriteTable table, BatchPolicy batchPolicy, + ScheduledExecutorService batchTimerExecutorService) { + Preconditions.checkNotNull(table); + Preconditions.checkNotNull(batchPolicy); + Preconditions.checkNotNull(batchTimerExecutorService); + + this.table = table; + batchProcessor = new BatchProcessor<>(new TableBatchHandler<>(table), batchPolicy, batchTimerExecutorService); + } + + @Override + public V get(K key) { + Preconditions.checkNotNull(key); + + try { + return getAsync(key).get(); + } catch (Exception e) { + throw new SamzaException(e); + } + } + + @Override + public Map<K, V> getAll(List<K> keys) { + Preconditions.checkNotNull(keys); + + try { + return getAllAsync(keys).get(); + } catch (Exception e) { + throw new SamzaException(e); + } + } + + @Override + public void put(K key, V value) { + Preconditions.checkNotNull(key); + + try { + putAsync(key, value).get(); + } catch (Exception e) { + throw new SamzaException(e); + } + } + + @Override + public void putAll(List<Entry<K, V>> entries) { + Preconditions.checkNotNull(entries); + + try { + putAllAsync(entries).get(); + } catch (Exception e) { + throw new SamzaException(e); + } + } + + @Override + public void delete(K key) { + Preconditions.checkNotNull(key); + + try { + deleteAsync(key).get(); + } catch (Exception e) { + throw new SamzaException(e); + } + } + + @Override + public void deleteAll(List<K> keys) { + Preconditions.checkNotNull(keys); + + try { + deleteAllAsync(keys).get(); + } catch (Exception e) { + throw new SamzaException(e); + } + } + + @Override + public CompletableFuture<V> getAsync(K key) { + Preconditions.checkNotNull(key); + + try { + return batchProcessor.processQueryOperation(new GetOperation<>(key)); + } catch (Exception e) { + throw new SamzaException(e); + } + } + + @Override + public CompletableFuture<Map<K, V>> getAllAsync(List<K> keys) { Review comment: You don't need to provide an implementation, this call can just be passed to the underlying table. ---------------------------------------------------------------- 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: [email protected] With regards, Apache Git Services
