@Override public CompletableFuture<Void> putAllAsync(List<Entry<K, V>> records) { Preconditions.checkNotNull(writeFn, "null write function"); Preconditions.checkNotNull(records, "null records"); if (records.isEmpty()) { return CompletableFuture.completedFuture(null); } List<K> deleteKeys = records.stream() .filter(e -> e.getValue() == null).map(Entry::getKey).collect(Collectors.toList()); List<Entry<K, V>> putRecords = records.stream() .filter(e -> e.getValue() != null).collect(Collectors.toList()); CompletableFuture<Void> deleteFuture = deleteKeys.isEmpty() ? CompletableFuture.completedFuture(null) : deleteAllAsync(deleteKeys); // Return the combined future return CompletableFuture.allOf( deleteFuture, instrument(() -> asyncTable.putAllAsync(putRecords), metrics.numPutAlls, metrics.putAllNs)) .exceptionally(e -> { String strKeys = records.stream().map(r -> r.getKey().toString()).collect(Collectors.joining(",")); throw new SamzaException(String.format("Failed to put records with keys=" + strKeys), e); }); }
@Override public CompletableFuture<Void> putAsync(K key, V value) { Preconditions.checkNotNull(writeFn, "null write function"); Preconditions.checkNotNull(key, "null key"); if (value == null) { return deleteAsync(key); } return instrument(() -> asyncTable.putAsync(key, value), metrics.numPuts, metrics.putNs) .exceptionally(e -> { throw new SamzaException("Failed to put a record with key=" + key, (Throwable) e); }); }
@Override public Map<K, V> getAll(List<K> keys) { try { return getAllAsync(keys).get(); } catch (Exception e) { throw new SamzaException(e); } }
private <K, V, T extends RemoteTable<K, V>> T getTable(String tableId, TableReadFunction<K, V> readFn, TableWriteFunction<K, V> writeFn, ExecutorService cbExecutor, boolean retry) { TableRateLimiter<K, V> readRateLimiter = mock(TableRateLimiter.class); TableRateLimiter<K, V> writeRateLimiter = mock(TableRateLimiter.class); TableRetryPolicy readPolicy = retry ? new TableRetryPolicy() : null; TableRetryPolicy writePolicy = retry ? new TableRetryPolicy() : null; ExecutorService rateLimitingExecutor = Executors.newSingleThreadExecutor(); ScheduledExecutorService retryExecutor = Executors.newSingleThreadScheduledExecutor(); RemoteTable<K, V> table = new RemoteTable(tableId, readFn, writeFn, readRateLimiter, writeRateLimiter, rateLimitingExecutor, readPolicy, writePolicy, retryExecutor, cbExecutor); table.init(getMockContext()); return (T) table; }
@Override public V get(K key) { try { return getAsync(key).get(); } catch (Exception e) { throw new SamzaException(e); } }
private void doTestGetAll(boolean sync, boolean error, boolean partial) throws Exception { TableReadFunction<String, String> readFn = mock(TableReadFunction.class); Map<String, String> res = new HashMap<>(); res.put("foo1", "bar1"); if (!partial) { res.put("foo2", "bar2"); } CompletableFuture<Map<String, String>> future; if (error) { future = new CompletableFuture(); future.completeExceptionally(new RuntimeException("Test exception")); } else { future = CompletableFuture.completedFuture(res); } // Sync is backed by async so needs to mock the async method doReturn(future).when(readFn).getAllAsync(any()); RemoteTable<String, String> table = getTable("testGetAll-" + sync + error + partial, readFn, null, false); Assert.assertEquals(res, sync ? table.getAll(Arrays.asList("foo1", "foo2")) : table.getAllAsync(Arrays.asList("foo1", "foo2")).get()); verify(table.readRateLimiter, times(1)).throttle(anyCollection()); }
private void doTestDelete(boolean sync, boolean error) throws Exception { TableWriteFunction<String, String> writeFn = mock(TableWriteFunction.class); RemoteTable<String, String> table = getTable("testDelete-" + sync + error, mock(TableReadFunction.class), writeFn, false); CompletableFuture<Void> future; if (error) { future = new CompletableFuture(); future.completeExceptionally(new RuntimeException("Test exception")); } else { future = CompletableFuture.completedFuture(null); } // Sync is backed by async so needs to mock the async method doReturn(future).when(writeFn).deleteAsync(any()); ArgumentCaptor<String> argCaptor = ArgumentCaptor.forClass(String.class); if (sync) { table.delete("foo"); } else { table.deleteAsync("foo").get(); } verify(writeFn, times(1)).deleteAsync(argCaptor.capture()); Assert.assertEquals("foo", argCaptor.getValue()); verify(table.writeRateLimiter, times(1)).throttle(anyString()); }
private void doTestGet(boolean sync, boolean error, boolean retry) throws Exception { String tableId = "testGet-" + sync + error + retry; TableReadFunction<String, String> readFn = mock(TableReadFunction.class); // Sync is backed by async so needs to mock the async method CompletableFuture<String> future; if (error) { future = new CompletableFuture(); future.completeExceptionally(new RuntimeException("Test exception")); if (!retry) { doReturn(future).when(readFn).getAsync(anyString()); } else { final int [] times = new int[] {0}; doAnswer(args -> times[0]++ == 0 ? future : CompletableFuture.completedFuture("bar")) .when(readFn).getAsync(anyString()); } } else { future = CompletableFuture.completedFuture("bar"); doReturn(future).when(readFn).getAsync(anyString()); } if (retry) { doReturn(true).when(readFn).isRetriable(any()); } RemoteTable<String, String> table = getTable(tableId, readFn, null, retry); Assert.assertEquals("bar", sync ? table.get("foo") : table.getAsync("foo").get()); verify(table.readRateLimiter, times(error && retry ? 2 : 1)).throttle(anyString()); }
public void doTestDeleteAll(boolean sync, boolean error) throws Exception { TableWriteFunction<String, String> writeFn = mock(TableWriteFunction.class); RemoteTable<String, String> table = getTable("testDeleteAll-" + sync + error, mock(TableReadFunction.class), writeFn, false); CompletableFuture<Void> future; if (error) { future = new CompletableFuture(); future.completeExceptionally(new RuntimeException("Test exception")); } else { future = CompletableFuture.completedFuture(null); } // Sync is backed by async so needs to mock the async method doReturn(future).when(writeFn).deleteAllAsync(any()); List<String> keys = Arrays.asList("foo1", "foo2"); ArgumentCaptor<List> argCaptor = ArgumentCaptor.forClass(List.class); if (sync) { table.deleteAll(keys); } else { table.deleteAllAsync(keys).get(); } verify(writeFn, times(1)).deleteAllAsync(argCaptor.capture()); Assert.assertEquals(keys, argCaptor.getValue()); verify(table.writeRateLimiter, times(1)).throttle(anyCollection()); }
ArgumentCaptor<List> argCaptor = ArgumentCaptor.forClass(List.class); if (sync) { table.putAll(entries); } else { table.putAllAsync(entries).get();
doReturn(CompletableFuture.completedFuture(null)).when(writeFn).deleteAsync(any()); final RemoteTable<String, String> remoteTable = new RemoteTable<>( tableId, readFn, writeFn, rateLimitHelper, rateLimitHelper, Executors.newSingleThreadExecutor(),
@Override public void putAll(List<Entry<K, V>> entries) { try { putAllAsync(entries).get(); } catch (Exception e) { throw new SamzaException(e); } }
@Override public void delete(K key) { try { deleteAsync(key).get(); } catch (Exception e) { throw new SamzaException(e); } }
@Override public void put(K key, V value) { try { putAsync(key, value).get(); } catch (Exception e) { throw new SamzaException(e); } }
@Override public void deleteAll(List<K> keys) { try { deleteAllAsync(keys).get(); } catch (Exception e) { throw new SamzaException(e); } }
@Override public void close() { super.close(); tables.forEach(t -> t.close()); rateLimitingExecutors.values().forEach(e -> e.shutdown()); callbackExecutors.values().forEach(e -> e.shutdown()); }
RemoteTable table = new RemoteTable(tableId, readFn, writeFn, readRateLimiter, writeRateLimiter, rateLimitingExecutors.get(tableId), readRetryPolicy, writeRetryPolicy, retryExecutor, callbackExecutors.get(tableId)); table.init(this.context); tables.add(table); return table;
@Test public void testGetMultipleTables() { TableReadFunction<String, String> readFn1 = mock(TableReadFunction.class); TableReadFunction<String, String> readFn2 = mock(TableReadFunction.class); // Sync is backed by async so needs to mock the async method doReturn(CompletableFuture.completedFuture("bar1")).when(readFn1).getAsync(anyString()); doReturn(CompletableFuture.completedFuture("bar2")).when(readFn1).getAsync(anyString()); RemoteTable<String, String> table1 = getTable("testGetMultipleTables-1", readFn1, null, false); RemoteTable<String, String> table2 = getTable("testGetMultipleTables-2", readFn2, null, false); CompletableFuture<String> future1 = table1.getAsync("foo1"); CompletableFuture<String> future2 = table2.getAsync("foo2"); CompletableFuture.allOf(future1, future2) .thenAccept(u -> { Assert.assertEquals(future1.join(), "bar1"); Assert.assertEquals(future2.join(), "bar1"); }); }
TableReadFunction<String, String> readFn = mock(TableReadFunction.class); TableWriteFunction<String, String> writeFn = mock(TableWriteFunction.class); final RemoteTable<String, String> remoteTable = new RemoteTable<>( tableId + "-remote", readFn, writeFn, rateLimitHelper, rateLimitHelper, Executors.newSingleThreadExecutor(),