Addressed comments

This commit is contained in:
Wei 2017-10-06 13:34:57 -07:00
parent a2fd07c7b0
commit be34a50f59
13 changed files with 23 additions and 26 deletions

View file

@ -49,11 +49,6 @@ public class BlockingGetRecordsCache implements GetRecordsCache {
// //
} }
@Override
public void setMetricOperation(String operation){
}
@Override @Override
public ProcessRecordsInput getNextResult() { public ProcessRecordsInput getNextResult() {
sleepBeforeNextCall(); sleepBeforeNextCall();

View file

@ -41,6 +41,4 @@ public interface GetRecordsCache {
* This method calls the shutdown behavior on the cache, if available. * This method calls the shutdown behavior on the cache, if available.
*/ */
void shutdown(); void shutdown();
void setMetricOperation(String operation);
} }

View file

@ -83,7 +83,6 @@ class InitializeTask implements ITask {
ExtendedSequenceNumber initialCheckpoint = initialCheckpointObject.getCheckpoint(); ExtendedSequenceNumber initialCheckpoint = initialCheckpointObject.getCheckpoint();
dataFetcher.initialize(initialCheckpoint.getSequenceNumber(), streamConfig.getInitialPositionInStream()); dataFetcher.initialize(initialCheckpoint.getSequenceNumber(), streamConfig.getInitialPositionInStream());
getRecordsCache.setMetricOperation("InitializeTask");
getRecordsCache.start(); getRecordsCache.start();
recordProcessorCheckpointer.setLargestPermittedCheckpointValue(initialCheckpoint); recordProcessorCheckpointer.setLargestPermittedCheckpointValue(initialCheckpoint);
recordProcessorCheckpointer.setInitialCheckpointValue(initialCheckpoint); recordProcessorCheckpointer.setInitialCheckpointValue(initialCheckpoint);

View file

@ -78,7 +78,8 @@ public class PrefetchGetRecordsCache implements GetRecordsCache {
@NonNull final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, @NonNull final GetRecordsRetrievalStrategy getRecordsRetrievalStrategy,
@NonNull final ExecutorService executorService, @NonNull final ExecutorService executorService,
long idleMillisBetweenCalls, long idleMillisBetweenCalls,
@NonNull final IMetricsFactory metricsFactory) { @NonNull final IMetricsFactory metricsFactory,
@NonNull String operation) {
this.getRecordsRetrievalStrategy = getRecordsRetrievalStrategy; this.getRecordsRetrievalStrategy = getRecordsRetrievalStrategy;
this.maxRecordsPerCall = maxRecordsPerCall; this.maxRecordsPerCall = maxRecordsPerCall;
this.maxPendingProcessRecordsInput = maxPendingProcessRecordsInput; this.maxPendingProcessRecordsInput = maxPendingProcessRecordsInput;
@ -90,6 +91,7 @@ public class PrefetchGetRecordsCache implements GetRecordsCache {
this.metricsFactory = new ThreadSafeMetricsDelegatingFactory(metricsFactory); this.metricsFactory = new ThreadSafeMetricsDelegatingFactory(metricsFactory);
this.idleMillisBetweenCalls = idleMillisBetweenCalls; this.idleMillisBetweenCalls = idleMillisBetweenCalls;
this.defaultGetRecordsCacheDaemon = new DefaultGetRecordsCacheDaemon(); this.defaultGetRecordsCacheDaemon = new DefaultGetRecordsCacheDaemon();
this.operation = operation;
} }
@Override @Override
@ -129,10 +131,6 @@ public class PrefetchGetRecordsCache implements GetRecordsCache {
return getRecordsRetrievalStrategy; return getRecordsRetrievalStrategy;
} }
@Override
public void setMetricOperation(String operation) {
this.operation = operation;
}
@Override @Override
public void shutdown() { public void shutdown() {
defaultGetRecordsCacheDaemon.isShutdown = true; defaultGetRecordsCacheDaemon.isShutdown = true;

View file

@ -148,7 +148,6 @@ class ProcessTask implements ITask {
scope.addDimension(MetricsHelper.SHARD_ID_DIMENSION_NAME, shardInfo.getShardId()); scope.addDimension(MetricsHelper.SHARD_ID_DIMENSION_NAME, shardInfo.getShardId());
scope.addData(RECORDS_PROCESSED_METRIC, 0, StandardUnit.Count, MetricsLevel.SUMMARY); scope.addData(RECORDS_PROCESSED_METRIC, 0, StandardUnit.Count, MetricsLevel.SUMMARY);
scope.addData(DATA_BYTES_PROCESSED_METRIC, 0, StandardUnit.Bytes, MetricsLevel.SUMMARY); scope.addData(DATA_BYTES_PROCESSED_METRIC, 0, StandardUnit.Bytes, MetricsLevel.SUMMARY);
getRecordsCache.setMetricOperation("ProcessTask");
Exception exception = null; Exception exception = null;
try { try {

View file

@ -25,10 +25,12 @@ public interface RecordsFetcherFactory {
* *
* @param getRecordsRetrievalStrategy GetRecordsRetrievalStrategy to be used with the GetRecordsCache * @param getRecordsRetrievalStrategy GetRecordsRetrievalStrategy to be used with the GetRecordsCache
* @param shardId ShardId of the shard that the fetcher will retrieve records for * @param shardId ShardId of the shard that the fetcher will retrieve records for
* @param metricsFactory MetricsFactory used to create metricScope
* @param operation operation used to set metricScope
* *
* @return GetRecordsCache used to get records from Kinesis. * @return GetRecordsCache used to get records from Kinesis.
*/ */
GetRecordsCache createRecordsFetcher(GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, String shardId, IMetricsFactory metricsFactory); GetRecordsCache createRecordsFetcher(GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, String shardId, IMetricsFactory metricsFactory, String operation);
/** /**
* Sets the maximum number of ProcessRecordsInput objects the GetRecordsCache can hold, before further requests are * Sets the maximum number of ProcessRecordsInput objects the GetRecordsCache can hold, before further requests are

View file

@ -235,7 +235,7 @@ class ShardConsumer {
this.dataFetcher = kinesisDataFetcher; this.dataFetcher = kinesisDataFetcher;
this.getRecordsCache = config.getRecordsFetcherFactory().createRecordsFetcher( this.getRecordsCache = config.getRecordsFetcherFactory().createRecordsFetcher(
makeStrategy(this.dataFetcher, retryGetRecordsInSeconds, maxGetRecordsThreadPool, this.shardInfo), makeStrategy(this.dataFetcher, retryGetRecordsInSeconds, maxGetRecordsThreadPool, this.shardInfo),
this.getShardInfo().getShardId(), metricsFactory); this.getShardInfo().getShardId(), metricsFactory, "ProcessTask");
} }
/** /**

View file

@ -35,7 +35,7 @@ public class SimpleRecordsFetcherFactory implements RecordsFetcherFactory {
} }
@Override @Override
public GetRecordsCache createRecordsFetcher(GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, String shardId, IMetricsFactory metricsFactory) { public GetRecordsCache createRecordsFetcher(GetRecordsRetrievalStrategy getRecordsRetrievalStrategy, String shardId, IMetricsFactory metricsFactory, String operation) {
if(dataFetchingStrategy.equals(DataFetchingStrategy.DEFAULT)) { if(dataFetchingStrategy.equals(DataFetchingStrategy.DEFAULT)) {
return new BlockingGetRecordsCache(maxRecords, getRecordsRetrievalStrategy, idleMillisBetweenCalls); return new BlockingGetRecordsCache(maxRecords, getRecordsRetrievalStrategy, idleMillisBetweenCalls);
} else { } else {
@ -45,7 +45,7 @@ public class SimpleRecordsFetcherFactory implements RecordsFetcherFactory {
.setDaemon(true) .setDaemon(true)
.setNameFormat("prefetch-cache-" + shardId + "-%04d") .setNameFormat("prefetch-cache-" + shardId + "-%04d")
.build()), .build()),
idleMillisBetweenCalls, metricsFactory); idleMillisBetweenCalls, metricsFactory, operation);
} }
} }

View file

@ -66,6 +66,7 @@ public class PrefetchGetRecordsCacheIntegrationTest {
private KinesisDataFetcher dataFetcher; private KinesisDataFetcher dataFetcher;
private ExecutorService executorService; private ExecutorService executorService;
private List<Record> records; private List<Record> records;
private String operation = "ProcessTask";
@Mock @Mock
private IKinesisProxy proxy; private IKinesisProxy proxy;
@ -87,7 +88,8 @@ public class PrefetchGetRecordsCacheIntegrationTest {
getRecordsRetrievalStrategy, getRecordsRetrievalStrategy,
executorService, executorService,
IDLE_MILLIS_BETWEEN_CALLS, IDLE_MILLIS_BETWEEN_CALLS,
new NullMetricsFactory()); new NullMetricsFactory(),
operation);
} }
@Test @Test
@ -132,7 +134,8 @@ public class PrefetchGetRecordsCacheIntegrationTest {
getRecordsRetrievalStrategy2, getRecordsRetrievalStrategy2,
executorService2, executorService2,
IDLE_MILLIS_BETWEEN_CALLS, IDLE_MILLIS_BETWEEN_CALLS,
new NullMetricsFactory()); new NullMetricsFactory(),
operation);
getRecordsCache.start(); getRecordsCache.start();
sleep(IDLE_MILLIS_BETWEEN_CALLS); sleep(IDLE_MILLIS_BETWEEN_CALLS);

View file

@ -71,6 +71,7 @@ public class PrefetchGetRecordsCacheTest {
private ExecutorService executorService; private ExecutorService executorService;
private LinkedBlockingQueue<ProcessRecordsInput> spyQueue; private LinkedBlockingQueue<ProcessRecordsInput> spyQueue;
private PrefetchGetRecordsCache getRecordsCache; private PrefetchGetRecordsCache getRecordsCache;
private String operation = "ProcessTask";
@Before @Before
public void setup() { public void setup() {
@ -83,7 +84,8 @@ public class PrefetchGetRecordsCacheTest {
getRecordsRetrievalStrategy, getRecordsRetrievalStrategy,
executorService, executorService,
IDLE_MILLIS_BETWEEN_CALLS, IDLE_MILLIS_BETWEEN_CALLS,
new NullMetricsFactory()); new NullMetricsFactory(),
operation);
spyQueue = spy(getRecordsCache.getRecordsResultQueue); spyQueue = spy(getRecordsCache.getRecordsResultQueue);
records = spy(new ArrayList<>()); records = spy(new ArrayList<>());

View file

@ -13,6 +13,7 @@ import org.mockito.MockitoAnnotations;
public class RecordsFetcherFactoryTest { public class RecordsFetcherFactoryTest {
private String shardId = "TestShard"; private String shardId = "TestShard";
private RecordsFetcherFactory recordsFetcherFactory; private RecordsFetcherFactory recordsFetcherFactory;
private String operation = "ProcessTask";
@Mock @Mock
private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy; private GetRecordsRetrievalStrategy getRecordsRetrievalStrategy;
@ -29,7 +30,7 @@ public class RecordsFetcherFactoryTest {
@Test @Test
public void createDefaultRecordsFetcherTest() { public void createDefaultRecordsFetcherTest() {
GetRecordsCache recordsCache = recordsFetcherFactory.createRecordsFetcher(getRecordsRetrievalStrategy, shardId, GetRecordsCache recordsCache = recordsFetcherFactory.createRecordsFetcher(getRecordsRetrievalStrategy, shardId,
metricsFactory); metricsFactory, operation);
assertThat(recordsCache, instanceOf(BlockingGetRecordsCache.class)); assertThat(recordsCache, instanceOf(BlockingGetRecordsCache.class));
} }
@ -37,7 +38,7 @@ public class RecordsFetcherFactoryTest {
public void createPrefetchRecordsFetcherTest() { public void createPrefetchRecordsFetcherTest() {
recordsFetcherFactory.setDataFetchingStrategy(DataFetchingStrategy.PREFETCH_CACHED); recordsFetcherFactory.setDataFetchingStrategy(DataFetchingStrategy.PREFETCH_CACHED);
GetRecordsCache recordsCache = recordsFetcherFactory.createRecordsFetcher(getRecordsRetrievalStrategy, shardId, GetRecordsCache recordsCache = recordsFetcherFactory.createRecordsFetcher(getRecordsRetrievalStrategy, shardId,
metricsFactory); metricsFactory, operation);
assertThat(recordsCache, instanceOf(PrefetchGetRecordsCache.class)); assertThat(recordsCache, instanceOf(PrefetchGetRecordsCache.class));
} }

View file

@ -341,7 +341,7 @@ public class ShardConsumerTest {
getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords,
new SynchronousGetRecordsRetrievalStrategy(dataFetcher), new SynchronousGetRecordsRetrievalStrategy(dataFetcher),
0L)); 0L));
when(recordsFetcherFactory.createRecordsFetcher(any(), anyString(),any())).thenReturn(getRecordsCache); when(recordsFetcherFactory.createRecordsFetcher(any(), anyString(),any(),anyString())).thenReturn(getRecordsCache);
ShardConsumer consumer = ShardConsumer consumer =
new ShardConsumer(shardInfo, new ShardConsumer(shardInfo,
@ -471,7 +471,7 @@ public class ShardConsumerTest {
getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords, getRecordsCache = spy(new BlockingGetRecordsCache(maxRecords,
new SynchronousGetRecordsRetrievalStrategy(dataFetcher), new SynchronousGetRecordsRetrievalStrategy(dataFetcher),
0L)); 0L));
when(recordsFetcherFactory.createRecordsFetcher(any(), anyString(),any())).thenReturn(getRecordsCache); when(recordsFetcherFactory.createRecordsFetcher(any(), anyString(),any(),anyString())).thenReturn(getRecordsCache);
ShardConsumer consumer = ShardConsumer consumer =
new ShardConsumer(shardInfo, new ShardConsumer(shardInfo,

View file

@ -621,7 +621,7 @@ public class WorkerTest {
RecordsFetcherFactory recordsFetcherFactory = mock(RecordsFetcherFactory.class); RecordsFetcherFactory recordsFetcherFactory = mock(RecordsFetcherFactory.class);
GetRecordsCache getRecordsCache = mock(GetRecordsCache.class); GetRecordsCache getRecordsCache = mock(GetRecordsCache.class);
when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory); when(config.getRecordsFetcherFactory()).thenReturn(recordsFetcherFactory);
when(recordsFetcherFactory.createRecordsFetcher(any(), anyString(),any())).thenReturn(getRecordsCache); when(recordsFetcherFactory.createRecordsFetcher(any(), anyString(),any(), anyString())).thenReturn(getRecordsCache);
when(getRecordsCache.getNextResult()).thenReturn(new ProcessRecordsInput().withRecords(Collections.emptyList()).withMillisBehindLatest(0L)); when(getRecordsCache.getNextResult()).thenReturn(new ProcessRecordsInput().withRecords(Collections.emptyList()).withMillisBehindLatest(0L));
WorkerThread workerThread = runWorker(shardList, WorkerThread workerThread = runWorker(shardList,